zentol commented on a change in pull request #6477: [FLINK-10027] Add logging
to StreamingFileSink
URL: https://github.com/apache/flink/pull/6477#discussion_r207226949
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java
##########
@@ -53,175 +58,260 @@
private final RecoverableWriter fsWriter;
- private final Map<Long, List<RecoverableWriter.CommitRecoverable>>
pendingPerCheckpoint = new HashMap<>();
-
- private long partCounter;
-
- private PartFileWriter<IN, BucketID> currentPart;
+ private final RollingPolicy<IN, BucketID> rollingPolicy;
- 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();
+ }
+ }
+ }
+
+ 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);
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Subtask {} merging buckets for bucket
id={}", subtaskIndex, bucketId);
+ }
}
- public BucketID getBucketId() {
+ 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)) {
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Subtask {} closing in-progress part
file for bucket id={} due to element {}.",
+ subtaskIndex, bucketId,
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);
+
+ final Path partFilePath = assembleNewPartPath();
+ inProgressPart = partFileFactory.openNew(bucketId, fsWriter,
partFilePath, currentTime);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Subtask {} opening new part file \"{}\" for
bucket id={}.",
+ subtaskIndex, partFilePath.getName(),
bucketId);
+ }
+
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)) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Subtask {} closing in-progress part
file for bucket id={} on checkpoint.", subtaskIndex, bucketId);
+ }
+ 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)) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Subtask {} closing in-progress part
file for bucket id={} due to timeout " +
+ "(in-progress file created @
{}, last updated @ {} and now is {}).",
+ subtaskIndex, bucketId,
inProgressPart.getCreationTime(), inProgressPart.getLastUpdateTime(),
timestamp);
+ }
+ closePartFile();
}
- return new BucketState<>(bucketId, bucketPath, creationTime,
resumable, pendingPerCheckpoint);
}
- private Path getNewPartPath() {
+ private Path assembleNewPartPath() {
Review comment:
move below closePartFile so it's closer to where it is used. We could even
in-line it since it's only used once.
----------------------------------------------------------------
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