Skip to content
Merged
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 @@ -151,7 +151,7 @@ public void snapshotState(StateSnapshotContext context) throws Exception {
public void endInput() throws Exception {
if (!isCheckpointingEnabled || isBatchMode) {
// There will be no final checkpoint, all committables should be committed here
commitAndEmitCheckpoints(lastCompletedCheckpointId + 1);
commitAndEmitCheckpoints(Long.MAX_VALUE);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,11 +36,14 @@
import org.assertj.core.api.ListAssert;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.CsvSource;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.jupiter.params.provider.ValueSource;

import java.util.Collection;
import java.util.function.IntSupplier;
import java.util.stream.Stream;

import static org.apache.flink.streaming.api.connector.sink2.SinkV2Assertions.committableSummary;
import static org.apache.flink.streaming.api.connector.sink2.SinkV2Assertions.committableWithLineage;
Expand Down Expand Up @@ -85,9 +88,17 @@ SinkAndCounters sinkWithoutPostCommit() {
() -> committer.successfulCommits);
}

static Stream<Arguments> testParameters() {
return Stream.of(
Arguments.of(true, false),
Arguments.of(true, true),
Arguments.of(false, false),
Arguments.of(false, true));
}

@ParameterizedTest
@ValueSource(booleans = {true, false})
void testEmitCommittables(boolean withPostCommitTopology) throws Exception {
@MethodSource("testParameters")
void testEmitCommittables(boolean withPostCommitTopology, boolean isBatch) throws Exception {
SinkAndCounters sinkAndCounters;
if (withPostCommitTopology) {
// Insert global committer to simulate post commit topology
Expand All @@ -99,7 +110,8 @@ void testEmitCommittables(boolean withPostCommitTopology) throws Exception {
CommittableMessage<String>, CommittableMessage<String>>
testHarness =
new OneInputStreamOperatorTestHarness<>(
new CommitterOperatorFactory<>(sinkAndCounters.sink, false, true));
new CommitterOperatorFactory<>(
sinkAndCounters.sink, isBatch, true));
testHarness.open();

final CommittableSummary<String> committableSummary =
Expand Down Expand Up @@ -127,6 +139,33 @@ void testEmitCommittables(boolean withPostCommitTopology) throws Exception {
testHarness.close();
}

@Test
void testEmitCommittablesBatch() throws Exception {
SinkAndCounters sinkAndCounters = sinkWithoutPostCommit();
final OneInputStreamOperatorTestHarness<
CommittableMessage<String>, CommittableMessage<String>>
testHarness =
new OneInputStreamOperatorTestHarness<>(
new CommitterOperatorFactory<>(sinkAndCounters.sink, true, false));
testHarness.open();

// Test that all committables up to Long.MAX_VALUE are committed.
long checkpointId = Long.MAX_VALUE;
final CommittableSummary<String> committableSummary =
new CommittableSummary<>(1, 1, checkpointId, 1, 0);
testHarness.processElement(new StreamRecord<>(committableSummary));
final CommittableWithLineage<String> committableWithLineage =
new CommittableWithLineage<>("1", checkpointId, 1);
testHarness.processElement(new StreamRecord<>(committableWithLineage));

testHarness.endInput();

assertThat(sinkAndCounters.commitCounter.getAsInt()).isEqualTo(1);
assertThat(testHarness.getOutput()).isEmpty();

testHarness.close();
}

@Test
void ensureAllCommittablesArrivedBeforeCommitting() throws Exception {
SinkAndCounters sinkAndCounters = sinkWithPostCommit();
Expand Down