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

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

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

    https://github.com/apache/flink/pull/3522#discussion_r108718168
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStream.java
 ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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.state.filesystem;
    +
    +import org.apache.flink.core.fs.FSDataOutputStream;
    +import org.apache.flink.core.fs.FileSystem;
    +import org.apache.flink.core.fs.FileSystem.WriteMode;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.runtime.state.CheckpointStreamFactory;
    +import org.apache.flink.runtime.state.StreamStateHandle;
    +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
    +import org.apache.flink.util.FileUtils;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.UUID;
    +
    +/**
    + * A {@link CheckpointStreamFactory.CheckpointStateOutputStream} that 
writes into a file and
    + * returns a {@link StreamStateHandle} for the written data upon closing.
    + * 
    + * <p>This stream prevents the creation of very small files with sizes 
below a configurable
    + * threshold. If the written data is below that threshold, the returned 
handle is a
    + * {@link ByteStreamStateHandle} that directly contains the data. If the 
written data size is above
    + * the threshold, the data is written to a file and the returned handle is 
a {@link FileStateHandle}.
    + *
    + * <p>Note: Flushing the stream always creates the file. 
    + */
    +public final class FsCheckpointStateOutputStream extends 
CheckpointStreamFactory.CheckpointStateOutputStream {
    +
    +   private static final Logger LOG = 
LoggerFactory.getLogger(FsCheckpointStateOutputStream.class);
    +
    +   // 
------------------------------------------------------------------------
    +
    +   private final byte[] writeBuffer;
    +
    +   private int pos;
    +
    +   private FSDataOutputStream outStream;
    +
    +   private final int localStateThreshold;
    +
    +   private final Path basePath;
    +
    +   private final FileSystem fs;
    +
    +   private Path statePath;
    +
    +   private volatile boolean closed;
    +
    +   public FsCheckpointStateOutputStream(
    +                           Path basePath, FileSystem fs,
    +                           int bufferSize, int localStateThreshold)
    +   {
    +           if (bufferSize < localStateThreshold) {
    +                   throw new IllegalArgumentException();
    +           }
    +
    +           this.basePath = basePath;
    +           this.fs = fs;
    --- End diff --
    
    `checkNotNull` checks could be inserted here.


> Store Checkpoint Root Metadata in StateBackend (not in HA custom store)
> -----------------------------------------------------------------------
>
>                 Key: FLINK-5823
>                 URL: https://issues.apache.org/jira/browse/FLINK-5823
>             Project: Flink
>          Issue Type: Sub-task
>          Components: State Backends, Checkpointing
>            Reporter: Stephan Ewen
>            Assignee: Stephan Ewen
>             Fix For: 1.3.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to