Skip to content

Commit

Permalink
[FLINK-25941][streaming] Only emit committables with Long.MAX_VALUE a…
Browse files Browse the repository at this point in the history
…s checkpoint id in batch mode

Before this commit the SinkWriter and Committer operators emitted
committables on endInput. This was troublesome because by doing so the
checkpointId was set to effectively Long.MAX_VALUE because
the emission was not part of any checkpoint. With the completion of
FLIP-143 all jobs in streaming mode have a final checkpoint when they
transition to finish so we can rely on the normal checkpoint mechanism
and only need endInput for the batch execution.
  • Loading branch information
fapaul committed Feb 22, 2022
1 parent ec46816 commit de46599
Show file tree
Hide file tree
Showing 13 changed files with 266 additions and 69 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -114,15 +114,6 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception {
commit(lastCompletedCheckpointId);
}

private Collection<? extends CheckpointCommittableManager<CommT>> getCommittables() {
final Collection<? extends CheckpointCommittableManager<CommT>> committables =
committableCollector.getEndOfInputCommittables();
if (committables == null) {
return Collections.emptyList();
}
return committables;
}

private Collection<? extends CheckpointCommittableManager<CommT>> getCommittables(
long checkpointId) {
final Collection<? extends CheckpointCommittableManager<CommT>> committables =
Expand All @@ -142,11 +133,13 @@ private void commit(long checkpointId) throws IOException, InterruptedException

@Override
public void endInput() throws Exception {
do {
for (CommittableManager<CommT> committable : getCommittables()) {
committable.commit(false, committer);
}
} while (!committableCollector.isFinished());
final CommittableManager<CommT> endOfInputCommittable =
committableCollector.getEndOfInputCommittable();
if (endOfInputCommittable != null) {
do {
endOfInputCommittable.commit(false, committer);
} while (!committableCollector.isFinished());
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,9 +64,13 @@ class CommitterOperator<CommT> extends AbstractStreamOperator<CommittableMessage
private final SimpleVersionedSerializer<CommT> committableSerializer;
private final Committer<CommT> committer;
private final boolean emitDownstream;
private final boolean isCheckpointingOrBatchModeEnabled;
private CommittableCollector<CommT> committableCollector;
private long lastCompletedCheckpointId = -1;

private boolean endInput = false;
private boolean finalEmission = false;

/** The operator's state descriptor. */
private static final ListStateDescriptor<byte[]> STREAMING_COMMITTER_RAW_STATES_DESC =
new ListStateDescriptor<>(
Expand All @@ -79,8 +83,10 @@ public CommitterOperator(
ProcessingTimeService processingTimeService,
SimpleVersionedSerializer<CommT> committableSerializer,
Committer<CommT> committer,
boolean emitDownstream) {
boolean emitDownstream,
boolean isCheckpointingOrBatchModeEnabled) {
this.emitDownstream = emitDownstream;
this.isCheckpointingOrBatchModeEnabled = isCheckpointingOrBatchModeEnabled;
this.processingTimeService = checkNotNull(processingTimeService);
this.committableSerializer = checkNotNull(committableSerializer);
this.committer = checkNotNull(committer);
Expand Down Expand Up @@ -123,20 +129,31 @@ public void snapshotState(StateSnapshotContext context) throws Exception {

@Override
public void endInput() throws Exception {
Collection<? extends CommittableManager<CommT>> endOfInputCommittables =
committableCollector.getEndOfInputCommittables();
endInput = true;
final CommittableManager<CommT> endOfInputCommittable =
committableCollector.getEndOfInputCommittable();
// indicates batch
if (endOfInputCommittables != null) {
if (endOfInputCommittable != null) {
do {
for (CommittableManager<CommT> endOfInputCommittable : endOfInputCommittables) {
commitAndEmit(endOfInputCommittable, false);
}
commitAndEmit(endOfInputCommittable, false);
} while (!committableCollector.isFinished());
}
if (!isCheckpointingOrBatchModeEnabled) {
notifyCheckpointComplete(
lastCompletedCheckpointId == -1 ? 1 : lastCompletedCheckpointId + 1);
}
}

@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {
// If a streaming job finishes and a savepoint is triggered afterwards we do not want to
// flush again
if (finalEmission) {
return;
}
if (endInput) {
finalEmission = true;
}
super.notifyCheckpointComplete(checkpointId);
lastCompletedCheckpointId = Math.max(lastCompletedCheckpointId, checkpointId);
commitAndEmitCheckpoints();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,12 @@ public final class CommitterOperatorFactory<CommT>
CommittableMessage<CommT>, CommittableMessage<CommT>> {

private final TwoPhaseCommittingSink<?, CommT> sink;
private final boolean isCheckpointingOrBatchModeEnabled;

public CommitterOperatorFactory(TwoPhaseCommittingSink<?, CommT> sink) {
public CommitterOperatorFactory(
TwoPhaseCommittingSink<?, CommT> sink, boolean isCheckpointingOrBatchModeEnabled) {
this.sink = checkNotNull(sink);
this.isCheckpointingOrBatchModeEnabled = isCheckpointingOrBatchModeEnabled;
}

@Override
Expand All @@ -58,7 +61,8 @@ public <T extends StreamOperator<CommittableMessage<CommT>>> T createStreamOpera
processingTimeService,
sink.getCommittableSerializer(),
sink.createCommitter(),
sink instanceof WithPostCommitTopology);
sink instanceof WithPostCommitTopology,
isCheckpointingOrBatchModeEnabled);
committerOperator.setup(
parameters.getContainingTask(),
parameters.getStreamConfig(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,8 @@ class SinkWriterOperator<InputT, CommT> extends AbstractStreamOperator<Committab
private final Context<InputT> context;

private final boolean emitDownstream;
private final boolean isBatchMode;
private final boolean isCheckpointingEnabled;

// ------------------------------- runtime fields ---------------------------------------

Expand All @@ -78,17 +80,23 @@ class SinkWriterOperator<InputT, CommT> extends AbstractStreamOperator<Committab

private SinkWriter<InputT> sinkWriter;

private OptionalLong restoredCheckpointId = OptionalLong.empty();

private final SinkWriterStateHandler<InputT> writerStateHandler;

private final MailboxExecutor mailboxExecutor;
// record endOfInput state to avoid duplicate prepareCommit on final notifyCheckpointComplete
// once FLIP-147 is fully operational all endOfInput processing needs to be removed

private boolean endOfInput = false;
private boolean finalEmission = false;

SinkWriterOperator(
Sink<InputT> sink,
ProcessingTimeService processingTimeService,
MailboxExecutor mailboxExecutor) {
MailboxExecutor mailboxExecutor,
boolean isBatchMode,
boolean isCheckpointingEnabled) {
this.isBatchMode = isBatchMode;
this.isCheckpointingEnabled = isCheckpointingEnabled;
this.processingTimeService = checkNotNull(processingTimeService);
this.mailboxExecutor = checkNotNull(mailboxExecutor);
this.context = new Context<>();
Expand All @@ -106,6 +114,7 @@ class SinkWriterOperator<InputT, CommT> extends AbstractStreamOperator<Committab
public void initializeState(StateInitializationContext context) throws Exception {
super.initializeState(context);
OptionalLong checkpointId = context.getRestoredCheckpointId();
restoredCheckpointId = checkpointId;
InitContext initContext =
createInitContext(checkpointId.isPresent() ? checkpointId.getAsLong() : null);

Expand All @@ -126,11 +135,17 @@ public void processElement(StreamRecord<InputT> element) throws Exception {

@Override
public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
super.prepareSnapshotPreBarrier(checkpointId);
if (!endOfInput) {
sinkWriter.flush(false);
emitCommittables(checkpointId);
// If a streaming job finishes and a savepoint is triggered afterwards we do not want to
// flush again
if (finalEmission) {
return;
}
if (endOfInput) {
finalEmission = true;
}
super.prepareSnapshotPreBarrier(checkpointId);
sinkWriter.flush(endOfInput);
emitCommittables(checkpointId);
}

@Override
Expand All @@ -144,8 +159,18 @@ public void processWatermark(Watermark mark) throws Exception {
@Override
public void endInput() throws Exception {
endOfInput = true;
sinkWriter.flush(true);
emitCommittables(Long.MAX_VALUE);
// Only in batch mode we want to emit with the Long.MAX_VALUE checkpoint id. In streaming
// mode there will be a final checkpoint after endInput that flushes all pending
// committables.
if (isBatchMode) {
sinkWriter.flush(true);
emitCommittables(Long.MAX_VALUE);
return;
}
// There will be no final checkpoint but the job runs in streaming mode, so we try to commit
if (!isCheckpointingEnabled) {
prepareSnapshotPreBarrier(restoredCheckpointId.orElse(0) + 1);
}
}

private void emitCommittables(Long checkpointId) throws IOException, InterruptedException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,16 +45,26 @@ public final class SinkWriterOperatorFactory<InputT, CommT>
YieldingOperatorFactory<CommittableMessage<CommT>> {

private final Sink<InputT> sink;
private final boolean isBatchMode;
private final boolean isCheckpointingEnabled;

public SinkWriterOperatorFactory(Sink<InputT> sink) {
public SinkWriterOperatorFactory(
Sink<InputT> sink, boolean isBatchMode, boolean isCheckpointingEnabled) {
this.sink = checkNotNull(sink);
this.isBatchMode = isBatchMode;
this.isCheckpointingEnabled = isCheckpointingEnabled;
}

public <T extends StreamOperator<CommittableMessage<CommT>>> T createStreamOperator(
StreamOperatorParameters<CommittableMessage<CommT>> parameters) {
try {
final SinkWriterOperator<InputT, CommT> writerOperator =
new SinkWriterOperator<>(sink, processingTimeService, getMailboxExecutor());
new SinkWriterOperator<>(
sink,
processingTimeService,
getMailboxExecutor(),
isBatchMode,
isCheckpointingEnabled);
writerOperator.setup(
parameters.getContainingTask(),
parameters.getStreamConfig(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,14 @@
import org.apache.flink.streaming.api.connector.sink2.CommittableSummary;
import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;

import javax.annotation.Nullable;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;

import static org.apache.flink.util.Preconditions.checkArgument;
Expand All @@ -36,11 +39,12 @@ class CheckpointCommittableManagerImpl<CommT> implements CheckpointCommittableMa
/** Mapping of subtask id to {@link SubtaskCommittableManager}. */
private final Map<Integer, SubtaskCommittableManager<CommT>> subtasksCommittableManagers;

private final long checkpointId;
@Nullable private final Long checkpointId;
private final int subtaskId;
private final int numberOfSubtasks;

CheckpointCommittableManagerImpl(int subtaskId, int numberOfSubtasks, long checkpointId) {
CheckpointCommittableManagerImpl(
int subtaskId, int numberOfSubtasks, @Nullable Long checkpointId) {
this.subtaskId = subtaskId;
this.numberOfSubtasks = numberOfSubtasks;
this.checkpointId = checkpointId;
Expand All @@ -49,7 +53,7 @@ class CheckpointCommittableManagerImpl<CommT> implements CheckpointCommittableMa

CheckpointCommittableManagerImpl(
Map<Integer, SubtaskCommittableManager<CommT>> subtasksCommittableManagers,
long checkpointId) {
@Nullable Long checkpointId) {
this.subtasksCommittableManagers = checkNotNull(subtasksCommittableManagers);
this.subtaskId = 0;
this.numberOfSubtasks = 1;
Expand All @@ -58,6 +62,7 @@ class CheckpointCommittableManagerImpl<CommT> implements CheckpointCommittableMa

@Override
public long getCheckpointId() {
checkNotNull(checkpointId);
return checkpointId;
}

Expand Down Expand Up @@ -138,7 +143,7 @@ Collection<CommittableWithLineage<CommT>> drainFinished() {
}

CheckpointCommittableManagerImpl<CommT> merge(CheckpointCommittableManagerImpl<CommT> other) {
checkArgument(other.checkpointId == checkpointId);
checkArgument(Objects.equals(other.checkpointId, checkpointId));
for (Map.Entry<Integer, SubtaskCommittableManager<CommT>> subtaskEntry :
other.subtasksCommittableManagers.entrySet()) {
subtasksCommittableManagers.merge(
Expand All @@ -150,7 +155,7 @@ CheckpointCommittableManagerImpl<CommT> merge(CheckpointCommittableManagerImpl<C
}

CheckpointCommittableManagerImpl<CommT> copy() {
return new CheckpointCommittableManagerImpl<CommT>(
return new CheckpointCommittableManagerImpl<>(
subtasksCommittableManagers.entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, (e) -> e.getValue().copy())),
checkpointId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,13 +144,14 @@ public Collection<? extends CheckpointCommittableManager<CommT>> getCheckpointCo
}

/**
* Returns all {@link CheckpointCommittableManager} that are currently hold by the collector.
* Returns {@link CheckpointCommittableManager} that is currently hold by the collector and
* associated with the {@link CommittableCollector#EOI} checkpoint id.
*
* @return collection of {@link CheckpointCommittableManager}
* @return {@link CheckpointCommittableManager}
*/
@Nullable
public Collection<? extends CheckpointCommittableManager<CommT>> getEndOfInputCommittables() {
return getCheckpointCommittablesUpTo(EOI);
public CommittableManager<CommT> getEndOfInputCommittable() {
return checkpointCommittables.get(EOI);
}

/**
Expand Down
Loading

0 comments on commit de46599

Please sign in to comment.