[ 
https://issues.apache.org/jira/browse/FLINK-7068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16188039#comment-16188039
 ] 

ASF GitHub Bot commented on FLINK-7068:
---------------------------------------

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4358#discussion_r142126654
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/blob/AbstractBlobCache.java
 ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.blob;
    +
    +import org.apache.flink.api.common.JobID;
    +import org.apache.flink.configuration.BlobServerOptions;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.util.FileUtils;
    +
    +import org.slf4j.Logger;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.File;
    +import java.io.IOException;
    +import java.net.InetSocketAddress;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.ReadWriteLock;
    +import java.util.concurrent.locks.ReentrantReadWriteLock;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * Abstract base class for permanent and transient BLOB files.
    + */
    +public abstract class AbstractBlobCache implements Closeable {
    +
    +   /**
    +    * The log object used for debugging.
    +    */
    +   protected final Logger LOG;
    +
    +   /**
    +    * Counter to generate unique names for temporary files.
    +    */
    +   protected final AtomicLong tempFileCounter = new AtomicLong(0);
    +
    +   protected final InetSocketAddress serverAddress;
    +
    +   /**
    +    * Root directory for local file storage.
    +    */
    +   protected final File storageDir;
    +
    +   /**
    +    * Blob store for distributed file storage, e.g. in HA.
    +    */
    +   protected final BlobView blobView;
    +
    +   protected final AtomicBoolean shutdownRequested = new AtomicBoolean();
    +
    +   /**
    +    * Shutdown hook thread to ensure deletion of the local storage 
directory.
    +    */
    +   protected final Thread shutdownHook;
    +
    +   /**
    +    * The number of retries when the transfer fails.
    +    */
    +   protected final int numFetchRetries;
    +
    +   /**
    +    * Configuration for the blob client like ssl parameters required to 
connect to the blob
    +    * server.
    +    */
    +   protected final Configuration blobClientConfig;
    +
    +   /**
    +    * Lock guarding concurrent file accesses.
    +    */
    +   protected final ReadWriteLock readWriteLock;
    +
    +   public AbstractBlobCache(
    +                   final InetSocketAddress serverAddress,
    +                   final Configuration blobClientConfig,
    +                   final BlobView blobView,
    +                   final Logger logger) throws IOException {
    +
    +           this.LOG = logger;
    +
    +           this.serverAddress = checkNotNull(serverAddress);
    +           this.blobClientConfig = checkNotNull(blobClientConfig);
    +           this.blobView = checkNotNull(blobView, "blobStore");
    --- End diff --
    
    `blobStore` != `blobView`


> change BlobService sub-classes for permanent and transient BLOBs
> ----------------------------------------------------------------
>
>                 Key: FLINK-7068
>                 URL: https://issues.apache.org/jira/browse/FLINK-7068
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Distributed Coordination, Network
>    Affects Versions: 1.4.0
>            Reporter: Nico Kruber
>            Assignee: Nico Kruber
>
> A {{PermanentBlobStore}} should resemble use cases for BLOBs that are 
> permanently stored for a job's life time (HA and non-HA).
> A {{TransientBlobStore}} should reflect BLOB offloading for logs, RPC, etc. 
> which even does not have to be reflected by files.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to