zentol commented on a change in pull request #6477: [FLINK-10027] Add logging 
to StreamingFileSink
URL: https://github.com/apache/flink/pull/6477#discussion_r207221173
 
 

 ##########
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java
 ##########
 @@ -53,175 +53,234 @@
 
        private final RecoverableWriter fsWriter;
 
-       private final Map<Long, List<RecoverableWriter.CommitRecoverable>> 
pendingPerCheckpoint = new HashMap<>();
+       private final RollingPolicy<IN, BucketID> rollingPolicy;
 
-       private long partCounter;
-
-       private PartFileWriter<IN, BucketID> currentPart;
-
-       private List<RecoverableWriter.CommitRecoverable> pending;
-
-       /**
-        * Constructor to restore a bucket from checkpointed state.
-        */
-       public Bucket(
-                       RecoverableWriter fsWriter,
-                       int subtaskIndex,
-                       long initialPartCounter,
-                       PartFileWriter.PartFileFactory<IN, BucketID> 
partFileFactory,
-                       BucketState<BucketID> bucketState) throws IOException {
+       private final Map<Long, List<RecoverableWriter.CommitRecoverable>> 
pendingPartsPerCheckpoint = new HashMap<>();
 
-               this(fsWriter, subtaskIndex, bucketState.getBucketId(), 
bucketState.getBucketPath(), initialPartCounter, partFileFactory);
-
-               // the constructor must have already initialized the filesystem 
writer
-               Preconditions.checkState(fsWriter != null);
-
-               // we try to resume the previous in-progress file, if the 
filesystem
-               // supports such operation. If not, we just commit the file and 
start fresh.
+       private long partCounter;
 
-               final RecoverableWriter.ResumeRecoverable resumable = 
bucketState.getInProgress();
-               if (resumable != null) {
-                       currentPart = partFileFactory.resumeFrom(
-                                       bucketId, fsWriter, resumable, 
bucketState.getCreationTime());
-               }
+       private PartFileWriter<IN, BucketID> inProgressPart;
 
-               // we commit pending files for previous checkpoints to the last 
successful one
-               // (from which we are recovering from)
-               for (List<RecoverableWriter.CommitRecoverable> commitables: 
bucketState.getPendingPerCheckpoint().values()) {
-                       for (RecoverableWriter.CommitRecoverable commitable: 
commitables) {
-                               
fsWriter.recoverForCommit(commitable).commitAfterRecovery();
-                       }
-               }
-       }
+       private List<RecoverableWriter.CommitRecoverable> 
pendingPartsForCurrentCheckpoint;
 
        /**
         * Constructor to create a new empty bucket.
         */
-       public Bucket(
-                       RecoverableWriter fsWriter,
-                       int subtaskIndex,
-                       BucketID bucketId,
-                       Path bucketPath,
-                       long initialPartCounter,
-                       PartFileWriter.PartFileFactory<IN, BucketID> 
partFileFactory) {
+       private Bucket(
+                       final RecoverableWriter fsWriter,
+                       final int subtaskIndex,
+                       final BucketID bucketId,
+                       final Path bucketPath,
+                       final long initialPartCounter,
+                       final PartFileWriter.PartFileFactory<IN, BucketID> 
partFileFactory,
+                       final RollingPolicy<IN, BucketID> rollingPolicy) {
 
                this.fsWriter = Preconditions.checkNotNull(fsWriter);
                this.subtaskIndex = subtaskIndex;
                this.bucketId = Preconditions.checkNotNull(bucketId);
                this.bucketPath = Preconditions.checkNotNull(bucketPath);
                this.partCounter = initialPartCounter;
+               this.rollingPolicy = Preconditions.checkNotNull(rollingPolicy);
                this.partFileFactory = 
Preconditions.checkNotNull(partFileFactory);
 
-               this.pending = new ArrayList<>();
+               this.pendingPartsForCurrentCheckpoint = new ArrayList<>();
        }
 
        /**
-        * Gets the information available for the currently
-        * open part file, i.e. the one we are currently writing to.
-        *
-        * <p>This will be null if there is no currently open part file. This
-        * is the case when we have a new, just created bucket or a bucket
-        * that has not received any data after the closing of its previously
-        * open in-progress file due to the specified rolling policy.
-        *
-        * @return The information about the currently in-progress part file
-        * or {@code null} if there is no open part file.
+        * Constructor to restore a bucket from checkpointed state.
         */
-       public PartFileInfo<BucketID> getInProgressPartInfo() {
-               return currentPart;
+       private Bucket(
+                       final RecoverableWriter fsWriter,
+                       final int subtaskIndex,
+                       final long initialPartCounter,
+                       final PartFileWriter.PartFileFactory<IN, BucketID> 
partFileFactory,
+                       final RollingPolicy<IN, BucketID> rollingPolicy,
+                       final BucketState<BucketID> bucketState) throws 
IOException {
+
+               this(
+                               fsWriter,
+                               subtaskIndex,
+                               bucketState.getBucketId(),
+                               bucketState.getBucketPath(),
+                               initialPartCounter,
+                               partFileFactory,
+                               rollingPolicy);
+
+               restoreInProgressFile(bucketState);
+               commitRecoveredPendingFiles(bucketState);
+       }
+
+       private void restoreInProgressFile(final BucketState<BucketID> state) 
throws IOException {
+               if (state.hasInProgressResumableFile()) {
+                       final RecoverableWriter.ResumeRecoverable resumable = 
state.getInProgressResumableFile();
+                       inProgressPart = partFileFactory.resumeFrom(
+                                       bucketId, fsWriter, resumable, 
state.getInProgressFileCreationTime());
+               }
+       }
+
+       private void commitRecoveredPendingFiles(final BucketState<BucketID> 
state) throws IOException {
+               for (List<RecoverableWriter.CommitRecoverable> committables: 
state.getCommittableFilesPerCheckpoint().values()) {
+                       for (RecoverableWriter.CommitRecoverable committable: 
committables) {
+                               
fsWriter.recoverForCommit(committable).commitAfterRecovery();
+                       }
+               }
        }
 
-       public BucketID getBucketId() {
+       void merge(final Bucket<IN, BucketID> bucket) throws IOException {
+               Preconditions.checkNotNull(bucket);
+               Preconditions.checkState(Objects.equals(bucket.bucketPath, 
bucketPath));
+
+               // there should be no pending files in the "to-merge" states.
+               
Preconditions.checkState(bucket.pendingPartsForCurrentCheckpoint.isEmpty());
+               
Preconditions.checkState(bucket.pendingPartsPerCheckpoint.isEmpty());
+
+               RecoverableWriter.CommitRecoverable committable = 
bucket.closePartFile();
+               if (committable != null) {
+                       pendingPartsForCurrentCheckpoint.add(committable);
+               }
+       }
+
+       BucketID getBucketId() {
                return bucketId;
        }
 
-       public Path getBucketPath() {
+       Path getBucketPath() {
                return bucketPath;
        }
 
-       public long getPartCounter() {
+       long getPartCounter() {
                return partCounter;
        }
 
-       public boolean isActive() {
-               return currentPart != null || !pending.isEmpty() || 
!pendingPerCheckpoint.isEmpty();
+       boolean isActive() {
+               return inProgressPart != null || 
!pendingPartsForCurrentCheckpoint.isEmpty() || 
!pendingPartsPerCheckpoint.isEmpty();
        }
 
        void write(IN element, long currentTime) throws IOException {
-               Preconditions.checkState(currentPart != null, "bucket has been 
closed");
-               currentPart.write(element, currentTime);
+               if (inProgressPart == null || 
rollingPolicy.shouldRollOnEvent(inProgressPart, element)) {
+                       rollPartFile(currentTime);
+               }
+               inProgressPart.write(element, currentTime);
        }
 
-       void rollPartFile(final long currentTime) throws IOException {
+       private void rollPartFile(final long currentTime) throws IOException {
                closePartFile();
-               currentPart = partFileFactory.openNew(bucketId, fsWriter, 
getNewPartPath(), currentTime);
+               inProgressPart = partFileFactory.openNew(bucketId, fsWriter, 
assembleNewPartPath(), currentTime);
                partCounter++;
        }
 
-       void merge(final Bucket<IN, BucketID> bucket) throws IOException {
-               Preconditions.checkNotNull(bucket);
-               Preconditions.checkState(Objects.equals(bucket.getBucketPath(), 
bucketPath));
-
-               // there should be no pending files in the "to-merge" states.
-               Preconditions.checkState(bucket.pending.isEmpty());
-               Preconditions.checkState(bucket.pendingPerCheckpoint.isEmpty());
+       private RecoverableWriter.CommitRecoverable closePartFile() throws 
IOException {
+               RecoverableWriter.CommitRecoverable committable = null;
+               if (inProgressPart != null) {
+                       committable = inProgressPart.closeForCommit();
+                       pendingPartsForCurrentCheckpoint.add(committable);
+                       inProgressPart = null;
+               }
+               return committable;
+       }
 
-               RecoverableWriter.CommitRecoverable commitable = 
bucket.closePartFile();
-               if (commitable != null) {
-                       pending.add(commitable);
+       void disposePartFile() {
+               if (inProgressPart != null) {
+                       inProgressPart.dispose();
                }
        }
 
-       RecoverableWriter.CommitRecoverable closePartFile() throws IOException {
-               RecoverableWriter.CommitRecoverable commitable = null;
-               if (currentPart != null) {
-                       commitable = currentPart.closeForCommit();
-                       pending.add(commitable);
-                       currentPart = null;
+       BucketState<BucketID> onReceptionOfCheckpoint(long checkpointId) throws 
IOException {
+               prepareBucketForCheckpointing(checkpointId);
+
+               RecoverableWriter.ResumeRecoverable inProgressResumable = null;
+               long inProgressFileCreationTime = Long.MAX_VALUE;
+
+               if (inProgressPart != null) {
+                       inProgressResumable = inProgressPart.persist();
+                       inProgressFileCreationTime = 
inProgressPart.getCreationTime();
                }
-               return commitable;
+
+               return new BucketState<>(bucketId, bucketPath, 
inProgressFileCreationTime, inProgressResumable, pendingPartsPerCheckpoint);
        }
 
-       public void dispose() {
-               if (currentPart != null) {
-                       currentPart.dispose();
+       private void prepareBucketForCheckpointing(long checkpointId) throws 
IOException {
+               if (inProgressPart != null && 
rollingPolicy.shouldRollOnCheckpoint(inProgressPart)) {
+                       closePartFile();
+               }
+
+               if (!pendingPartsForCurrentCheckpoint.isEmpty()) {
+                       pendingPartsPerCheckpoint.put(checkpointId, 
pendingPartsForCurrentCheckpoint);
+                       pendingPartsForCurrentCheckpoint = new ArrayList<>();
                }
        }
 
-       public void onCheckpointAcknowledgment(long checkpointId) throws 
IOException {
+       void onSuccessfulCompletionOfCheckpoint(long checkpointId) throws 
IOException {
                Preconditions.checkNotNull(fsWriter);
 
                Iterator<Map.Entry<Long, 
List<RecoverableWriter.CommitRecoverable>>> it =
-                               pendingPerCheckpoint.entrySet().iterator();
+                               pendingPartsPerCheckpoint.entrySet().iterator();
 
                while (it.hasNext()) {
                        Map.Entry<Long, 
List<RecoverableWriter.CommitRecoverable>> entry = it.next();
+
                        if (entry.getKey() <= checkpointId) {
-                               for (RecoverableWriter.CommitRecoverable 
commitable : entry.getValue()) {
-                                       
fsWriter.recoverForCommit(commitable).commit();
+                               for (RecoverableWriter.CommitRecoverable 
committable : entry.getValue()) {
+                                       
fsWriter.recoverForCommit(committable).commit();
                                }
                                it.remove();
                        }
                }
        }
 
-       public BucketState<BucketID> onCheckpoint(long checkpointId) throws 
IOException {
-               RecoverableWriter.ResumeRecoverable resumable = null;
-               long creationTime = Long.MAX_VALUE;
-
-               if (currentPart != null) {
-                       resumable = currentPart.persist();
-                       creationTime = currentPart.getCreationTime();
-               }
-
-               if (!pending.isEmpty()) {
-                       pendingPerCheckpoint.put(checkpointId, pending);
-                       pending = new ArrayList<>();
+       void rollOnProcessingTimeIfNeeded(long timestamp) throws IOException {
+               if (inProgressPart != null && 
rollingPolicy.shouldRollOnProcessingTime(inProgressPart, timestamp)) {
+                       closePartFile();
                }
-               return new BucketState<>(bucketId, bucketPath, creationTime, 
resumable, pendingPerCheckpoint);
        }
 
-       private Path getNewPartPath() {
+       private Path assembleNewPartPath() {
                return new Path(bucketPath, PART_PREFIX + '-' + subtaskIndex + 
'-' + partCounter);
        }
+
+       // --------------------------- Static Factory Methods 
-----------------------------
+
+       /**
+        * Creates a new empty {@code Bucket}.
+        * @param fsWriter the filesystem-specific {@link RecoverableWriter}.
+        * @param subtaskIndex the index of the subtask creating the bucket.
+        * @param bucketId the identifier of the bucket, as returned by the 
{@link BucketAssigner}.
+        * @param bucketPath the path to where the part files for the bucket 
will be written to.
+        * @param initialPartCounter the initial counter for the part files of 
the bucket.
+        * @param partFileFactory the {@link PartFileWriter.PartFileFactory} 
the factory creating part file writers.
+        * @param <IN> the type of input elements to the sink.
+        * @param <BucketID> the type of the identifier of the bucket, as 
returned by the {@link BucketAssigner}
+        * @return The new Bucket.
+        */
+       static <IN, BucketID> Bucket<IN, BucketID> getNew(
+                       final RecoverableWriter fsWriter,
+                       final int subtaskIndex,
+                       final BucketID bucketId,
+                       final Path bucketPath,
+                       final long initialPartCounter,
+                       final PartFileWriter.PartFileFactory<IN, BucketID> 
partFileFactory,
+                       final RollingPolicy<IN, BucketID> rollingPolicy) {
+               return new Bucket<>(fsWriter, subtaskIndex, bucketId, 
bucketPath, initialPartCounter, partFileFactory, rollingPolicy);
+       }
+
+       /**
+        * Restores a {@code Bucket} from the state included in the provided 
{@link BucketState}.
+        * @param fsWriter the filesystem-specific {@link RecoverableWriter}.
+        * @param subtaskIndex the index of the subtask creating the bucket.
+        * @param initialPartCounter the initial counter for the part files of 
the bucket.
+        * @param partFileFactory the {@link PartFileWriter.PartFileFactory} 
the factory creating part file writers.
+        * @param bucketState the initial state of the restored bucket.
+        * @param <IN> the type of input elements to the sink.
+        * @param <BucketID> the type of the identifier of the bucket, as 
returned by the {@link BucketAssigner}
+        * @return The new Bucket.
 
 Review comment:
   We could say "The restored Bucket."

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to