[
https://issues.apache.org/jira/browse/FLINK-7068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16155560#comment-16155560
]
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_r137287126
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobService.java
---
@@ -0,0 +1,54 @@
+/*
+ * 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 java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * A service to retrieve permanent binary large objects (BLOBs).
+ * <p>
+ * These include per-job BLOBs that are covered by high-availability (HA)
mode, e.g. a job's JAR
+ * files, parts of an off-loaded {@link
org.apache.flink.runtime.deployment.TaskDeploymentDescriptor}
+ * or files in the {@link
org.apache.flink.api.common.cache.DistributedCache}.
+ */
+public interface PermanentBlobService extends Closeable {
+
+ /**
+ * Returns the path to a local copy of the file associated with the
provided job ID and blob
+ * key.
+ *
+ * @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
+ */
+ File getHAFile(JobID jobId, BlobKey key) throws IOException;
--- End diff --
Not sure whether this is the right name because HA does not depend on the
`PermanentBlobService` but on the the `BlobStore`. I would suggest to rename it.
> 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)