[
https://issues.apache.org/jira/browse/FLINK-7068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16155562#comment-16155562
]
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_r137270473
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java ---
@@ -389,95 +413,332 @@ public File getFile(JobID jobId, BlobKey key) throws
IOException {
*
* @param jobId
* ID of the job this blob belongs to (or <tt>null</tt> if
job-unrelated)
- * @param requiredBlob
+ * @param blobKey
* blob key associated with the requested file
+ * @param highlyAvailable
+ * whether to the requested file is highly available (HA)
*
* @return file referring to the local storage location of the BLOB
*
* @throws IOException
* Thrown if the file retrieval failed.
*/
- private File getFileInternal(@Nullable JobID jobId, BlobKey
requiredBlob) throws IOException {
- checkArgument(requiredBlob != null, "BLOB key cannot be null.");
+ private File getFileInternal(@Nullable JobID jobId, BlobKey blobKey,
boolean highlyAvailable) throws IOException {
--- End diff --
Maybe we could introduce an enum here as well for the `highlyAvailable`
boolean argument.
> 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)