From b2f0f8fb67e236214458a0f685b881b45e7ba003 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Tue, 5 Dec 2023 04:08:48 -0800 Subject: [PATCH] Flink: fix flaky test that might fail due to classloader check (#9216) --- .../source/TestIcebergSourceWithWatermarkExtractor.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 0bb2eb7766e9..aa4b9cd79e55 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -40,6 +40,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; @@ -92,7 +93,10 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(new Configuration())) + .setConfiguration( + reporter.addToConfiguration( + // disable classloader check as Avro may cache class in the serializers. + new Configuration().set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false))) .withHaLeadershipControl() .build());