Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.beam.runners.core.metrics.MetricsContainerImpl;
import org.apache.beam.runners.flink.FlinkPipelineOptions;
import org.apache.beam.sdk.io.Source;
import org.apache.beam.sdk.io.UnboundedSource;
import org.apache.beam.sdk.metrics.MetricsEnvironment;
import org.apache.beam.sdk.options.PipelineOptions;

Expand Down Expand Up @@ -69,4 +70,18 @@ public boolean invokeAdvance(ReaderT reader) throws IOException {
return reader.advance();
}
}

public UnboundedSource.CheckpointMark invokeCheckpointMark(
UnboundedSource.UnboundedReader<OutputT> reader) throws IOException {
if (enableMetrics) {
try (Closeable ignored =
MetricsEnvironment.scopedMetricsContainer(container.getMetricsContainer(stepName))) {
UnboundedSource.CheckpointMark result = reader.getCheckpointMark();
container.updateMetrics(stepName);
return result;
}
} else {
return reader.getCheckpointMark();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -391,6 +391,9 @@ public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throw
return;
}

ReaderInvocationUtil<OutputT, UnboundedSource.UnboundedReader<OutputT>> readerInvoker =
new ReaderInvocationUtil<>(stepName, serializedOptions.get(), metricContainer);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This can be moved to a field and reused between calls to run and snapshotState.


stateForCheckpoint.clear();

long checkpointId = functionSnapshotContext.getCheckpointId();
Expand All @@ -405,7 +408,7 @@ public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throw
UnboundedSource.UnboundedReader<OutputT> reader = localReaders.get(i);

@SuppressWarnings("unchecked")
CheckpointMarkT mark = (CheckpointMarkT) reader.getCheckpointMark();
CheckpointMarkT mark = (CheckpointMarkT) readerInvoker.invokeCheckpointMark(reader);
checkpointMarks.add(mark);
KV<UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT> kv = KV.of(source, mark);
stateForCheckpoint.add(kv);
Expand Down
Loading