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

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_r142138008
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobCache.java
 ---
    @@ -0,0 +1,290 @@
    +/*
    + * 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.annotation.VisibleForTesting;
    +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.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.net.InetSocketAddress;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * Provides a cache for permanent BLOB files including a per-job 
ref-counting and a staged cleanup.
    + *
    + * <p>When requesting BLOBs via {@link #getPermanentFile(JobID, BlobKey)}, 
the cache will first attempt to
    + * serve the file from its local cache. Only if the local cache does not 
contain the desired BLOB,
    + * it will try to download it from a distributed HA file system (if 
available) or the BLOB server.
    + *
    + * <p>If files for a job are not needed any more, they will enter a 
staged, i.e. deferred, cleanup.
    + * Files may thus still be be accessible upon recovery and do not need to 
be re-downloaded.
    + */
    +public class PermanentBlobCache extends AbstractBlobCache implements 
PermanentBlobService {
    +
    +   /**
    +    * Job reference counters with a time-to-live (TTL).
    +    */
    +   @VisibleForTesting
    +   static class RefCount {
    +           /**
    +            * Number of references to a job.
    +            */
    +           public int references = 0;
    +
    +           /**
    +            * Timestamp in milliseconds when any job data should be 
cleaned up (no cleanup for
    +            * non-positive values).
    +            */
    +           public long keepUntil = -1;
    +   }
    +
    +   /**
    +    * Map to store the number of references to a specific job.
    +    */
    +   private final Map<JobID, RefCount> jobRefCounters = new HashMap<>();
    +
    +   /**
    +    * Time interval (ms) to run the cleanup task; also used as the default 
TTL.
    +    */
    +   private final long cleanupInterval;
    +
    +   private final Timer cleanupTimer;
    +
    +   /**
    +    * Instantiates a new cache for permanent BLOBs which are also 
available in an HA store.
    +    *
    +    * @param serverAddress
    +    *              address of the {@link BlobServer} to use for fetching 
files from
    +    * @param blobClientConfig
    +    *              global configuration
    +    * @param blobView
    +    *              (distributed) HA blob store file system to retrieve 
files from first
    +    *
    +    * @throws IOException
    +    *              thrown if the (local or distributed) file storage 
cannot be created or is not usable
    +    */
    +   public PermanentBlobCache(
    +                   final InetSocketAddress serverAddress,
    +                   final Configuration blobClientConfig,
    +                   final BlobView blobView) throws IOException {
    +
    +           super(serverAddress, blobClientConfig, blobView,
    +                   LoggerFactory.getLogger(PermanentBlobCache.class));
    +
    +           // Initializing the clean up task
    +           this.cleanupTimer = new Timer(true);
    +
    +           this.cleanupInterval = 
blobClientConfig.getLong(BlobServerOptions.CLEANUP_INTERVAL) * 1000;
    +           this.cleanupTimer.schedule(new PermanentBlobCleanupTask(), 
cleanupInterval, cleanupInterval);
    +   }
    +
    +   /**
    +    * Registers use of job-related BLOBs.
    +    *
    +    * <p>Using any other method to access BLOBs, e.g. {@link 
#getPermanentFile}, is only valid within
    +    * calls to <tt>registerJob(JobID)</tt> and {@link #releaseJob(JobID)}.
    +    *
    +    * @param jobId
    +    *              ID of the job this blob belongs to
    +    *
    +    * @see #releaseJob(JobID)
    +    */
    +   public void registerJob(JobID jobId) {
    +           checkNotNull(jobId);
    +
    +           synchronized (jobRefCounters) {
    +                   RefCount ref = jobRefCounters.get(jobId);
    +                   if (ref == null) {
    +                           ref = new RefCount();
    +                           jobRefCounters.put(jobId, ref);
    +                   } else {
    +                           // reset cleanup timeout
    +                           ref.keepUntil = -1;
    +                   }
    +                   ++ref.references;
    +           }
    +   }
    +
    +   /**
    +    * Unregisters use of job-related BLOBs and allow them to be released.
    +    *
    +    * @param jobId
    +    *              ID of the job this blob belongs to
    +    *
    +    * @see #registerJob(JobID)
    +    */
    +   public void releaseJob(JobID jobId) {
    +           checkNotNull(jobId);
    +
    +           synchronized (jobRefCounters) {
    +                   RefCount ref = jobRefCounters.get(jobId);
    +
    +                   if (ref == null || ref.references == 0) {
    +                           LOG.warn("improper use of releaseJob() without 
a matching number of registerJob() calls for jobId " + jobId);
    +                           return;
    +                   }
    +
    +                   --ref.references;
    +                   if (ref.references == 0) {
    +                           ref.keepUntil = System.currentTimeMillis() + 
cleanupInterval;
    +                   }
    +           }
    +   }
    +
    +   public int getNumberOfReferenceHolders(JobID jobId) {
    +           checkNotNull(jobId);
    +
    +           synchronized (jobRefCounters) {
    +                   RefCount ref = jobRefCounters.get(jobId);
    +                   if (ref == null) {
    +                           return 0;
    +                   } else {
    +                           return ref.references;
    +                   }
    +           }
    +   }
    +
    +   public int getNumberOfCachedJobs() {
    +           return jobRefCounters.size();
    +   }
    +
    +   /**
    +    * Returns the path to a local copy of the file associated with the 
provided job ID and blob
    +    * key.
    +    *
    +    * <p>We will first attempt to serve the BLOB from the local storage. 
If the BLOB is not in
    +    * there, we will try to download it from the HA store, or directly 
from the {@link BlobServer}.
    +    *
    +    * @param jobId
    +    *              ID of the job this blob belongs to
    +    * @param key
    +    *              blob key associated with the requested file
    +    *
    +    * @return The path to the file.
    +    *
    +    * @throws java.io.FileNotFoundException
    +    *              if the BLOB does not exist;
    +    * @throws IOException
    +    *              if any other error occurs when retrieving the file
    +    */
    +   @Override
    +   public File getPermanentFile(JobID jobId, BlobKey key) throws 
IOException {
    +           checkNotNull(jobId);
    +           return getTransientFileInternal(jobId, key);
    --- End diff --
    
    Maybe we could rename `getTransientFileInternal` into `getFileInternal`.


> 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