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

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_r137266916
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java ---
    @@ -220,19 +260,27 @@ InputStream getInternal(@Nullable JobID jobId, 
BlobKey blobKey) throws IOExcepti
         *              ID of the job this blob belongs to (or <tt>null</tt> if 
job-unrelated)
         * @param blobKey
         *              blob key associated with the requested file
    +    * @param permanentBlob
    +    *              whether the BLOB is permanent (<tt>true</tt>) or 
transient (<tt>false</tt>)
         *
         * @throws IOException
         *         thrown if an I/O error occurs while writing the header data 
to the output stream
         */
    -   private static void sendGetHeader(OutputStream outputStream, @Nullable 
JobID jobId, BlobKey blobKey) throws IOException {
    +   private static void sendGetHeader(
    +                   OutputStream outputStream, @Nullable JobID jobId, 
BlobKey blobKey, boolean permanentBlob)
    +                   throws IOException {
                checkNotNull(blobKey);
    +           checkArgument(jobId != null || !permanentBlob, "permanent BLOBs 
must be job-related");
     
                // Signal type of operation
                outputStream.write(GET_OPERATION);
     
                // Send job ID and key
                if (jobId == null) {
                        outputStream.write(CONTENT_NO_JOB);
    --- End diff --
    
    Should we rename this variable to `TRANSIENT_CONTENT`?


> 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