Skip to content

Commit 765b1e7

Browse files
committed
[FLINK-37611] Deflake ExactlyOnceKafkaWriterITCase#shouldAbortLingeringTransactions
The test was actually not working correctly since writer's get their unique prefix (pool rework PR). The test mostly succeeded since the partitions in which records were written was non-deterministic and more often than not the 3 records didn't meet in the 10 partitions which resulted in an incorrect pass. Now the value is also passed as a key, which makes partitions assignment deterministic and for this test we just write the exact same value 3 times.
1 parent 1b9b1aa commit 765b1e7

File tree

3 files changed

+54
-27
lines changed

3 files changed

+54
-27
lines changed

flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java

Lines changed: 39 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.flink.connector.kafka.sink;
1919

20+
import org.apache.flink.api.java.tuple.Tuple2;
2021
import org.apache.flink.configuration.Configuration;
2122
import org.apache.flink.connector.base.DeliveryGuarantee;
2223
import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer;
@@ -43,6 +44,7 @@
4344
import java.util.Collection;
4445
import java.util.List;
4546
import java.util.Properties;
47+
import java.util.function.Consumer;
4648

4749
import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic;
4850
import static org.assertj.core.api.Assertions.assertThat;
@@ -61,6 +63,9 @@ public class ExactlyOnceKafkaWriterITCase extends KafkaWriterTestBase {
6163
.setConfiguration(new Configuration())
6264
.build());
6365

66+
private static final Consumer<KafkaSinkBuilder<?>> EXACTLY_ONCE =
67+
sink -> sink.setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE);
68+
6469
@Test
6570
void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception {
6671
Properties properties = getKafkaClientConfiguration();
@@ -197,36 +202,35 @@ private void triggerProducerException(KafkaWriter<Integer> writer, Properties pr
197202

198203
/** Test that producer is not accidentally recreated or pool is used. */
199204
@Test
200-
void testLingeringTransaction() throws Exception {
201-
final KafkaWriter<Integer> failedWriter = createWriter(DeliveryGuarantee.EXACTLY_ONCE);
205+
void shouldAbortLingeringTransactions() throws Exception {
206+
try (final ExactlyOnceKafkaWriter<Integer> failedWriter =
207+
createWriter(DeliveryGuarantee.EXACTLY_ONCE)) {
202208

203-
// create two lingering transactions
204-
failedWriter.flush(false);
205-
failedWriter.prepareCommit();
206-
failedWriter.snapshotState(1);
207-
failedWriter.flush(false);
208-
failedWriter.prepareCommit();
209-
failedWriter.snapshotState(2);
209+
// create two lingering transactions
210+
onCheckpointBarrier(failedWriter, 1);
211+
onCheckpointBarrier(failedWriter, 2);
210212

211-
try (final KafkaWriter<Integer> recoveredWriter =
212-
createWriter(DeliveryGuarantee.EXACTLY_ONCE)) {
213-
recoveredWriter.write(1, SINK_WRITER_CONTEXT);
213+
// use state to ensure that the new writer knows about the old prefix
214+
KafkaWriterState state = new KafkaWriterState(failedWriter.getTransactionalIdPrefix());
214215

215-
recoveredWriter.flush(false);
216-
Collection<KafkaCommittable> committables = recoveredWriter.prepareCommit();
217-
recoveredWriter.snapshotState(1);
218-
assertThat(committables).hasSize(1);
219-
final KafkaCommittable committable = committables.stream().findFirst().get();
220-
assertThat(committable.getProducer().isPresent()).isTrue();
216+
try (final KafkaWriter<Integer> recoveredWriter =
217+
restoreWriter(EXACTLY_ONCE, List.of(state), createInitContext())) {
218+
recoveredWriter.write(1, SINK_WRITER_CONTEXT);
221219

222-
committable.getProducer().get().commitTransaction();
220+
recoveredWriter.flush(false);
221+
Collection<KafkaCommittable> committables = recoveredWriter.prepareCommit();
222+
recoveredWriter.snapshotState(1);
223+
assertThat(committables).hasSize(1);
224+
final KafkaCommittable committable = committables.stream().findFirst().get();
225+
assertThat(committable.getProducer().isPresent()).isTrue();
223226

224-
List<ConsumerRecord<byte[], byte[]>> records =
225-
drainAllRecordsFromTopic(topic, getKafkaClientConfiguration(), true);
226-
assertThat(records).hasSize(1);
227-
}
227+
committable.getProducer().get().commitTransaction();
228228

229-
failedWriter.close();
229+
List<ConsumerRecord<byte[], byte[]>> records =
230+
drainAllRecordsFromTopic(topic, getKafkaClientConfiguration(), true);
231+
assertThat(records).hasSize(1);
232+
}
233+
}
230234
}
231235

232236
/** Test that producers are reused when committed. */
@@ -332,4 +336,15 @@ private static Collection<FlinkKafkaInternalProducer<byte[], byte[]>> getProduce
332336
ExactlyOnceKafkaWriter<Integer> writer) {
333337
return ((ProducerPoolImpl) writer.getProducerPool()).getProducers();
334338
}
339+
340+
private Tuple2<KafkaWriterState, KafkaCommittable> onCheckpointBarrier(
341+
KafkaWriter<Integer> failedWriter, int checkpointId)
342+
throws IOException, InterruptedException {
343+
// constant number to force the same partition
344+
failedWriter.write(1, SINK_WRITER_CONTEXT);
345+
failedWriter.flush(false);
346+
KafkaCommittable committable = Iterables.getOnlyElement(failedWriter.prepareCommit());
347+
KafkaWriterState state = Iterables.getOnlyElement(failedWriter.snapshotState(checkpointId));
348+
return Tuple2.of(state, committable);
349+
}
335350
}

flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,7 @@
5656

5757
import java.io.IOException;
5858
import java.nio.ByteBuffer;
59+
import java.util.Collection;
5960
import java.util.Collections;
6061
import java.util.Comparator;
6162
import java.util.Map;
@@ -129,6 +130,14 @@ <T extends KafkaWriter<?>> T createWriter(
129130
return (T) createSink(sinkBuilderAdjuster).createWriter(sinkInitContext);
130131
}
131132

133+
@SuppressWarnings("unchecked")
134+
<T extends KafkaWriter<?>> T restoreWriter(
135+
Consumer<KafkaSinkBuilder<?>> sinkBuilderAdjuster,
136+
Collection<KafkaWriterState> recoveredState,
137+
SinkInitContext initContext) {
138+
return (T) createSink(sinkBuilderAdjuster).restoreWriter(initContext, recoveredState);
139+
}
140+
132141
KafkaSink<Integer> createSink(Consumer<KafkaSinkBuilder<?>> sinkBuilderAdjuster) {
133142
KafkaSinkBuilder<Integer> builder =
134143
KafkaSink.<Integer>builder()
@@ -223,7 +232,8 @@ public ProducerRecord<byte[], byte[]> serialize(
223232
// in general, serializers should be allowed to skip invalid elements
224233
return null;
225234
}
226-
return new ProducerRecord<>(topic, ByteBuffer.allocate(4).putInt(element).array());
235+
byte[] bytes = ByteBuffer.allocate(4).putInt(element).array();
236+
return new ProducerRecord<>(topic, bytes, bytes);
227237
}
228238
}
229239

flink-connector-kafka/src/test/resources/log4j2-test.properties

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,9 +29,11 @@ appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
2929

3030
# Overwrite the level for all Flink related loggers
3131
logger.flink.name = org.apache.flink
32-
logger.flink.level = OFF # WARN for starting debugging
32+
# WARN for starting debugging
33+
logger.flink.level = OFF
3334
logger.flinkconnector.name = org.apache.flink.connector
34-
logger.flinkconnector.level = OFF # INFO/DEBUG for starting debugging
35+
# INFO/DEBUG for starting debugging
36+
logger.flinkconnector.level = OFF
3537

3638
# Kafka producer and consumer level
3739
logger.kafka.name = org.apache.kafka

0 commit comments

Comments
 (0)