From 15b3645abe9611ec49f3bc0d6875e87bc8fd14ec Mon Sep 17 00:00:00 2001 From: Alex Sorokoumov Date: Sun, 26 Mar 2023 15:36:30 -0700 Subject: [PATCH 001/322] [FLINK-31620] [upsert-kafka] Flush wrapped writer - with the solution, the actual flushing only still happens on checkpoints - when buffer size is full or periodically triggered, we don't actually do flushing on the wrapped writer. This closes #17. --- .../connectors/kafka/table/ReducingUpsertWriter.java | 9 +++++---- .../connectors/kafka/table/ReducingUpsertWriterTest.java | 7 ++++++- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java index 5a87add74..67df4a69b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java @@ -87,7 +87,8 @@ public void write(RowData element, Context context) throws IOException, Interrup @Override public void flush(boolean endOfInput) throws IOException, InterruptedException { - flush(); + sinkBuffer(); + wrappedWriter.flush(endOfInput); } @Override @@ -109,7 +110,7 @@ private void addToBuffer(RowData row, Long timestamp) throws IOException, Interr reduceBuffer.put(key, new Tuple2<>(changeFlag(value), timestamp)); if (reduceBuffer.size() >= batchMaxRowNums) { - flush(); + sinkBuffer(); } } @@ -121,7 +122,7 @@ private void registerFlush() { lastFlush + batchIntervalMs, (t) -> { if (t >= lastFlush + batchIntervalMs) { - flush(); + sinkBuffer(); } registerFlush(); }); @@ -140,7 +141,7 @@ private RowData changeFlag(RowData value) { return value; } - private void flush() throws IOException, InterruptedException { + private void sinkBuffer() throws IOException, InterruptedException { for (Tuple2 value : reduceBuffer.values()) { wrappedContext.setTimestamp(value.f1); wrappedWriter.write(value.f0, wrappedContext); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java index b6f2788e9..1ad9d0940 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java @@ -222,6 +222,7 @@ public void testFlushDataWhenCheckpointing() throws Exception { writeData(bufferedWriter, new ReusableIterator(0, 4)); // snapshot should flush the buffer bufferedWriter.flush(true); + assertThat(writer.flushed).isTrue(); HashMap> expected = new HashMap<>(); expected.put( @@ -328,6 +329,8 @@ public ScheduledFuture registerTimer( private static class MockedSinkWriter implements StatefulSink.StatefulSinkWriter { + boolean flushed = false; + transient List rowDataCollectors; MockedSinkWriter() { @@ -343,7 +346,9 @@ public void write(RowData element, Context context) } @Override - public void flush(boolean endOfInput) throws IOException, InterruptedException {} + public void flush(boolean endOfInput) throws IOException, InterruptedException { + flushed = true; + } @Override public void close() throws Exception {} From d72cbdff29419d5b21a1b8d5e3e570a7f4793183 Mon Sep 17 00:00:00 2001 From: mas-chen Date: Fri, 10 Mar 2023 00:42:21 -0800 Subject: [PATCH 002/322] [FLINK-31305] Propagate producer exceptions outside of mailbox executor so that checkpoints can correctly fail This closes #19. --- .../connector/kafka/sink/KafkaWriter.java | 57 ++++++++++++++++--- .../kafka/sink/KafkaWriterITCase.java | 26 +++++++++ 2 files changed, 76 insertions(+), 7 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java index ba2cb4e48..12cf425f1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java @@ -89,7 +89,7 @@ class KafkaWriter private final KafkaRecordSerializationSchema recordSerializer; private final Callback deliveryCallback; private final KafkaRecordSerializationSchema.KafkaSinkContext kafkaSinkContext; - + private volatile Exception asyncProducerException; private final Map previouslyCreatedMetrics = new HashMap<>(); private final SinkWriterMetricGroup metricGroup; private final boolean disabledMetrics; @@ -139,6 +139,7 @@ class KafkaWriter this.kafkaProducerConfig = checkNotNull(kafkaProducerConfig, "kafkaProducerConfig"); this.transactionalIdPrefix = checkNotNull(transactionalIdPrefix, "transactionalIdPrefix"); this.recordSerializer = checkNotNull(recordSerializer, "recordSerializer"); + checkNotNull(sinkInitContext, "sinkInitContext"); this.deliveryCallback = new WriterCallback( sinkInitContext.getMailboxExecutor(), @@ -150,7 +151,6 @@ class KafkaWriter || kafkaProducerConfig.containsKey(KEY_REGISTER_METRICS) && !Boolean.parseBoolean( kafkaProducerConfig.get(KEY_REGISTER_METRICS).toString()); - checkNotNull(sinkInitContext, "sinkInitContext"); this.timeService = sinkInitContext.getProcessingTimeService(); this.metricGroup = sinkInitContext.metricGroup(); this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); @@ -192,6 +192,7 @@ class KafkaWriter @Override public void write(@Nullable IN element, Context context) throws IOException { + checkAsyncException(); final ProducerRecord record = recordSerializer.serialize(element, kafkaSinkContext, context.timestamp()); if (record != null) { @@ -206,6 +207,8 @@ public void flush(boolean endOfInput) throws IOException, InterruptedException { LOG.debug("final flush={}", endOfInput); currentProducer.flush(); } + + checkAsyncException(); } @Override @@ -241,6 +244,9 @@ public void close() throws Exception { checkState(currentProducer.isClosed()); currentProducer = null; }); + + // Rethrow exception for the case in which close is called before writer() and flush(). + checkAsyncException(); } private void abortCurrentProducer() { @@ -264,6 +270,16 @@ FlinkKafkaInternalProducer getCurrentProducer() { return currentProducer; } + @VisibleForTesting + Exception getAsyncProducerException() { + return asyncProducerException; + } + + @VisibleForTesting + void setAsyncProducerException(Exception asyncProducerException) { + this.asyncProducerException = asyncProducerException; + } + void abortLingeringTransactions( Collection recoveredStates, long startCheckpointId) { List prefixesToAbort = Lists.newArrayList(transactionalIdPrefix); @@ -397,6 +413,18 @@ private void registerMetricSync() { }); } + /** This logic needs to be invoked by write AND flush since we support various semantics. */ + private void checkAsyncException() throws IOException { + // reset this exception since we could close the writer later on + Exception e = asyncProducerException; + if (e != null) { + + asyncProducerException = null; + throw new IOException( + "One or more Kafka Producer send requests have encountered exception", e); + } + } + private class WriterCallback implements Callback { private final MailboxExecutor mailboxExecutor; @Nullable private final Consumer metadataConsumer; @@ -413,12 +441,27 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { if (exception != null) { FlinkKafkaInternalProducer producer = KafkaWriter.this.currentProducer; - mailboxExecutor.execute( + + // Propagate the first exception since amount of exceptions could be large. Need to + // do this in Producer IO thread since flush() guarantees that the future will + // complete. The same guarantee does not hold for tasks executed in separate + // executor e.g. mailbox executor. flush() needs to have the exception immediately + // available to fail the checkpoint. + if (asyncProducerException != null) { + asyncProducerException = decorateException(metadata, exception, producer); + } + + mailboxExecutor.submit( () -> { + // Need to send metrics through mailbox thread since we are in the + // producer io + // thread numRecordsOutErrorsCounter.inc(); - throwException(metadata, exception, producer); + + // Checking for exceptions from previous writes + checkAsyncException(); }, - "Failed to send data to Kafka"); + "Update error metric"); } if (metadataConsumer != null) { @@ -426,7 +469,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } } - private void throwException( + private FlinkRuntimeException decorateException( RecordMetadata metadata, Exception exception, FlinkKafkaInternalProducer producer) { @@ -435,7 +478,7 @@ private void throwException( if (exception instanceof UnknownProducerIdException) { message += KafkaCommitter.UNKNOWN_PRODUCER_ID_ERROR_MESSAGE; } - throw new FlinkRuntimeException(message, exception); + return new FlinkRuntimeException(message, exception); } } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index 98f64ed13..e7627b265 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -75,6 +75,7 @@ import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic; import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode; /** Tests for the standalone KafkaWriter. */ @ExtendWith(TestLoggerExtension.class) @@ -223,6 +224,9 @@ void testNumRecordsOutErrorsCounterMetric() throws Exception { } writer.write(3, SINK_WRITER_CONTEXT); + // this doesn't throw exception because the exception is thrown in the Producer IO + // thread in unit tests due to the mock mailbox executor, while it would be thrown in + // flush() when the real mailbox executor is configured writer.flush(false); writer.prepareCommit(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); @@ -383,6 +387,28 @@ void testAbortOnClose() throws Exception { } } + @Test + public void testErrorPropagation() { + Properties properties = getKafkaClientConfiguration(); + final KafkaWriter writer = + createWriterWithConfiguration(properties, DeliveryGuarantee.AT_LEAST_ONCE); + try { + writer.setAsyncProducerException( + new IOException("previous send request encountered error.")); + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .hasRootCauseExactlyInstanceOf(IOException.class); + + writer.setAsyncProducerException( + new IOException("previous send request encountered error.")); + assertThatCode(() -> writer.flush(false)) + .hasRootCauseExactlyInstanceOf(IOException.class); + } finally { + writer.setAsyncProducerException( + new IOException("previous send request encountered error.")); + assertThatCode(writer::close).hasRootCauseExactlyInstanceOf(IOException.class); + } + } + private void assertKafkaMetricNotPresent( DeliveryGuarantee guarantee, String configKey, String configValue) throws Exception { final Properties config = getKafkaClientConfiguration(); From 150eaf888f7beba1224f7e102dc93be2fccd0494 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 23 Mar 2023 14:18:08 -0700 Subject: [PATCH 003/322] [FLINK-31363] [kafka] Add hasDataInTransaction flag in FlinkKafkaInternalProducer This closes #15. --- .../sink/FlinkKafkaInternalProducer.java | 29 +++++++++ .../connector/kafka/sink/KafkaWriter.java | 11 +++- .../FlinkKafkaInternalProducerITCase.java | 59 +++++++++++++++++-- .../kafka/sink/KafkaWriterITCase.java | 26 ++++++++ 4 files changed, 118 insertions(+), 7 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java index 7a3ed5698..e514054d7 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java @@ -17,8 +17,11 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.clients.producer.internals.TransactionManager; import org.apache.kafka.clients.producer.internals.TransactionalRequestResult; import org.apache.kafka.common.errors.ProducerFencedException; @@ -33,6 +36,7 @@ import java.lang.reflect.Method; import java.time.Duration; import java.util.Properties; +import java.util.concurrent.Future; import static org.apache.flink.util.Preconditions.checkState; @@ -49,6 +53,7 @@ class FlinkKafkaInternalProducer extends KafkaProducer { @Nullable private String transactionalId; private volatile boolean inTransaction; + private volatile boolean hasRecordsInTransaction; private volatile boolean closed; public FlinkKafkaInternalProducer(Properties properties, @Nullable String transactionalId) { @@ -67,6 +72,14 @@ private static Properties withTransactionalId( return props; } + @Override + public Future send(ProducerRecord record, Callback callback) { + if (inTransaction) { + hasRecordsInTransaction = true; + } + return super.send(record, callback); + } + @Override public void flush() { super.flush(); @@ -86,6 +99,7 @@ public void abortTransaction() throws ProducerFencedException { LOG.debug("abortTransaction {}", transactionalId); checkState(inTransaction, "Transaction was not started"); inTransaction = false; + hasRecordsInTransaction = false; super.abortTransaction(); } @@ -94,6 +108,7 @@ public void commitTransaction() throws ProducerFencedException { LOG.debug("commitTransaction {}", transactionalId); checkState(inTransaction, "Transaction was not started"); inTransaction = false; + hasRecordsInTransaction = false; super.commitTransaction(); } @@ -101,6 +116,10 @@ public boolean isInTransaction() { return inTransaction; } + public boolean hasRecordsInTransaction() { + return hasRecordsInTransaction; + } + @Override public void close() { closed = true; @@ -301,8 +320,18 @@ public void resumeTransaction(long producerId, short epoch) { transitionTransactionManagerStateTo(transactionManager, "READY"); transitionTransactionManagerStateTo(transactionManager, "IN_TRANSACTION"); + + // the transactionStarted flag in the KafkaProducer controls whether + // an EndTxnRequest will actually be sent to Kafka for a commit + // or abort API call. This flag is set only after the first send (i.e. + // only if data is actually written to some partition). + // In checkpoints, we only ever store metadata of pre-committed + // transactions that actually have records; therefore, on restore + // when we create recovery producers to resume transactions and commit + // them, we should always set this flag. setField(transactionManager, "transactionStarted", true); this.inTransaction = true; + this.hasRecordsInTransaction = true; } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java index 12cf425f1..bcdb0d697 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java @@ -213,13 +213,22 @@ public void flush(boolean endOfInput) throws IOException, InterruptedException { @Override public Collection prepareCommit() { - if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) { + if (deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE) { + return Collections.emptyList(); + } + + // only return a KafkaCommittable if the current transaction has been written some data + if (currentProducer.hasRecordsInTransaction()) { final List committables = Collections.singletonList( KafkaCommittable.of(currentProducer, producerPool::add)); LOG.debug("Committing {} committables.", committables); return committables; } + + // otherwise, we commit the empty transaction as is (no-op) and just recycle the producer + currentProducer.commitTransaction(); + producerPool.add(currentProducer); return Collections.emptyList(); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java index 51770f033..dd15ec226 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java @@ -28,6 +28,7 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.InvalidTxnStateException; import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; @@ -63,16 +64,15 @@ class FlinkKafkaInternalProducerITCase { private static final KafkaContainer KAFKA_CONTAINER = createKafkaContainer(KAFKA, LOG).withEmbeddedZookeeper(); - private static final String TRANSACTION_PREFIX = "test-transaction-"; - @Test void testInitTransactionId() { final String topic = "test-init-transactions"; + final String transactionIdPrefix = "testInitTransactionId-"; try (FlinkKafkaInternalProducer reuse = new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { int numTransactions = 20; for (int i = 1; i <= numTransactions; i++) { - reuse.initTransactionId(TRANSACTION_PREFIX + i); + reuse.initTransactionId(transactionIdPrefix + i); reuse.beginTransaction(); reuse.send(new ProducerRecord<>(topic, "test-value-" + i)); if (i % 2 == 0) { @@ -81,12 +81,59 @@ void testInitTransactionId() { reuse.flush(); reuse.abortTransaction(); } - assertNumTransactions(i); + assertNumTransactions(i, transactionIdPrefix); assertThat(readRecords(topic).count()).isEqualTo(i / 2); } } } + @Test + void testCommitResumedTransaction() { + final String topic = "test-commit-resumed-transaction"; + final String transactionIdPrefix = "testCommitResumedTransaction-"; + final String transactionalId = transactionIdPrefix + "id"; + + KafkaCommittable snapshottedCommittable; + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(getProperties(), transactionalId)) { + producer.initTransactions(); + producer.beginTransaction(); + producer.send(new ProducerRecord<>(topic, "test-value")); + producer.flush(); + snapshottedCommittable = KafkaCommittable.of(producer, ignored -> {}); + } + + try (FlinkKafkaInternalProducer resumedProducer = + new FlinkKafkaInternalProducer<>(getProperties(), transactionalId)) { + resumedProducer.resumeTransaction( + snapshottedCommittable.getProducerId(), snapshottedCommittable.getEpoch()); + resumedProducer.commitTransaction(); + } + + assertNumTransactions(1, transactionIdPrefix); + assertThat(readRecords(topic).count()).isEqualTo(1); + } + + @Test + void testCommitResumedEmptyTransactionShouldFail() { + KafkaCommittable snapshottedCommittable; + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { + producer.initTransactions(); + producer.beginTransaction(); + snapshottedCommittable = KafkaCommittable.of(producer, ignored -> {}); + } + + try (FlinkKafkaInternalProducer resumedProducer = + new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { + resumedProducer.resumeTransaction( + snapshottedCommittable.getProducerId(), snapshottedCommittable.getEpoch()); + + assertThatThrownBy(resumedProducer::commitTransaction) + .isInstanceOf(InvalidTxnStateException.class); + } + } + @ParameterizedTest @MethodSource("provideTransactionsFinalizer") void testResetInnerTransactionIfFinalizingTransactionFailed( @@ -131,10 +178,10 @@ private static Properties getProperties() { FlinkKafkaInternalProducer::abortTransaction); } - private void assertNumTransactions(int numTransactions) { + private void assertNumTransactions(int numTransactions, String transactionIdPrefix) { List transactions = new KafkaTransactionLog(getProperties()) - .getTransactions(id -> id.startsWith(TRANSACTION_PREFIX)); + .getTransactions(id -> id.startsWith(transactionIdPrefix)); assertThat( transactions.stream() .map(KafkaTransactionLog.TransactionRecord::getTransactionId) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index e7627b265..fb0aa2ec9 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -321,6 +321,7 @@ void usePoolForTransactional() throws Exception { getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { assertThat(writer.getProducerPool()).hasSize(0); + writer.write(1, SINK_WRITER_CONTEXT); writer.flush(false); Collection committables0 = writer.prepareCommit(); writer.snapshotState(1); @@ -340,6 +341,7 @@ void usePoolForTransactional() throws Exception { committable.getProducer().get().close(); assertThat(writer.getProducerPool()).hasSize(1); + writer.write(1, SINK_WRITER_CONTEXT); writer.flush(false); Collection committables1 = writer.prepareCommit(); writer.snapshotState(2); @@ -353,6 +355,30 @@ void usePoolForTransactional() throws Exception { } } + /** + * Tests that if a pre-commit attempt occurs on an empty transaction, the writer should not emit + * a KafkaCommittable, and instead immediately commit the empty transaction and recycle the + * producer. + */ + @Test + void prepareCommitForEmptyTransaction() throws Exception { + try (final KafkaWriter writer = + createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { + assertThat(writer.getProducerPool()).hasSize(0); + + // no data written to current transaction + writer.flush(false); + Collection emptyCommittables = writer.prepareCommit(); + + assertThat(emptyCommittables).hasSize(0); + assertThat(writer.getProducerPool()).hasSize(1); + final FlinkKafkaInternalProducer recycledProducer = + writer.getProducerPool().pop(); + assertThat(recycledProducer.isInTransaction()).isFalse(); + } + } + /** * Tests that open transactions are automatically aborted on close such that successive writes * succeed. From 0939def6e4f5ddeab9fb52005210fe540418a3a2 Mon Sep 17 00:00:00 2001 From: mas-chen Date: Wed, 29 Mar 2023 16:40:22 -0700 Subject: [PATCH 004/322] [FLINK-31305] fix error propagation bug in WriterCallback and use TestSinkInitContext general purpose sink testing tool This closes #22303. --- flink-connector-kafka/pom.xml | 8 + .../connector/kafka/sink/KafkaWriter.java | 23 +-- .../kafka/sink/KafkaWriterITCase.java | 190 +++++++++++++----- 3 files changed, 149 insertions(+), 72 deletions(-) diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 0355775c0..23598f41d 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -180,6 +180,14 @@ under the License. test + + org.apache.flink + flink-connector-base + ${flink.version} + test + test-jar + + org.apache.flink flink-metrics-jmx diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java index bcdb0d697..1829d5a62 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java @@ -279,16 +279,6 @@ FlinkKafkaInternalProducer getCurrentProducer() { return currentProducer; } - @VisibleForTesting - Exception getAsyncProducerException() { - return asyncProducerException; - } - - @VisibleForTesting - void setAsyncProducerException(Exception asyncProducerException) { - this.asyncProducerException = asyncProducerException; - } - void abortLingeringTransactions( Collection recoveredStates, long startCheckpointId) { List prefixesToAbort = Lists.newArrayList(transactionalIdPrefix); @@ -422,13 +412,17 @@ private void registerMetricSync() { }); } - /** This logic needs to be invoked by write AND flush since we support various semantics. */ + /** + * This method should only be invoked in the mailbox thread since the counter is not volatile. + * Logic needs to be invoked by write AND flush since we support various semantics. + */ private void checkAsyncException() throws IOException { // reset this exception since we could close the writer later on Exception e = asyncProducerException; if (e != null) { asyncProducerException = null; + numRecordsOutErrorsCounter.inc(); throw new IOException( "One or more Kafka Producer send requests have encountered exception", e); } @@ -456,17 +450,12 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { // complete. The same guarantee does not hold for tasks executed in separate // executor e.g. mailbox executor. flush() needs to have the exception immediately // available to fail the checkpoint. - if (asyncProducerException != null) { + if (asyncProducerException == null) { asyncProducerException = decorateException(metadata, exception, producer); } mailboxExecutor.submit( () -> { - // Need to send metrics through mailbox thread since we are in the - // producer io - // thread - numRecordsOutErrorsCounter.inc(); - // Checking for exceptions from previous writes checkAsyncException(); }, diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index fb0aa2ec9..c1b022dce 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -17,20 +17,18 @@ package org.apache.flink.connector.kafka.sink; -import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.base.sink.writer.TestSinkInitContext; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.OperatorIOMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.metrics.testutils.MetricListener; -import org.apache.flink.runtime.mailbox.SyncMailboxExecutor; import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.TestLoggerExtension; @@ -41,6 +39,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; @@ -198,39 +197,135 @@ public void testCurrentSendTimeMetric() throws Exception { } @Test - void testNumRecordsOutErrorsCounterMetric() throws Exception { + void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - final InternalSinkWriterMetricGroup metricGroup = - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()); - try (final KafkaWriter writer = + SinkInitContext sinkInitContext = + new SinkInitContext( + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + timeService, + null); + final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup)) { - final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = + sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + + // test flush + assertThatCode(() -> writer.flush(false)) + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .as("the exception is not thrown again") + .doesNotThrowAnyException(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + } - writer.write(1, SINK_WRITER_CONTEXT); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + @Test + void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); - final String transactionalId = writer.getCurrentProducer().getTransactionalId(); + SinkInitContext sinkInitContext = + new SinkInitContext( + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + timeService, + null); + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = + sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + // to ensure that the exceptional send request has completed + writer.getCurrentProducer().flush(); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .as("the exception is not thrown again") + .doesNotThrowAnyException(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + } - try (FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, transactionalId)) { + @Test + void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); - producer.initTransactions(); - producer.beginTransaction(); - producer.send(new ProducerRecord(topic, "2".getBytes())); - producer.commitTransaction(); - } + SinkInitContext sinkInitContext = + new SinkInitContext( + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + timeService, + null); + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = + sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); - writer.write(3, SINK_WRITER_CONTEXT); - // this doesn't throw exception because the exception is thrown in the Producer IO - // thread in unit tests due to the mock mailbox executor, while it would be thrown in - // flush() when the real mailbox executor is configured - writer.flush(false); - writer.prepareCommit(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + triggerProducerException(writer, properties); + // to ensure that the exceptional send request has completed + writer.getCurrentProducer().flush(); + + while (sinkInitContext.getMailboxExecutor().tryYield()) { + // execute all mails + } + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .as("the exception is not thrown again") + .doesNotThrowAnyException(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + } + + @Test + void testCloseAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); + + SinkInitContext sinkInitContext = + new SinkInitContext( + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + timeService, + null); + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = + sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + // to ensure that the exceptional send request has completed + writer.getCurrentProducer().flush(); + + // test flush + assertThatCode(writer::close) + .as("flush should throw the exception from the WriterCallback") + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + } + + private void triggerProducerException(KafkaWriter writer, Properties properties) + throws IOException { + final String transactionalId = writer.getCurrentProducer().getTransactionalId(); + + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(properties, transactionalId)) { + producer.initTransactions(); + producer.beginTransaction(); + producer.send(new ProducerRecord(topic, "1".getBytes())); + producer.commitTransaction(); } + + writer.write(1, SINK_WRITER_CONTEXT); } @Test @@ -413,28 +508,6 @@ void testAbortOnClose() throws Exception { } } - @Test - public void testErrorPropagation() { - Properties properties = getKafkaClientConfiguration(); - final KafkaWriter writer = - createWriterWithConfiguration(properties, DeliveryGuarantee.AT_LEAST_ONCE); - try { - writer.setAsyncProducerException( - new IOException("previous send request encountered error.")); - assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) - .hasRootCauseExactlyInstanceOf(IOException.class); - - writer.setAsyncProducerException( - new IOException("previous send request encountered error.")); - assertThatCode(() -> writer.flush(false)) - .hasRootCauseExactlyInstanceOf(IOException.class); - } finally { - writer.setAsyncProducerException( - new IOException("previous send request encountered error.")); - assertThatCode(writer::close).hasRootCauseExactlyInstanceOf(IOException.class); - } - } - private void assertKafkaMetricNotPresent( DeliveryGuarantee guarantee, String configKey, String configValue) throws Exception { final Properties config = getKafkaClientConfiguration(); @@ -475,6 +548,18 @@ private KafkaWriter createWriterWithConfiguration( ImmutableList.of()); } + private KafkaWriter createWriterWithConfiguration( + Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) { + return new KafkaWriter<>( + guarantee, + config, + "test-prefix", + sinkInitContext, + new DummyRecordSerializer(), + new DummySchemaContext(), + ImmutableList.of()); + } + private static Properties getKafkaClientConfiguration() { final Properties standardProps = new Properties(); standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers()); @@ -486,7 +571,7 @@ private static Properties getKafkaClientConfiguration() { return standardProps; } - private static class SinkInitContext implements Sink.InitContext { + private static class SinkInitContext extends TestSinkInitContext { private final SinkWriterMetricGroup metricGroup; private final ProcessingTimeService timeService; @@ -506,11 +591,6 @@ public UserCodeClassLoader getUserCodeClassLoader() { throw new UnsupportedOperationException("Not implemented."); } - @Override - public MailboxExecutor getMailboxExecutor() { - return new SyncMailboxExecutor(); - } - @Override public ProcessingTimeService getProcessingTimeService() { return timeService; From f2bc01a13eba019fe42daf7fef8c185fbf38670c Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 31 Mar 2023 13:50:49 -0700 Subject: [PATCH 005/322] Update version to 4.1-SNAPSHOT --- .../flink-end-to-end-tests-common-kafka/pom.xml | 2 +- .../flink-streaming-kafka-test-base/pom.xml | 2 +- .../flink-streaming-kafka-test/pom.xml | 2 +- flink-connector-kafka-e2e-tests/pom.xml | 2 +- flink-connector-kafka/pom.xml | 2 +- flink-sql-connector-kafka/pom.xml | 2 +- pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index 4cb2c19ad..6a87c4623 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 4.0-SNAPSHOT + 4.1-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml index d4b7dfbcd..ef3d0b135 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 4.0-SNAPSHOT + 4.1-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml index f2d9a2cc1..b7080ea4f 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 4.0-SNAPSHOT + 4.1-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/pom.xml b/flink-connector-kafka-e2e-tests/pom.xml index ddc83d60d..d74e1a29e 100644 --- a/flink-connector-kafka-e2e-tests/pom.xml +++ b/flink-connector-kafka-e2e-tests/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.0-SNAPSHOT + 4.1-SNAPSHOT pom diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 23598f41d..5c11705c5 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.0-SNAPSHOT + 4.1-SNAPSHOT flink-connector-kafka diff --git a/flink-sql-connector-kafka/pom.xml b/flink-sql-connector-kafka/pom.xml index 3be182c62..0750e8bff 100644 --- a/flink-sql-connector-kafka/pom.xml +++ b/flink-sql-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.0-SNAPSHOT + 4.1-SNAPSHOT flink-sql-connector-kafka diff --git a/pom.xml b/pom.xml index 8bb33946d..3f13b6c06 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.0-SNAPSHOT + 4.1-SNAPSHOT Flink : Connectors : Kafka : Parent pom 2022 From 02a48fae9a63fba0926d99be301424c9a9d61a15 Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Tue, 4 Apr 2023 15:44:19 +0200 Subject: [PATCH 006/322] [hotfix] Correct/synchronize multiple dependencies in main POM 1. Set correct groupId for `flink-connector-parent` 2. Remove `archunit` version management since it can cause issues during the weekly builds. Instead, rely on flink-architecture-tests-test dependencies. --- pom.xml | 23 ++++------------------- 1 file changed, 4 insertions(+), 19 deletions(-) diff --git a/pom.xml b/pom.xml index 3f13b6c06..2557a589d 100644 --- a/pom.xml +++ b/pom.xml @@ -20,9 +20,9 @@ under the License. xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> - io.github.zentol.flink + org.apache.flink flink-connector-parent - 1.0 + 1.0.0 4.0.0 @@ -49,7 +49,7 @@ under the License. - 1.17-SNAPSHOT + 1.17.0 16.1 3.4.0 3.5.9 @@ -58,7 +58,6 @@ under the License. 4.13.2 5.9.1 3.23.1 - 1.0.0 1.17.2 3.4.6 1.3 @@ -70,7 +69,7 @@ under the License. 1.11.1 false - 1.16.0 + 1.17.0 1.7.32 2.17.1 @@ -400,20 +399,6 @@ under the License. import - - com.tngtech.archunit - archunit - ${archunit.version} - test - - - - com.tngtech.archunit - archunit-junit5 - ${archunit.version} - test - - From 40cf9994dd847c13602acf1f90895cf9f89b2ce6 Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Tue, 4 Apr 2023 15:45:49 +0200 Subject: [PATCH 007/322] [hotfix] Test PRs against Flink 1.17.0 and all nightly builds against 1.17-SNAPSHOT and 1.18-SNAPSHOT Dropping 1.16-SNAPSHOT since it's incompatible with this version of the Flink Kafka connector --- .github/workflows/push_pr.yml | 2 +- .github/workflows/weekly.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 3e9e1c205..335f108f6 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -25,4 +25,4 @@ jobs: compile_and_test: uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: - flink_version: 1.17-SNAPSHOT + flink_version: 1.17.0 diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index 9d34bc1b4..b4daceb73 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -26,7 +26,7 @@ jobs: if: github.repository_owner == 'apache' strategy: matrix: - flink: [1.16-SNAPSHOT, 1.17-SNAPSHOT] + flink: [1.17-SNAPSHOT, 1.18-SNAPSHOT] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} From 85bdc841e72e8d4ac41c1824139b3e9730a1ed2d Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 5 Apr 2023 14:58:32 -0700 Subject: [PATCH 008/322] [hotfix] Refactor test util method for collecting results of bounded query --- .../kafka/table/KafkaTableITCase.java | 18 +++--------------- .../kafka/table/KafkaTableTestUtils.java | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 15 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java index 515526f93..2674183fb 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java @@ -29,7 +29,6 @@ import org.apache.flink.table.utils.EncodingUtils; import org.apache.flink.test.util.SuccessException; import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; import org.apache.kafka.clients.consumer.OffsetAndMetadata; @@ -58,6 +57,7 @@ import java.util.stream.IntStream; import static org.apache.flink.core.testutils.CommonTestUtils.waitUtil; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectAllRows; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectRows; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.readLines; import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SOURCE_IDLE_TIMEOUT; @@ -225,13 +225,7 @@ public void testKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { // ---------- Consume stream from Kafka ------------------- - List results = new ArrayList<>(); - try (CloseableIterator resultsItr = - tEnv.sqlQuery("SELECT * from kafka").execute().collect()) { - while (resultsItr.hasNext()) { - results.add(resultsItr.next()); - } - } + List results = collectAllRows(tEnv.sqlQuery("SELECT * from kafka")); assertThat(results) .containsExactly(Row.of(1, 1102, "behavior 1"), Row.of(2, 1103, "behavior 2")); @@ -286,13 +280,7 @@ public void testKafkaSourceSinkWithBoundedTimestamp() throws Exception { // ---------- Consume stream from Kafka ------------------- - List results = new ArrayList<>(); - try (CloseableIterator resultsItr = - tEnv.sqlQuery("SELECT * from kafka").execute().collect()) { - while (resultsItr.hasNext()) { - results.add(resultsItr.next()); - } - } + List results = collectAllRows(tEnv.sqlQuery("SELECT * from kafka")); assertThat(results) .containsExactly( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java index c1aada46d..793d8da78 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java @@ -68,6 +68,23 @@ public static List collectRows(Table table, int expectedSize) throws Except return collectedRows; } + /** + * Variant of {@link #collectRows(Table, int)} for bounded queries. This should not run + * indefinitely if there is a bounded number of returned rows. + */ + public static List collectAllRows(Table table) throws Exception { + final TableResult result = table.execute(); + + final List collectedRows = new ArrayList<>(); + try (CloseableIterator iterator = result.collect()) { + while (iterator.hasNext()) { + collectedRows.add(iterator.next()); + } + } + + return collectedRows; + } + public static List readLines(String resource) throws IOException { final URL url = KafkaChangelogTableITCase.class.getClassLoader().getResource(resource); assertThat(url).isNotNull(); From fc31198d864ebafd36cf23ce302b6c1b4038b579 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 5 Apr 2023 14:49:12 -0700 Subject: [PATCH 009/322] [FLINK-31740] [upsert-kafka] Allow setting boundedness options for upsert-kafka This closes #22. --- .../table/KafkaConnectorOptionsUtil.java | 2 +- .../table/UpsertKafkaDynamicTableFactory.java | 19 +- .../UpsertKafkaDynamicTableFactoryTest.java | 227 +++++++++++++++++- .../kafka/table/UpsertKafkaTableITCase.java | 199 +++++++++++++++ 4 files changed, 441 insertions(+), 6 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java index ef70644e5..d6390e27a 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java @@ -189,7 +189,7 @@ private static void validateScanStartupMode(ReadableConfig tableOptions) { }); } - private static void validateScanBoundedMode(ReadableConfig tableOptions) { + static void validateScanBoundedMode(ReadableConfig tableOptions) { tableOptions .getOptional(SCAN_BOUNDED_MODE) .ifPresent( diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java index 254e1bf98..b9f2ea71e 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java @@ -25,8 +25,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.base.DeliveryGuarantee; -import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.BoundedOptions; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; @@ -57,6 +57,9 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_INTERVAL; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_MAX_ROWS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARALLELISM; @@ -68,9 +71,11 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.autoCompleteSchemaRegistrySubject; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createKeyFormatProjection; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createValueFormatProjection; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getBoundedOptions; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopicPattern; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopics; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.validateScanBoundedMode; /** Upsert-Kafka factory. */ public class UpsertKafkaDynamicTableFactory @@ -101,6 +106,9 @@ public Set> optionalOptions() { options.add(SINK_PARALLELISM); options.add(SINK_BUFFER_FLUSH_INTERVAL); options.add(SINK_BUFFER_FLUSH_MAX_ROWS); + options.add(SCAN_BOUNDED_MODE); + options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS); + options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS); return options; } @@ -129,6 +137,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { // always use earliest to keep data integrity StartupMode earliest = StartupMode.EARLIEST; + final BoundedOptions boundedOptions = getBoundedOptions(tableOptions); + return new KafkaDynamicSource( context.getPhysicalRowDataType(), keyDecodingFormat, @@ -142,9 +152,9 @@ public DynamicTableSource createDynamicTableSource(Context context) { earliest, Collections.emptyMap(), 0, - BoundedMode.UNBOUNDED, - Collections.emptyMap(), - 0, + boundedOptions.boundedMode, + boundedOptions.specificOffsets, + boundedOptions.boundedTimestampMillis, true, context.getObjectIdentifier().asSummaryString()); } @@ -228,6 +238,7 @@ private static void validateSource( Format valueFormat, int[] primaryKeyIndexes) { validateTopic(tableOptions); + validateScanBoundedMode(tableOptions); validateFormat(keyFormat, valueFormat, tableOptions); validatePKConstraints(primaryKeyIndexes); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java index 5caaaa0ac..959c44cf3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java @@ -21,11 +21,14 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.dag.Transformation; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.sink.KafkaSink; import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.KafkaSourceTestUtils; import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; import org.apache.flink.formats.avro.RowDataToAvroConverters; @@ -64,23 +67,29 @@ import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.util.TestLogger; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.TopicPartition; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.function.Consumer; +import java.util.function.Function; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.AVRO_CONFLUENT; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link UpsertKafkaDynamicTableFactory}. */ public class UpsertKafkaDynamicTableFactoryTest extends TestLogger { @@ -390,6 +399,134 @@ private SerializationSchema createConfluentAvroSerSchema( RowDataToAvroConverters.createConverter(rowType)); } + // -------------------------------------------------------------------------------------------- + // Bounded end-offset tests + // -------------------------------------------------------------------------------------------- + + @Test + public void testBoundedSpecificOffsetsValidate() { + final Map options = getFullSourceOptions(); + options.put( + KafkaConnectorOptions.SCAN_BOUNDED_MODE.key(), + ScanBoundedMode.SPECIFIC_OFFSETS.toString()); + + assertThatThrownBy(() -> createTableSource(SOURCE_SCHEMA, options)) + .isInstanceOf(ValidationException.class) + .cause() + .hasMessageContaining( + "'scan.bounded.specific-offsets' is required in 'specific-offsets' bounded mode but missing."); + } + + @Test + public void testBoundedSpecificOffsets() { + testBoundedOffsets( + ScanBoundedMode.SPECIFIC_OFFSETS, + options -> { + options.put("scan.bounded.specific-offsets", "partition:0,offset:2"); + }, + source -> { + assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); + OffsetsInitializer offsetsInitializer = + KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); + TopicPartition partition = new TopicPartition(SOURCE_TOPIC, 0); + Map partitionOffsets = + offsetsInitializer.getPartitionOffsets( + Collections.singletonList(partition), + MockPartitionOffsetsRetriever.noInteractions()); + assertThat(partitionOffsets) + .containsOnlyKeys(partition) + .containsEntry(partition, 2L); + }); + } + + @Test + public void testBoundedLatestOffset() { + testBoundedOffsets( + ScanBoundedMode.LATEST_OFFSET, + options -> {}, + source -> { + assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); + OffsetsInitializer offsetsInitializer = + KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); + TopicPartition partition = new TopicPartition(SOURCE_TOPIC, 0); + Map partitionOffsets = + offsetsInitializer.getPartitionOffsets( + Collections.singletonList(partition), + MockPartitionOffsetsRetriever.noInteractions()); + assertThat(partitionOffsets) + .containsOnlyKeys(partition) + .containsEntry(partition, KafkaPartitionSplit.LATEST_OFFSET); + }); + } + + @Test + public void testBoundedGroupOffsets() { + testBoundedOffsets( + ScanBoundedMode.GROUP_OFFSETS, + options -> { + options.put("properties.group.id", "dummy"); + }, + source -> { + assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); + OffsetsInitializer offsetsInitializer = + KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); + TopicPartition partition = new TopicPartition(SOURCE_TOPIC, 0); + Map partitionOffsets = + offsetsInitializer.getPartitionOffsets( + Collections.singletonList(partition), + MockPartitionOffsetsRetriever.noInteractions()); + assertThat(partitionOffsets) + .containsOnlyKeys(partition) + .containsEntry(partition, KafkaPartitionSplit.COMMITTED_OFFSET); + }); + } + + @Test + public void testBoundedTimestamp() { + testBoundedOffsets( + ScanBoundedMode.TIMESTAMP, + options -> { + options.put("scan.bounded.timestamp-millis", "1"); + }, + source -> { + assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); + OffsetsInitializer offsetsInitializer = + KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); + TopicPartition partition = new TopicPartition(SOURCE_TOPIC, 0); + long offsetForTimestamp = 123L; + Map partitionOffsets = + offsetsInitializer.getPartitionOffsets( + Collections.singletonList(partition), + MockPartitionOffsetsRetriever.timestampAndEnd( + partitions -> { + assertThat(partitions) + .containsOnlyKeys(partition) + .containsEntry(partition, 1L); + Map result = + new HashMap<>(); + result.put( + partition, + new OffsetAndTimestamp( + offsetForTimestamp, 1L)); + return result; + }, + partitions -> { + Map result = new HashMap<>(); + result.put( + partition, + // the end offset is bigger than given by + // timestamp + // to make sure the one for timestamp is + // used + offsetForTimestamp + 1000L); + return result; + })); + assertThat(partitionOffsets) + .containsOnlyKeys(partition) + .containsEntry(partition, offsetForTimestamp); + }); + } + // -------------------------------------------------------------------------------------------- // Negative tests // -------------------------------------------------------------------------------------------- @@ -647,7 +784,7 @@ private static KafkaDynamicSink createExpectedSink( null); } - private void assertKafkaSource(ScanTableSource.ScanRuntimeProvider provider) { + private KafkaSource assertKafkaSource(ScanTableSource.ScanRuntimeProvider provider) { assertThat(provider).isInstanceOf(DataStreamScanProvider.class); final DataStreamScanProvider dataStreamScanProvider = (DataStreamScanProvider) provider; final Transformation transformation = @@ -662,5 +799,93 @@ private void assertKafkaSource(ScanTableSource.ScanRuntimeProvider provider) { (SourceTransformation) transformation; assertThat(sourceTransformation.getSource()).isInstanceOf(KafkaSource.class); + return (KafkaSource) sourceTransformation.getSource(); + } + + private void testBoundedOffsets( + ScanBoundedMode boundedMode, + Consumer> optionsConfig, + Consumer> validator) { + final Map options = getFullSourceOptions(); + options.put(KafkaConnectorOptions.SCAN_BOUNDED_MODE.key(), boundedMode.toString()); + optionsConfig.accept(options); + + final DynamicTableSource tableSource = createTableSource(SOURCE_SCHEMA, options); + assertThat(tableSource).isInstanceOf(KafkaDynamicSource.class); + ScanTableSource.ScanRuntimeProvider provider = + ((KafkaDynamicSource) tableSource) + .getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertThat(provider).isInstanceOf(DataStreamScanProvider.class); + final KafkaSource kafkaSource = assertKafkaSource(provider); + validator.accept(kafkaSource); + } + + private interface OffsetsRetriever + extends Function, Map> {} + + private interface TimestampOffsetsRetriever + extends Function, Map> {} + + private static final class MockPartitionOffsetsRetriever + implements OffsetsInitializer.PartitionOffsetsRetriever { + + public static final OffsetsRetriever UNSUPPORTED_RETRIEVAL = + partitions -> { + throw new UnsupportedOperationException( + "The method was not supposed to be called"); + }; + private final OffsetsRetriever committedOffsets; + private final OffsetsRetriever endOffsets; + private final OffsetsRetriever beginningOffsets; + private final TimestampOffsetsRetriever offsetsForTimes; + + static MockPartitionOffsetsRetriever noInteractions() { + return new MockPartitionOffsetsRetriever( + UNSUPPORTED_RETRIEVAL, + UNSUPPORTED_RETRIEVAL, + UNSUPPORTED_RETRIEVAL, + partitions -> { + throw new UnsupportedOperationException( + "The method was not supposed to be called"); + }); + } + + static MockPartitionOffsetsRetriever timestampAndEnd( + TimestampOffsetsRetriever retriever, OffsetsRetriever endOffsets) { + return new MockPartitionOffsetsRetriever( + UNSUPPORTED_RETRIEVAL, endOffsets, UNSUPPORTED_RETRIEVAL, retriever); + } + + private MockPartitionOffsetsRetriever( + OffsetsRetriever committedOffsets, + OffsetsRetriever endOffsets, + OffsetsRetriever beginningOffsets, + TimestampOffsetsRetriever offsetsForTimes) { + this.committedOffsets = committedOffsets; + this.endOffsets = endOffsets; + this.beginningOffsets = beginningOffsets; + this.offsetsForTimes = offsetsForTimes; + } + + @Override + public Map committedOffsets(Collection partitions) { + return committedOffsets.apply(partitions); + } + + @Override + public Map endOffsets(Collection partitions) { + return endOffsets.apply(partitions); + } + + @Override + public Map beginningOffsets(Collection partitions) { + return beginningOffsets.apply(partitions); + } + + @Override + public Map offsetsForTimes( + Map timestampsToSearch) { + return offsetsForTimes.apply(timestampsToSearch); + } } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java index 109f4402a..23f638f69 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java @@ -33,17 +33,20 @@ import java.time.Duration; import java.time.LocalDateTime; +import java.time.ZoneId; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.stream.Collectors; import static org.apache.flink.api.common.typeinfo.Types.INT; import static org.apache.flink.api.common.typeinfo.Types.LOCAL_DATE_TIME; import static org.apache.flink.api.common.typeinfo.Types.ROW_NAMED; import static org.apache.flink.api.common.typeinfo.Types.STRING; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectAllRows; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectRows; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.comparedWithKeyAndOrder; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.waitingExpectedResults; @@ -388,6 +391,202 @@ public void testKafkaSourceSinkWithKeyAndFullValue() throws Exception { deleteTestTopic(topic); } + @Test + public void testUpsertKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { + final String topic = "bounded_upsert_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic, 1, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + final String bootstraps = getBootstrapServers(); + + // table with upsert-kafka connector, bounded mode up to offset=2 + final String createTableSql = + String.format( + "CREATE TABLE upsert_kafka (\n" + + " `user_id` BIGINT,\n" + + " `event_id` BIGINT,\n" + + " `payload` STRING,\n" + + " PRIMARY KEY (event_id, user_id) NOT ENFORCED" + + ") WITH (\n" + + " 'connector' = 'upsert-kafka',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'value.format' = '%s',\n" + + " 'value.fields-include' = 'ALL',\n" + + " 'scan.bounded.mode' = 'specific-offsets',\n" + + " 'scan.bounded.specific-offsets' = 'partition:0,offset:2'" + + ")", + topic, bootstraps, format, format); + tEnv.executeSql(createTableSql); + + // insert multiple values to have more records past offset=2 + final String insertValuesSql = + "INSERT INTO upsert_kafka\n" + + "VALUES\n" + + " (1, 100, 'payload 1'),\n" + + " (1, 100, 'payload 1-new'),\n" + + " (2, 101, 'payload 2'),\n" + + " (3, 102, 'payload 3')"; + tEnv.executeSql(insertValuesSql).await(); + + // results should only have records up to offset=2 + final List results = collectAllRows(tEnv.sqlQuery("SELECT * from upsert_kafka")); + final List expected = + Arrays.asList( + changelogRow("+I", 1L, 100L, "payload 1"), + changelogRow("-U", 1L, 100L, "payload 1"), + changelogRow("+U", 1L, 100L, "payload 1-new")); + assertThat(results).satisfies(matching(deepEqualTo(expected, true))); + + // ------------- cleanup ------------------- + + deleteTestTopic(topic); + } + + @Test + public void testUpsertKafkaSourceSinkWithBoundedTimestamp() throws Exception { + final String topic = "bounded_upsert_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic, 1, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + final String bootstraps = getBootstrapServers(); + + // table with upsert-kafka connector, bounded mode up to timestamp 2023-03-10T14:00:00.000 + final String createTableSql = + String.format( + "CREATE TABLE upsert_kafka (\n" + + " `user_id` BIGINT,\n" + + " `timestamp` TIMESTAMP(3) METADATA,\n" + + " `event_id` BIGINT,\n" + + " `payload` STRING,\n" + + " PRIMARY KEY (event_id, user_id) NOT ENFORCED" + + ") WITH (\n" + + " 'connector' = 'upsert-kafka',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'value.format' = '%s',\n" + + " 'value.fields-include' = 'ALL',\n" + + " 'scan.bounded.mode' = 'timestamp',\n" + + " 'scan.bounded.timestamp-millis' = '%d'" + + ")", + topic, + bootstraps, + format, + format, + LocalDateTime.parse("2023-03-10T14:00:00.000") + .atZone(ZoneId.systemDefault()) + .toInstant() + .toEpochMilli()); + tEnv.executeSql(createTableSql); + + // insert multiple values with timestamp starting from 2023-03-08 up to 2023-03-11 + final String insertValuesSql = + "INSERT INTO upsert_kafka\n" + + "VALUES\n" + + " (1, TIMESTAMP '2023-03-08 08:10:10.666', 100, 'payload 1'),\n" + + " (2, TIMESTAMP '2023-03-09 13:12:11.123', 101, 'payload 2'),\n" + + " (1, TIMESTAMP '2023-03-10 12:09:50.321', 100, 'payload 1-new'),\n" + + " (2, TIMESTAMP '2023-03-11 17:15:13.457', 101, 'payload 2-new')"; + tEnv.executeSql(insertValuesSql).await(); + + // results should only have records up to timestamp 2023-03-10T14:00:00.000 + final List results = collectAllRows(tEnv.sqlQuery("SELECT * from upsert_kafka")); + final List expected = + Arrays.asList( + changelogRow( + "+I", + 1L, + LocalDateTime.parse("2023-03-08T08:10:10.666"), + 100L, + "payload 1"), + changelogRow( + "+I", + 2L, + LocalDateTime.parse("2023-03-09T13:12:11.123"), + 101L, + "payload 2"), + changelogRow( + "-U", + 1L, + LocalDateTime.parse("2023-03-08T08:10:10.666"), + 100L, + "payload 1"), + changelogRow( + "+U", + 1L, + LocalDateTime.parse("2023-03-10T12:09:50.321"), + 100L, + "payload 1-new")); + assertThat(results).satisfies(matching(deepEqualTo(expected, true))); + + // ------------- cleanup ------------------- + + deleteTestTopic(topic); + } + + /** + * Tests that setting bounded end offset that is before the earliest offset results in 0 + * results. + */ + @Test + public void testUpsertKafkaSourceSinkWithZeroLengthBoundedness() throws Exception { + final String topic = "bounded_upsert_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic, 1, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + final String bootstraps = getBootstrapServers(); + + // table with upsert-kafka connector, bounded mode up to timestamp 2023-03-10T14:00:00.000 + final String createTableSql = + String.format( + "CREATE TABLE upsert_kafka (\n" + + " `user_id` BIGINT,\n" + + " `timestamp` TIMESTAMP(3) METADATA,\n" + + " `event_id` BIGINT,\n" + + " `payload` STRING,\n" + + " PRIMARY KEY (event_id, user_id) NOT ENFORCED" + + ") WITH (\n" + + " 'connector' = 'upsert-kafka',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'value.format' = '%s',\n" + + " 'value.fields-include' = 'ALL',\n" + + " 'scan.bounded.mode' = 'timestamp',\n" + + " 'scan.bounded.timestamp-millis' = '%d'" + + ")", + topic, + bootstraps, + format, + format, + LocalDateTime.parse("2023-03-10T14:00:00.000") + .atZone(ZoneId.systemDefault()) + .toInstant() + .toEpochMilli()); + tEnv.executeSql(createTableSql); + + // insert multiple values with timestamp starting from 2023-03-11 (which is past the bounded + // end timestamp) + final String insertValuesSql = + "INSERT INTO upsert_kafka\n" + + "VALUES\n" + + " (1, TIMESTAMP '2023-03-11 08:10:10.666', 100, 'payload 1'),\n" + + " (2, TIMESTAMP '2023-03-12 13:12:11.123', 101, 'payload 2'),\n" + + " (1, TIMESTAMP '2023-03-13 12:09:50.321', 100, 'payload 1-new'),\n" + + " (2, TIMESTAMP '2023-03-14 17:15:13.457', 101, 'payload 2-new')"; + tEnv.executeSql(insertValuesSql).await(); + + // results should be empty + final List results = collectAllRows(tEnv.sqlQuery("SELECT * from upsert_kafka")); + assertThat(results).satisfies(matching(deepEqualTo(Collections.emptyList(), true))); + + // ------------- cleanup ------------------- + + deleteTestTopic(topic); + } + private void wordCountToUpsertKafka(String wordCountTable) throws Exception { String bootstraps = getBootstrapServers(); From ffa11654b67885cf7beb823182e640b863162d72 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 5 Apr 2023 15:08:39 -0700 Subject: [PATCH 010/322] [hotfix] Refactor MockPartitionOffsetsRetriever as a common test utility --- .../table/KafkaDynamicTableFactoryTest.java | 72 +------------ .../UpsertKafkaDynamicTableFactoryTest.java | 72 +------------ .../MockPartitionOffsetsRetriever.java | 102 ++++++++++++++++++ 3 files changed, 104 insertions(+), 142 deletions(-) create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java index a0b74a5b7..7ab050359 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java @@ -46,6 +46,7 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode; +import org.apache.flink.streaming.connectors.kafka.testutils.MockPartitionOffsetsRetriever; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; @@ -87,7 +88,6 @@ import javax.annotation.Nullable; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -97,7 +97,6 @@ import java.util.Properties; import java.util.Set; import java.util.function.Consumer; -import java.util.function.Function; import java.util.regex.Pattern; import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; @@ -577,75 +576,6 @@ private void testBoundedOffsets( validator.accept(kafkaSource); } - private interface OffsetsRetriever - extends Function, Map> {} - - private interface TimestampOffsetsRetriever - extends Function, Map> {} - - private static final class MockPartitionOffsetsRetriever - implements OffsetsInitializer.PartitionOffsetsRetriever { - - public static final OffsetsRetriever UNSUPPORTED_RETRIEVAL = - partitions -> { - throw new UnsupportedOperationException( - "The method was not supposed to be called"); - }; - private final OffsetsRetriever committedOffsets; - private final OffsetsRetriever endOffsets; - private final OffsetsRetriever beginningOffsets; - private final TimestampOffsetsRetriever offsetsForTimes; - - static MockPartitionOffsetsRetriever noInteractions() { - return new MockPartitionOffsetsRetriever( - UNSUPPORTED_RETRIEVAL, - UNSUPPORTED_RETRIEVAL, - UNSUPPORTED_RETRIEVAL, - partitions -> { - throw new UnsupportedOperationException( - "The method was not supposed to be called"); - }); - } - - static MockPartitionOffsetsRetriever timestampAndEnd( - TimestampOffsetsRetriever retriever, OffsetsRetriever endOffsets) { - return new MockPartitionOffsetsRetriever( - UNSUPPORTED_RETRIEVAL, endOffsets, UNSUPPORTED_RETRIEVAL, retriever); - } - - private MockPartitionOffsetsRetriever( - OffsetsRetriever committedOffsets, - OffsetsRetriever endOffsets, - OffsetsRetriever beginningOffsets, - TimestampOffsetsRetriever offsetsForTimes) { - this.committedOffsets = committedOffsets; - this.endOffsets = endOffsets; - this.beginningOffsets = beginningOffsets; - this.offsetsForTimes = offsetsForTimes; - } - - @Override - public Map committedOffsets(Collection partitions) { - return committedOffsets.apply(partitions); - } - - @Override - public Map endOffsets(Collection partitions) { - return endOffsets.apply(partitions); - } - - @Override - public Map beginningOffsets(Collection partitions) { - return beginningOffsets.apply(partitions); - } - - @Override - public Map offsetsForTimes( - Map timestampsToSearch) { - return offsetsForTimes.apply(timestampsToSearch); - } - } - @Test public void testTableSink() { final Map modifiedOptions = diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java index 959c44cf3..ba424f79e 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java @@ -39,6 +39,7 @@ import org.apache.flink.streaming.api.transformations.SourceTransformation; import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.testutils.MockPartitionOffsetsRetriever; import org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperatorFactory; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; @@ -74,14 +75,12 @@ import org.junit.rules.ExpectedException; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.function.Consumer; -import java.util.function.Function; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode; @@ -819,73 +818,4 @@ private void testBoundedOffsets( final KafkaSource kafkaSource = assertKafkaSource(provider); validator.accept(kafkaSource); } - - private interface OffsetsRetriever - extends Function, Map> {} - - private interface TimestampOffsetsRetriever - extends Function, Map> {} - - private static final class MockPartitionOffsetsRetriever - implements OffsetsInitializer.PartitionOffsetsRetriever { - - public static final OffsetsRetriever UNSUPPORTED_RETRIEVAL = - partitions -> { - throw new UnsupportedOperationException( - "The method was not supposed to be called"); - }; - private final OffsetsRetriever committedOffsets; - private final OffsetsRetriever endOffsets; - private final OffsetsRetriever beginningOffsets; - private final TimestampOffsetsRetriever offsetsForTimes; - - static MockPartitionOffsetsRetriever noInteractions() { - return new MockPartitionOffsetsRetriever( - UNSUPPORTED_RETRIEVAL, - UNSUPPORTED_RETRIEVAL, - UNSUPPORTED_RETRIEVAL, - partitions -> { - throw new UnsupportedOperationException( - "The method was not supposed to be called"); - }); - } - - static MockPartitionOffsetsRetriever timestampAndEnd( - TimestampOffsetsRetriever retriever, OffsetsRetriever endOffsets) { - return new MockPartitionOffsetsRetriever( - UNSUPPORTED_RETRIEVAL, endOffsets, UNSUPPORTED_RETRIEVAL, retriever); - } - - private MockPartitionOffsetsRetriever( - OffsetsRetriever committedOffsets, - OffsetsRetriever endOffsets, - OffsetsRetriever beginningOffsets, - TimestampOffsetsRetriever offsetsForTimes) { - this.committedOffsets = committedOffsets; - this.endOffsets = endOffsets; - this.beginningOffsets = beginningOffsets; - this.offsetsForTimes = offsetsForTimes; - } - - @Override - public Map committedOffsets(Collection partitions) { - return committedOffsets.apply(partitions); - } - - @Override - public Map endOffsets(Collection partitions) { - return endOffsets.apply(partitions); - } - - @Override - public Map beginningOffsets(Collection partitions) { - return beginningOffsets.apply(partitions); - } - - @Override - public Map offsetsForTimes( - Map timestampsToSearch) { - return offsetsForTimes.apply(timestampsToSearch); - } - } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java new file mode 100644 index 000000000..175bddd6a --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.testutils; + +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; + +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.TopicPartition; + +import java.util.Collection; +import java.util.Map; +import java.util.function.Function; + +/** Fake {@link OffsetsInitializer.PartitionOffsetsRetriever} for unit tests. */ +public final class MockPartitionOffsetsRetriever + implements OffsetsInitializer.PartitionOffsetsRetriever { + + /** Fake offsets retriever for a given set of topic partitions. */ + public interface OffsetsRetriever + extends Function, Map> {} + + /** + * Fake offsets retrieve for a given set of topic partitions and their target timestamp + * position. + */ + public interface TimestampOffsetsRetriever + extends Function, Map> {} + + public static final OffsetsRetriever UNSUPPORTED_RETRIEVAL = + partitions -> { + throw new UnsupportedOperationException("The method was not supposed to be called"); + }; + private final OffsetsRetriever committedOffsets; + private final OffsetsRetriever endOffsets; + private final OffsetsRetriever beginningOffsets; + private final TimestampOffsetsRetriever offsetsForTimes; + + public static MockPartitionOffsetsRetriever noInteractions() { + return new MockPartitionOffsetsRetriever( + UNSUPPORTED_RETRIEVAL, + UNSUPPORTED_RETRIEVAL, + UNSUPPORTED_RETRIEVAL, + partitions -> { + throw new UnsupportedOperationException( + "The method was not supposed to be called"); + }); + } + + public static MockPartitionOffsetsRetriever timestampAndEnd( + TimestampOffsetsRetriever retriever, OffsetsRetriever endOffsets) { + return new MockPartitionOffsetsRetriever( + UNSUPPORTED_RETRIEVAL, endOffsets, UNSUPPORTED_RETRIEVAL, retriever); + } + + private MockPartitionOffsetsRetriever( + OffsetsRetriever committedOffsets, + OffsetsRetriever endOffsets, + OffsetsRetriever beginningOffsets, + TimestampOffsetsRetriever offsetsForTimes) { + this.committedOffsets = committedOffsets; + this.endOffsets = endOffsets; + this.beginningOffsets = beginningOffsets; + this.offsetsForTimes = offsetsForTimes; + } + + @Override + public Map committedOffsets(Collection partitions) { + return committedOffsets.apply(partitions); + } + + @Override + public Map endOffsets(Collection partitions) { + return endOffsets.apply(partitions); + } + + @Override + public Map beginningOffsets(Collection partitions) { + return beginningOffsets.apply(partitions); + } + + @Override + public Map offsetsForTimes( + Map timestampsToSearch) { + return offsetsForTimes.apply(timestampsToSearch); + } +} From 33bd5511eeafacf60c85f4d2ee611ec09aaf8d1c Mon Sep 17 00:00:00 2001 From: mas-chen Date: Fri, 24 Mar 2023 11:06:17 -0700 Subject: [PATCH 011/322] [hotfix] Update NOTICE files to reflect year 2023 This closes #23. --- NOTICE | 2 +- flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/NOTICE b/NOTICE index 02d9298c8..1914984d3 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Flink Kafka Connector -Copyright 2014-2022 The Apache Software Foundation +Copyright 2014-2023 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE b/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE index 641d4040d..1ca013b7b 100644 --- a/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE +++ b/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-kafka -Copyright 2014-2022 The Apache Software Foundation +Copyright 2014-2023 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From 16172a00211317702f5c057e98fae5ca30f1b7f2 Mon Sep 17 00:00:00 2001 From: zhang_yao Date: Fri, 10 Feb 2023 14:40:37 +0800 Subject: [PATCH 012/322] [FLINK-25916][connector-kafka] Using upsert-kafka with a flush buffer results in Null Pointer Exception This closes #5. --- .../kafka/table/ReducingUpsertWriter.java | 5 +- .../kafka/table/ReducingUpsertWriterTest.java | 53 ++++++++++++++- .../kafka/table/UpsertKafkaTableITCase.java | 65 +++++++++++++++++++ 3 files changed, 118 insertions(+), 5 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java index 67df4a69b..914874666 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java @@ -158,7 +158,7 @@ private void sinkBuffer() throws IOException, InterruptedException { * ReducingUpsertWriter} will emit the records in the buffer with memorized timestamp. */ private static class WrappedContext implements SinkWriter.Context { - private long timestamp; + private Long timestamp; private SinkWriter.Context context; @Override @@ -169,11 +169,10 @@ public long currentWatermark() { @Override public Long timestamp() { - checkNotNull(timestamp, "timestamp must to be set before retrieving it."); return timestamp; } - public void setTimestamp(long timestamp) { + public void setTimestamp(Long timestamp) { this.timestamp = timestamp; } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java index 1ad9d0940..5ef36e75b 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java @@ -262,6 +262,50 @@ public void testFlushDataWhenCheckpointing() throws Exception { compareCompactedResult(expected, writer.rowDataCollectors); } + @Test + public void testWriteDataWithNullTimestamp() throws Exception { + final MockedSinkWriter writer = new MockedSinkWriter(); + final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); + + bufferedWriter.write( + GenericRowData.ofKind( + INSERT, + 1001, + StringData.fromString("Java public for dummies"), + StringData.fromString("Tan Ah Teck"), + 11.11, + 11, + null), + new org.apache.flink.api.connector.sink2.SinkWriter.Context() { + @Override + public long currentWatermark() { + throw new UnsupportedOperationException("Not implemented."); + } + + @Override + public Long timestamp() { + return null; + } + }); + + bufferedWriter.flush(true); + + final HashMap> expected = new HashMap<>(); + expected.put( + 1001, + Collections.singletonList( + GenericRowData.ofKind( + UPDATE_AFTER, + 1001, + StringData.fromString("Java public for dummies"), + StringData.fromString("Tan Ah Teck"), + 11.11, + 11, + null))); + + compareCompactedResult(expected, writer.rowDataCollectors); + } + private void compareCompactedResult( Map> expected, List actual) { Map> actualMap = new HashMap<>(); @@ -340,8 +384,13 @@ private static class MockedSinkWriter @Override public void write(RowData element, Context context) throws IOException, InterruptedException { - assertThat(Instant.ofEpochMilli(context.timestamp())) - .isEqualTo(element.getTimestamp(TIMESTAMP_INDICES, 3).toInstant()); + // Allow comparison between null timestamps + if (context.timestamp() == null) { + assertThat(element.getTimestamp(TIMESTAMP_INDICES, 3)).isNull(); + } else { + assertThat(Instant.ofEpochMilli(context.timestamp())) + .isEqualTo(element.getTimestamp(TIMESTAMP_INDICES, 3).toInstant()); + } rowDataCollectors.add(element); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java index 23f638f69..1a6bf7e13 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java @@ -198,6 +198,71 @@ public void testBufferedUpsertSink() throws Exception { deleteTestTopic(topic); } + @Test + public void testBufferedUpsertSinkWithoutAssigningWatermark() throws Exception { + final String topic = "buffered_upsert_topic_without_assigning_watermark_" + format; + createTestTopic(topic, 1, 1); + String bootstraps = getBootstrapServers(); + env.setParallelism(1); + + Table table = + tEnv.fromDataStream( + env.fromElements( + Row.of(1, null, "payload 1"), + Row.of(2, null, "payload 2"), + Row.of(3, null, "payload 3"), + Row.of(3, null, "payload")) + .returns( + ROW_NAMED( + new String[] {"k_id", "ts", "payload"}, + INT, + LOCAL_DATE_TIME, + STRING)), + Schema.newBuilder() + .column("k_id", DataTypes.INT()) + .column("ts", DataTypes.TIMESTAMP(3)) + .column("payload", DataTypes.STRING()) + .build()); + + final String createTable = + String.format( + "CREATE TABLE upsert_kafka (\n" + + " `k_id` INTEGER,\n" + + " `ts` TIMESTAMP(3),\n" + + " `payload` STRING,\n" + + " PRIMARY KEY (k_id) NOT ENFORCED" + + ") WITH (\n" + + " 'connector' = 'upsert-kafka',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'sink.buffer-flush.max-rows' = '2',\n" + + " 'sink.buffer-flush.interval' = '100000',\n" + + " 'value.format' = '%s',\n" + + " 'value.fields-include' = 'ALL',\n" + + " 'key.csv.null-literal' = '',\n" + + " 'value.csv.null-literal' = ''\n" + + ")", + topic, bootstraps, "csv", "csv"); + + tEnv.executeSql(createTable); + + table.executeInsert("upsert_kafka").await(); + + final List result = collectRows(tEnv.sqlQuery("SELECT * FROM upsert_kafka"), 3); + final List expected = + Arrays.asList( + changelogRow("+I", 1, null, "payload 1"), + changelogRow("+I", 2, null, "payload 2"), + changelogRow("+I", 3, null, "payload")); + + assertThat(result).satisfies(matching(deepEqualTo(expected, true))); + + // ------------- cleanup ------------------- + + deleteTestTopic(topic); + } + @Test public void testSourceSinkWithKeyAndPartialValue() throws Exception { // we always use a different topic name for each parameterized topic, From df236ca7feb966edc6d4f499c3c70a8c2e0614ed Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 6 Apr 2023 16:02:24 -0700 Subject: [PATCH 013/322] [hotfix] Remove full qualifier for SinkWriter.Context in test --- .../connectors/kafka/table/ReducingUpsertWriterTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java index 5ef36e75b..f08ec1d25 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.connector.sink2.StatefulSink; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.catalog.Column; @@ -276,7 +277,7 @@ public void testWriteDataWithNullTimestamp() throws Exception { 11.11, 11, null), - new org.apache.flink.api.connector.sink2.SinkWriter.Context() { + new SinkWriter.Context() { @Override public long currentWatermark() { throw new UnsupportedOperationException("Not implemented."); From d06e027a084de5c2fb337565b2988ea9eafb8833 Mon Sep 17 00:00:00 2001 From: twalthr Date: Mon, 12 Feb 2018 18:18:44 +0100 Subject: [PATCH 014/322] [FLINK-8630] [formats] Add proper support for JSON formats This closes #5491. --- .../flink-json-debezium/pom.xml | 81 ++++ .../json/JsonNodeDeserializationSchema.java | 47 +++ .../json/JsonRowDeserializationSchema.java | 217 +++++++++++ .../json/JsonRowSerializationSchema.java | 208 ++++++++++ .../formats/json/JsonSchemaConverter.java | 361 ++++++++++++++++++ .../JsonNodeDeserializationSchemaTest.java | 47 +++ .../JsonRowDeserializationSchemaTest.java | 198 ++++++++++ .../json/JsonRowSerializationSchemaTest.java | 174 +++++++++ .../formats/json/JsonSchemaConverterTest.java | 114 ++++++ .../src/test/resources/complex-schema.json | 114 ++++++ .../src/test/resources/reference-schema.json | 61 +++ 11 files changed, 1622 insertions(+) create mode 100644 flink-formats-kafka/flink-json-debezium/pom.xml create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/complex-schema.json create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/reference-schema.json diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml new file mode 100644 index 000000000..d51f4fa02 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -0,0 +1,81 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-formats + 1.5-SNAPSHOT + .. + + + flink-json + flink-json + + jar + + + + + + + org.apache.flink + flink-shaded-jackson + provided + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + + + org.apache.flink + flink-core + ${project.version} + test + test-jar + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + + diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java new file mode 100644 index 000000000..7501cc3a8 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; + +import java.io.IOException; + +/** + * DeserializationSchema that deserializes a JSON String into an ObjectNode. + * + *

Fields can be accessed by calling objectNode.get(<name>).as(<type>) + */ +@PublicEvolving +public class JsonNodeDeserializationSchema extends AbstractDeserializationSchema { + + private static final long serialVersionUID = -1699854177598621044L; + + private ObjectMapper mapper; + + @Override + public ObjectNode deserialize(byte[] message) throws IOException { + if (mapper == null) { + mapper = new ObjectMapper(); + } + return mapper.readValue(message, ObjectNode.class); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java new file mode 100644 index 000000000..edc4b0115 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; +import java.lang.reflect.Array; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; + +/** + * Deserialization schema from JSON to Flink types. + * + *

Deserializes a byte[] message as a JSON object and reads + * the specified fields. + * + *

Failure during deserialization are forwarded as wrapped IOExceptions. + */ +@PublicEvolving +public class JsonRowDeserializationSchema implements DeserializationSchema { + + private static final long serialVersionUID = -228294330688809195L; + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Object mapper for parsing the JSON. */ + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + /** + * Creates a JSON deserialization schema for the given type information. + * + * @param typeInfo Type information describing the result type. The field names of {@link Row} + * are used to parse the JSON properties. + */ + public JsonRowDeserializationSchema(TypeInformation typeInfo) { + Preconditions.checkNotNull(typeInfo, "Type information"); + this.typeInfo = typeInfo; + + if (!(typeInfo instanceof RowTypeInfo)) { + throw new IllegalArgumentException("Row type information expected."); + } + } + + /** + * Creates a JSON deserialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * + * @see http://json-schema.org/ + */ + public JsonRowDeserializationSchema(String jsonSchema) { + this(JsonSchemaConverter.convert(jsonSchema)); + } + + @SuppressWarnings("unchecked") + @Override + public Row deserialize(byte[] message) throws IOException { + try { + final JsonNode root = objectMapper.readTree(message); + return convertRow(root, (RowTypeInfo) typeInfo); + } catch (Throwable t) { + throw new IOException("Failed to deserialize JSON object.", t); + } + } + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return typeInfo; + } + + /** + * Configures the failure behaviour if a JSON field is missing. + * + *

By default, a missing field is ignored and the field is set to null. + * + * @param failOnMissingField Flag indicating whether to fail or not on a missing field. + */ + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + + // -------------------------------------------------------------------------------------------- + + private Object convert(JsonNode node, TypeInformation info) { + if (info == Types.VOID || node.isNull()) { + return null; + } else if (info == Types.BOOLEAN) { + return node.asBoolean(); + } else if (info == Types.STRING) { + return node.asText(); + } else if (info == Types.BIG_DEC) { + return node.decimalValue(); + } else if (info == Types.BIG_INT) { + return node.bigIntegerValue(); + } else if (info == Types.SQL_DATE) { + return Date.valueOf(node.asText()); + } else if (info == Types.SQL_TIME) { + // according to RFC 3339 every full-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + final String time = node.asText(); + if (time.indexOf('Z') < 0 || time.indexOf('.') >= 0) { + throw new IllegalStateException( + "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet. " + + "Format: HH:mm:ss'Z'"); + } + return Time.valueOf(time.substring(0, time.length() - 1)); + } else if (info == Types.SQL_TIMESTAMP) { + // according to RFC 3339 every date-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + final String timestamp = node.asText(); + if (timestamp.indexOf('Z') < 0) { + throw new IllegalStateException( + "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + + "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + } + return Timestamp.valueOf(timestamp.substring(0, timestamp.length() - 1).replace('T', ' ')); + } else if (info instanceof RowTypeInfo) { + return convertRow(node, (RowTypeInfo) info); + } else if (info instanceof ObjectArrayTypeInfo) { + return convertObjectArray(node, ((ObjectArrayTypeInfo) info).getComponentInfo()); + } else if (info instanceof BasicArrayTypeInfo) { + return convertObjectArray(node, ((BasicArrayTypeInfo) info).getComponentInfo()); + } else if (info instanceof PrimitiveArrayTypeInfo && + ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return convertByteArray(node); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return objectMapper.treeToValue(node, info.getTypeClass()); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } + } + } + + private Row convertRow(JsonNode node, RowTypeInfo info) { + final String[] names = info.getFieldNames(); + final TypeInformation[] types = info.getFieldTypes(); + + final Row row = new Row(names.length); + for (int i = 0; i < names.length; i++) { + final String name = names[i]; + final JsonNode subNode = node.get(name); + if (subNode == null) { + if (failOnMissingField) { + throw new IllegalStateException( + "Could not find field with name '" + name + "'."); + } else { + row.setField(i, null); + } + } else { + row.setField(i, convert(subNode, types[i])); + } + } + + return row; + } + + private Object convertObjectArray(JsonNode node, TypeInformation elementType) { + final Object[] array = (Object[]) Array.newInstance(elementType.getTypeClass(), node.size()); + for (int i = 0; i < node.size(); i++) { + array[i] = convert(node.get(i), elementType); + } + return array; + } + + private Object convertByteArray(JsonNode node) { + try { + return node.binaryValue(); + } catch (IOException e) { + throw new RuntimeException("Unable to deserialize byte array.", e); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java new file mode 100644 index 000000000..8fee6a4d9 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Time; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; + +/** + * Serialization schema that serializes an object of Flink types into a JSON bytes. + * + *

Serializes the input Flink object into a JSON string and + * converts it into byte[]. + * + *

Result byte[] messages can be deserialized using {@link JsonRowDeserializationSchema}. + */ +@PublicEvolving +public class JsonRowSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = -2885556750743978636L; + + /** Type information describing the input type. */ + private final TypeInformation typeInfo; + + /** Object mapper that is used to create output JSON objects. */ + private final ObjectMapper mapper = new ObjectMapper(); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone, without milliseconds). */ + private SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm:ss'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone). */ + private SimpleDateFormat timeFormatWithMillis = new SimpleDateFormat("HH:mm:ss.SSS'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC timezone). */ + private SimpleDateFormat timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + + /** Reusable object node. */ + private transient ObjectNode node; + + /** + * Creates a JSON serialization schema for the given type information. + * + * @param typeInfo The field names of {@link Row} are used to map to JSON properties. + */ + public JsonRowSerializationSchema(TypeInformation typeInfo) { + Preconditions.checkNotNull(typeInfo, "Type information"); + this.typeInfo = typeInfo; + } + + /** + * Creates a JSON serialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * + * @see http://json-schema.org/ + */ + public JsonRowSerializationSchema(String jsonSchema) { + this(JsonSchemaConverter.convert(jsonSchema)); + } + + @Override + public byte[] serialize(Row row) { + if (node == null) { + node = mapper.createObjectNode(); + } + + try { + convertRow(node, (RowTypeInfo) typeInfo, row); + return mapper.writeValueAsBytes(node); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'. " + + "Make sure that the schema matches the input.", t); + } + } + + // -------------------------------------------------------------------------------------------- + + private ObjectNode convertRow(ObjectNode reuse, RowTypeInfo info, Row row) { + if (reuse == null) { + reuse = mapper.createObjectNode(); + } + final String[] fieldNames = info.getFieldNames(); + final TypeInformation[] fieldTypes = info.getFieldTypes(); + + // validate the row + if (row.getArity() != fieldNames.length) { + throw new IllegalStateException(String.format( + "Number of elements in the row '%s' is different from number of field names: %d", row, fieldNames.length)); + } + + for (int i = 0; i < fieldNames.length; i++) { + final String name = fieldNames[i]; + + final JsonNode fieldConverted = convert(reuse, reuse.get(name), fieldTypes[i], row.getField(i)); + reuse.set(name, fieldConverted); + } + + return reuse; + } + + private JsonNode convert(ContainerNode container, JsonNode reuse, TypeInformation info, Object object) { + if (info == Types.VOID || object == null) { + return container.nullNode(); + } else if (info == Types.BOOLEAN) { + return container.booleanNode((Boolean) object); + } else if (info == Types.STRING) { + return container.textNode((String) object); + } else if (info == Types.BIG_DEC) { + // convert decimal if necessary + if (object instanceof BigDecimal) { + return container.numberNode((BigDecimal) object); + } + return container.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); + } else if (info == Types.BIG_INT) { + // convert integer if necessary + if (object instanceof BigInteger) { + return container.numberNode((BigInteger) object); + } + return container.numberNode(BigInteger.valueOf(((Number) object).longValue())); + } else if (info == Types.SQL_DATE) { + return container.textNode(object.toString()); + } else if (info == Types.SQL_TIME) { + final Time time = (Time) object; + // strip milliseconds if possible + if (time.getTime() % 1000 > 0) { + return container.textNode(timeFormatWithMillis.format(time)); + } + return container.textNode(timeFormat.format(time)); + } else if (info == Types.SQL_TIMESTAMP) { + return container.textNode(timestampFormat.format((Timestamp) object)); + } else if (info instanceof RowTypeInfo) { + if (reuse != null && reuse instanceof ObjectNode) { + return convertRow((ObjectNode) reuse, (RowTypeInfo) info, (Row) object); + } else { + return convertRow(null, (RowTypeInfo) info, (Row) object); + } + } else if (info instanceof ObjectArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof BasicArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof PrimitiveArrayTypeInfo && ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return container.binaryNode((byte[]) object); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.valueToTree(object); + } catch (IllegalArgumentException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for object: " + object, e); + } + } + } + + private ArrayNode convertObjectArray(ArrayNode reuse, TypeInformation info, Object[] array) { + if (reuse == null) { + reuse = mapper.createArrayNode(); + } else { + reuse.removeAll(); + } + + for (Object object : array) { + reuse.add(convert(reuse, null, info, object)); + } + return reuse; + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java new file mode 100644 index 000000000..58495b0b6 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java @@ -0,0 +1,361 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * Converts a JSON schema into Flink's type information. It uses {@link Row} for representing + * objects and tuple arrays. + * + *

Note: This converter implements just a subset of the JSON schema specification. + * Union types (as well as "allOf", "anyOf", "not") are not supported yet. Simple + * references that link to a common definition in the document are supported. "oneOf" and + * arrays of types are only supported for specifying nullability. + * + *

This converter has been developed for JSON Schema draft-07 but also includes keywords of + * older drafts to be as compatible as possible. + */ +@SuppressWarnings("OptionalIsPresent") +public final class JsonSchemaConverter { + + private JsonSchemaConverter() { + // private + } + + // see https://spacetelescope.github.io/understanding-json-schema/UnderstandingJSONSchema.pdf + private static final String PROPERTIES = "properties"; + private static final String ADDITIONAL_PROPERTIES = "additionalProperties"; + private static final String TYPE = "type"; + private static final String FORMAT = "format"; + private static final String CONTENT_ENCODING = "contentEncoding"; + private static final String ITEMS = "items"; + private static final String ADDITIONAL_ITEMS = "additionalItems"; + private static final String REF = "$ref"; + private static final String ALL_OF = "allOf"; + private static final String ANY_OF = "anyOf"; + private static final String NOT = "not"; + private static final String ONE_OF = "oneOf"; + + // from https://tools.ietf.org/html/draft-zyp-json-schema-03#page-14 + private static final String DISALLOW = "disallow"; + private static final String EXTENDS = "extends"; + + private static final String TYPE_NULL = "null"; + private static final String TYPE_BOOLEAN = "boolean"; + private static final String TYPE_OBJECT = "object"; + private static final String TYPE_ARRAY = "array"; + private static final String TYPE_NUMBER = "number"; + private static final String TYPE_INTEGER = "integer"; + private static final String TYPE_STRING = "string"; + + private static final String FORMAT_DATE = "date"; + private static final String FORMAT_TIME = "time"; + private static final String FORMAT_DATE_TIME = "date-time"; + + private static final String CONTENT_ENCODING_BASE64 = "base64"; + + /** + * Converts a JSON schema into Flink's type information. Throws an exception if the schema + * cannot converted because of loss of precision or too flexible schema. + * + *

The converter can resolve simple schema references to solve those cases where entities + * are defined at the beginning and then used throughout a document. + */ + @SuppressWarnings("unchecked") + public static TypeInformation convert(String jsonSchema) { + Preconditions.checkNotNull(jsonSchema, "JSON schema"); + final ObjectMapper mapper = new ObjectMapper(); + mapper.getFactory() + .enable(JsonParser.Feature.ALLOW_COMMENTS) + .enable(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES) + .enable(JsonParser.Feature.ALLOW_SINGLE_QUOTES); + final JsonNode node; + try { + node = mapper.readTree(jsonSchema); + } catch (IOException e) { + throw new IllegalArgumentException( + "Invalid JSON schema.", e); + } + return (TypeInformation) convertType("", node, node); + } + + private static TypeInformation convertType(String location, JsonNode node, JsonNode root) { + // we use a set here to unify types (e.g. types that just add metadata such as 'multipleOf') + final Set> typeSet = new HashSet<>(); + + // search for ref + final Optional ref; + if (node.has(REF) && node.get(REF).isTextual()) { + // try a simple ref resolver to solve those cases where entities are defined at + // the beginning and then used throughout a document + ref = Optional.of(resolveReference(node.get(REF).asText(), node, root)); + } else { + ref = Optional.empty(); + } + + // use TYPE of this node + if (node.has(TYPE)) { + final JsonNode typeNode = node.get(TYPE); + + List types = new ArrayList<>(); + // array of types + if (typeNode.isArray()) { + final Iterator elements = typeNode.elements(); + while (elements.hasNext()) { + types.add(elements.next().asText()); + } + } + // single type + else if (typeNode.isTextual()) { + types.add(typeNode.asText()); + } + + for (String type : types) { + // set field type + switch (type) { + case TYPE_NULL: + typeSet.add(Types.VOID); + break; + case TYPE_BOOLEAN: + typeSet.add(Types.BOOLEAN); + break; + case TYPE_STRING: + if (node.has(FORMAT)) { + typeSet.add(convertStringFormat(location, node.get(FORMAT))); + } else if (node.has(CONTENT_ENCODING)) { + typeSet.add(convertStringEncoding(location, node.get(CONTENT_ENCODING))); + } else { + typeSet.add(Types.STRING); + } + break; + case TYPE_NUMBER: + typeSet.add(Types.BIG_DEC); + break; + case TYPE_INTEGER: + typeSet.add(Types.BIG_INT); + break; + case TYPE_OBJECT: + typeSet.add(convertObject(location, node, root)); + break; + case TYPE_ARRAY: + typeSet.add(convertArray(location, node, root)); + break; + default: + throw new IllegalArgumentException( + "Unsupported type '" + node.get(TYPE).asText() + "' in node: " + location); + } + } + } + // use TYPE of reference as fallback + else if (ref.isPresent() && ref.get().has(TYPE)) { + typeSet.add(convertType(node.get(REF).asText(), ref.get(), root)); + } + + // simple interpretation of ONE_OF for supporting "object or null" + if (node.has(ONE_OF) && node.get(ONE_OF).isArray()) { + final TypeInformation[] types = convertTypes(location + '/' + ONE_OF, node.get(ONE_OF), root); + typeSet.addAll(Arrays.asList(types)); + } + // use ONE_OF of reference as fallback + else if (ref.isPresent() && ref.get().has(ONE_OF) && ref.get().get(ONE_OF).isArray()) { + final TypeInformation[] types = convertTypes(node.get(REF).asText() + '/' + ONE_OF, ref.get().get(ONE_OF), root); + typeSet.addAll(Arrays.asList(types)); + } + + // validate no union types or extending + if (node.has(ALL_OF) || node.has(ANY_OF) || node.has(NOT) || node.has(EXTENDS) || node.has(DISALLOW)) { + throw new IllegalArgumentException( + "Union types are such as '" + ALL_OF + "', '" + ANY_OF + "' etc. " + + "and extending are not supported yet."); + } + + // only a type (with null) is supported yet + final List> types = new ArrayList<>(typeSet); + if (types.size() == 0) { + throw new IllegalArgumentException("No type could be found in node:" + location); + } else if (types.size() > 2 || (types.size() == 2 && !types.contains(Types.VOID))) { + throw new IllegalArgumentException( + "Union types with more than just a null type are not supported yet."); + } + + // return the first non-void type or void + if (types.size() == 2 && types.get(0) == Types.VOID) { + return types.get(1); + } else { + return types.get(0); + } + } + + private static TypeInformation convertObject(String location, JsonNode node, JsonNode root) { + // validate properties + if (!node.has(PROPERTIES)) { + return Types.ROW(); + } + if (!node.isObject()) { + throw new IllegalArgumentException( + "Invalid '" + PROPERTIES + "' property for object type in node: " + location); + } + final JsonNode props = node.get(PROPERTIES); + final String[] names = new String[props.size()]; + final TypeInformation[] types = new TypeInformation[props.size()]; + + final Iterator> fieldIter = props.fields(); + int i = 0; + while (fieldIter.hasNext()) { + final Map.Entry subNode = fieldIter.next(); + + // set field name + names[i] = subNode.getKey(); + + // set type + types[i] = convertType(location + '/' + subNode.getKey(), subNode.getValue(), root); + + i++; + } + + // validate that object does not contain additional properties + if (node.has(ADDITIONAL_PROPERTIES) && node.get(ADDITIONAL_PROPERTIES).isBoolean() && + node.get(ADDITIONAL_PROPERTIES).asBoolean()) { + throw new IllegalArgumentException( + "An object must not allow additional properties in node: " + location); + } + + return Types.ROW_NAMED(names, types); + } + + private static TypeInformation convertArray(String location, JsonNode node, JsonNode root) { + // validate items + if (!node.has(ITEMS)) { + throw new IllegalArgumentException( + "Arrays must specify an '" + ITEMS + "' property in node: " + location); + } + final JsonNode items = node.get(ITEMS); + + // list (translated to object array) + if (items.isObject()) { + final TypeInformation elementType = convertType( + location + '/' + ITEMS, + items, + root); + // result type might either be ObjectArrayTypeInfo or BasicArrayTypeInfo for Strings + return Types.OBJECT_ARRAY(elementType); + } + // tuple (translated to row) + else if (items.isArray()) { + final TypeInformation[] types = convertTypes(location + '/' + ITEMS, items, root); + + // validate that array does not contain additional items + if (node.has(ADDITIONAL_ITEMS) && node.get(ADDITIONAL_ITEMS).isBoolean() && + node.get(ADDITIONAL_ITEMS).asBoolean()) { + throw new IllegalArgumentException( + "An array tuple must not allow additional items in node: " + location); + } + + return Types.ROW(types); + } + throw new IllegalArgumentException( + "Invalid type for '" + ITEMS + "' property in node: " + location); + } + + private static TypeInformation convertStringFormat(String location, JsonNode node) { + if (!node.isTextual()) { + throw new IllegalArgumentException("Invalid '" + FORMAT + "' property in node: " + location); + } + + switch (node.asText()) { + case FORMAT_DATE: + return Types.SQL_DATE; + case FORMAT_TIME: + return Types.SQL_TIME; + case FORMAT_DATE_TIME: + return Types.SQL_TIMESTAMP; + default: + return Types.STRING; // unlikely that we will support other formats in the future + } + } + + private static TypeInformation convertStringEncoding(String location, JsonNode node) { + if (!node.isTextual()) { + throw new IllegalArgumentException("Invalid '" + CONTENT_ENCODING + "' property in node: " + location); + } + + // "If the instance value is a string, this property defines that the string SHOULD + // be interpreted as binary data and decoded using the encoding named by this property." + + switch (node.asText()) { + case CONTENT_ENCODING_BASE64: + return Types.PRIMITIVE_ARRAY(Types.BYTE); + default: + // we fail hard here: + // this gives us the chance to support more encodings in the future without problems + // of backwards compatibility + throw new IllegalArgumentException("Invalid encoding '" + node.asText() + "' in node: " + location); + } + } + + private static JsonNode resolveReference(String ref, JsonNode origin, JsonNode root) { + if (!ref.startsWith("#")) { + throw new IllegalArgumentException("Only JSON schemes with simple references " + + "(one indirection in the same document) are supported yet. But was: " + ref); + } + final String path = ref.substring(1); + final JsonNode foundNode = root.at(path); + if (foundNode.isMissingNode()) { + throw new IllegalArgumentException("Could not find reference: " + ref); + } + // prevent obvious cyclic references + if (foundNode == origin) { + throw new IllegalArgumentException("Cyclic references are not supported:" + ref); + } + return foundNode; + } + + private static TypeInformation[] convertTypes(String location, JsonNode arrayNode, JsonNode root) { + final TypeInformation[] types = new TypeInformation[arrayNode.size()]; + final Iterator elements = arrayNode.elements(); + int i = 0; + while (elements.hasNext()) { + final TypeInformation elementType = convertType( + location + '[' + i + ']', + elements.next(), + root); + types[i] = elementType; + i += 1; + } + return types; + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java new file mode 100644 index 000000000..1462a262e --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for the {@link JsonNodeDeserializationSchema}. + */ +public class JsonNodeDeserializationSchemaTest { + + @Test + public void testDeserialize() throws IOException { + ObjectMapper mapper = new ObjectMapper(); + ObjectNode initialValue = mapper.createObjectNode(); + initialValue.put("key", 4).put("value", "world"); + byte[] serializedValue = mapper.writeValueAsBytes(initialValue); + + JsonNodeDeserializationSchema schema = new JsonNodeDeserializationSchema(); + ObjectNode deserializedValue = schema.deserialize(serializedValue); + + assertEquals(4, deserializedValue.get("key").asInt()); + assertEquals("world", deserializedValue.get("value").asText()); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java new file mode 100644 index 000000000..1c4a17a8a --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.types.Row; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.concurrent.ThreadLocalRandom; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +/** + * Tests for the {@link JsonRowDeserializationSchema}. + */ +public class JsonRowDeserializationSchemaTest { + + /** + * Tests simple deserialization using type information. + */ + @Test + public void testTypeInfoDeserialization() throws Exception { + long id = 1238123899121L; + String name = "asdlkjasjkdla998y1122"; + byte[] bytes = new byte[1024]; + ThreadLocalRandom.current().nextBytes(bytes); + + ObjectMapper objectMapper = new ObjectMapper(); + + // Root + ObjectNode root = objectMapper.createObjectNode(); + root.put("id", id); + root.put("name", name); + root.put("bytes", bytes); + + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema( + Types.ROW_NAMED( + new String[] { "id", "name", "bytes" }, + Types.LONG, Types.STRING, Types.PRIMITIVE_ARRAY(Types.BYTE)) + ); + + Row deserialized = deserializationSchema.deserialize(serializedJson); + + assertEquals(3, deserialized.getArity()); + assertEquals(id, deserialized.getField(0)); + assertEquals(name, deserialized.getField(1)); + assertArrayEquals(bytes, (byte[]) deserialized.getField(2)); + } + + @Test + public void testSchemaDeserialization() throws Exception { + final BigInteger id = BigInteger.valueOf(1238123899121L); + final String name = "asdlkjasjkdla998y1122"; + final byte[] bytes = new byte[1024]; + ThreadLocalRandom.current().nextBytes(bytes); + final BigInteger[] numbers = new BigInteger[] { + BigInteger.valueOf(1), BigInteger.valueOf(2), BigInteger.valueOf(3)}; + final String[] strings = new String[] {"one", "two", "three"}; + + final ObjectMapper objectMapper = new ObjectMapper(); + + // Root + ObjectNode root = objectMapper.createObjectNode(); + root.put("id", id.longValue()); + root.putNull("idOrNull"); + root.put("name", name); + root.put("date", "1990-10-14"); + root.put("time", "12:12:43Z"); + root.put("timestamp", "1990-10-14T12:12:43Z"); + root.put("bytes", bytes); + root.putArray("numbers").add(1).add(2).add(3); + root.putArray("strings").add("one").add("two").add("three"); + root.putObject("nested").put("booleanField", true).put("decimalField", 12); + + final byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema( + "{" + + " type: 'object'," + + " properties: {" + + " id: { type: 'integer' }," + + " idOrNull: { type: ['integer', 'null'] }," + + " name: { type: 'string' }," + + " date: { type: 'string', format: 'date' }," + + " time: { type: 'string', format: 'time' }," + + " timestamp: { type: 'string', format: 'date-time' }," + + " bytes: { type: 'string', contentEncoding: 'base64' }," + + " numbers: { type: 'array', items: { type: 'integer' } }," + + " strings: { type: 'array', items: { type: 'string' } }," + + " nested: { " + + " type: 'object'," + + " properties: { " + + " booleanField: { type: 'boolean' }," + + " decimalField: { type: 'number' }" + + " }" + + " }" + + " }" + + "}"); + + final Row deserialized = deserializationSchema.deserialize(serializedJson); + + final Row expected = new Row(10); + expected.setField(0, id); + expected.setField(1, null); + expected.setField(2, name); + expected.setField(3, Date.valueOf("1990-10-14")); + expected.setField(4, Time.valueOf("12:12:43")); + expected.setField(5, Timestamp.valueOf("1990-10-14 12:12:43")); + expected.setField(6, bytes); + expected.setField(7, numbers); + expected.setField(8, strings); + final Row nestedRow = new Row(2); + nestedRow.setField(0, true); + nestedRow.setField(1, BigDecimal.valueOf(12)); + expected.setField(9, nestedRow); + + assertEquals(expected, deserialized); + } + + /** + * Tests deserialization with non-existing field name. + */ + @Test + public void testMissingNode() throws Exception { + ObjectMapper objectMapper = new ObjectMapper(); + + // Root + ObjectNode root = objectMapper.createObjectNode(); + root.put("id", 123123123); + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema( + Types.ROW_NAMED( + new String[] { "name" }, + Types.STRING) + ); + + Row row = deserializationSchema.deserialize(serializedJson); + + assertEquals(1, row.getArity()); + Assert.assertNull("Missing field not null", row.getField(0)); + + deserializationSchema.setFailOnMissingField(true); + + try { + deserializationSchema.deserialize(serializedJson); + Assert.fail("Did not throw expected Exception"); + } catch (IOException e) { + Assert.assertTrue(e.getCause() instanceof IllegalStateException); + } + } + + /** + * Tests that number of field names and types has to match. + */ + @Test + public void testNumberOfFieldNamesAndTypesMismatch() { + try { + new JsonRowDeserializationSchema( + Types.ROW_NAMED( + new String[]{"one", "two", "three"}, + Types.LONG)); + Assert.fail("Did not throw expected Exception"); + } catch (IllegalArgumentException ignored) { + // Expected + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java new file mode 100644 index 000000000..f956fc640 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.types.Row; + +import org.junit.Test; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.concurrent.ThreadLocalRandom; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for the {@link JsonRowSerializationSchema}. + */ +public class JsonRowSerializationSchemaTest { + + @Test + public void testRowSerialization() throws IOException { + final TypeInformation rowSchema = Types.ROW_NAMED( + new String[] {"f1", "f2", "f3"}, + Types.INT, Types.BOOLEAN, Types.STRING); + + final Row row = new Row(3); + row.setField(0, 1); + row.setField(1, true); + row.setField(2, "str"); + + final Row resultRow = serializeAndDeserialize(rowSchema, row); + assertEquals(row, resultRow); + } + + @Test + public void testSerializationOfTwoRows() throws IOException { + final TypeInformation rowSchema = Types.ROW_NAMED( + new String[] {"f1", "f2", "f3"}, + Types.INT, Types.BOOLEAN, Types.STRING); + + final Row row1 = new Row(3); + row1.setField(0, 1); + row1.setField(1, true); + row1.setField(2, "str"); + + final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema); + final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema); + + byte[] bytes = serializationSchema.serialize(row1); + assertEquals(row1, deserializationSchema.deserialize(bytes)); + + final Row row2 = new Row(3); + row2.setField(0, 10); + row2.setField(1, false); + row2.setField(2, "newStr"); + + bytes = serializationSchema.serialize(row2); + assertEquals(row2, deserializationSchema.deserialize(bytes)); + } + + @Test + public void testNestedSchema() throws IOException { + final TypeInformation rowSchema = Types.ROW_NAMED( + new String[] {"f1", "f2", "f3"}, + Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.DOUBLE)); + + final Row row = new Row(3); + row.setField(0, 42); + row.setField(1, false); + final Row nested = new Row(2); + nested.setField(0, 22); + nested.setField(1, 2.3); + row.setField(2, nested); + + final Row resultRow = serializeAndDeserialize(rowSchema, row); + assertEquals(row, resultRow); + } + + @Test(expected = RuntimeException.class) + public void testSerializeRowWithInvalidNumberOfFields() { + final TypeInformation rowSchema = Types.ROW_NAMED( + new String[] {"f1", "f2", "f3"}, + Types.INT, Types.BOOLEAN, Types.STRING); + + final Row row = new Row(1); + row.setField(0, 1); + + final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema); + serializationSchema.serialize(row); + } + + @Test + public void testSchema() throws IOException { + final TypeInformation rowSchema = JsonSchemaConverter.convert( + "{" + + " type: 'object'," + + " properties: {" + + " id: { type: 'integer' }," + + " idNumber: { type: 'number' }," + + " idOrNull: { type: ['integer', 'null'] }," + + " name: { type: 'string' }," + + " date: { type: 'string', format: 'date' }," + + " time: { type: 'string', format: 'time' }," + + " timestamp: { type: 'string', format: 'date-time' }," + + " bytes: { type: 'string', contentEncoding: 'base64' }," + + " numbers: { type: 'array', items: { type: 'integer' } }," + + " strings: { type: 'array', items: { type: 'string' } }," + + " nested: { " + + " type: 'object'," + + " properties: { " + + " booleanField: { type: 'boolean' }," + + " decimalField: { type: 'number' }" + + " }" + + " }" + + " }" + + "}"); + + final Row row = new Row(11); + row.setField(0, BigInteger.valueOf(-333)); + row.setField(1, BigDecimal.valueOf(12.2222)); + row.setField(2, null); + row.setField(3, ""); + row.setField(4, Date.valueOf("1990-10-14")); + row.setField(5, Time.valueOf("12:12:43")); + row.setField(6, Timestamp.valueOf("1990-10-14 12:12:43")); + + final byte[] bytes = new byte[1024]; + ThreadLocalRandom.current().nextBytes(bytes); + row.setField(7, bytes); + final BigInteger[] numbers = new BigInteger[] { + BigInteger.valueOf(1), BigInteger.valueOf(2), BigInteger.valueOf(3)}; + row.setField(8, numbers); + final String[] strings = new String[] {"one", "two", "three"}; + row.setField(9, strings); + final Row nestedRow = new Row(2); + nestedRow.setField(0, true); + nestedRow.setField(1, BigDecimal.valueOf(12)); + row.setField(10, nestedRow); + + final Row resultRow = serializeAndDeserialize(rowSchema, row); + assertEquals(row, resultRow); + } + + // -------------------------------------------------------------------------------------------- + + private Row serializeAndDeserialize(TypeInformation rowSchema, Row row) throws IOException { + final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema); + final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema); + + final byte[] bytes = serializationSchema.serialize(row); + return deserializationSchema.deserialize(bytes); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java new file mode 100644 index 000000000..d0726639a --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.util.FileUtils; + +import org.junit.Test; + +import java.io.File; +import java.net.URL; +import java.util.Objects; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link JsonSchemaConverter}. + */ +public class JsonSchemaConverterTest { + + @Test + public void testComplexSchema() throws Exception { + final URL url = getClass().getClassLoader().getResource("complex-schema.json"); + Objects.requireNonNull(url); + final String schema = FileUtils.readFileUtf8(new File(url.getFile())); + final TypeInformation result = JsonSchemaConverter.convert(schema); + + final TypeInformation expected = Types.ROW_NAMED( + new String[] {"fn", "familyName", "additionalName", "tuples", "honorificPrefix", "url", + "email", "tel", "sound", "org"}, + Types.STRING, Types.STRING, Types.BOOLEAN, Types.ROW(Types.BIG_DEC, Types.STRING, Types.STRING, Types.STRING), + Types.OBJECT_ARRAY(Types.STRING), Types.STRING, Types.ROW_NAMED(new String[] {"type", "value"}, Types.STRING, Types.STRING), + Types.ROW_NAMED(new String[] {"type", "value"}, Types.BIG_INT, Types.STRING), Types.VOID, + Types.ROW_NAMED(new String[] {"organizationUnit"}, Types.ROW())); + + assertEquals(expected, result); + } + + @Test + public void testReferenceSchema() throws Exception { + final URL url = getClass().getClassLoader().getResource("reference-schema.json"); + Objects.requireNonNull(url); + final String schema = FileUtils.readFileUtf8(new File(url.getFile())); + final TypeInformation result = JsonSchemaConverter.convert(schema); + + final TypeInformation expected = Types.ROW_NAMED( + new String[] {"billing_address", "shipping_address", "optional_address"}, + Types.ROW_NAMED(new String[] {"street_address", "city", "state"}, Types.STRING, Types.STRING, Types.STRING), + Types.ROW_NAMED(new String[] {"street_address", "city", "state"}, Types.STRING, Types.STRING, Types.STRING), + Types.ROW_NAMED(new String[] {"street_address", "city", "state"}, Types.STRING, Types.STRING, Types.STRING)); + + assertEquals(expected, result); + } + + @Test + public void testAtomicType() { + final TypeInformation result = JsonSchemaConverter.convert("{ type: 'number' }"); + + assertEquals(Types.BIG_DEC, result); + } + + @Test(expected = IllegalArgumentException.class) + public void testMissingType() { + JsonSchemaConverter.convert("{ }"); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongType() { + JsonSchemaConverter.convert("{ type: 'whatever' }"); + } + + @Test(expected = IllegalArgumentException.class) + public void testArrayWithAdditionalItems() { + JsonSchemaConverter.convert("{ type: 'array', items: [{type: 'integer'}], additionalItems: true }"); + } + + @Test + public void testMissingProperties() { + final TypeInformation result = JsonSchemaConverter.convert("{ type: 'object' }"); + + assertEquals(Types.ROW(), result); + } + + @Test + public void testNullUnionTypes() { + final TypeInformation result = JsonSchemaConverter.convert("{ type: ['string', 'null'] }"); + + assertEquals(Types.STRING, result); + } + + @Test + public void testTimestamp() { + final TypeInformation result = JsonSchemaConverter.convert("{ type: 'string', format: 'date-time' }"); + + assertEquals(Types.SQL_TIMESTAMP, result); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/complex-schema.json b/flink-formats-kafka/flink-json-debezium/src/test/resources/complex-schema.json new file mode 100644 index 000000000..86664284e --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/complex-schema.json @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +{ + "$schema": "http://json-schema.org/draft-06/schema#", + "description": "A representation of a person, company, organization, or place", + "type": "object", + "required": [ + "familyName", + "givenName" + ], + "properties": { + "fn": { + "description": "Formatted Name", + "type": "string" + }, + "familyName": { + "type": "string" + }, + "additionalName": { + "type": "boolean" + }, + "tuples": { + "type": "array", + "items": [ + { + "type": "number" + }, + { + "type": "string" + }, + { + "type": "string", + "enum": [ + "Street", + "Avenue", + "Boulevard" + ] + }, + { + "type": "string", + "enum": [ + "NW", + "NE", + "SW", + "SE" + ] + } + ], + "additionalItems": false + }, + "honorificPrefix": { + "type": "array", + "items": { + "type": "string" + } + }, + "url": { + "type": "string", + "format": "uri" + }, + "email": { + "type": "object", + "properties": { + "type": { + "type": "string" + }, + "value": { + "type": "string", + "format": "email" + } + } + }, + tel: { + "type": "object", + "properties": { + "type": { + "type": "integer" + }, + "value": { + "type": "string", + "format": "phone" + } + } + }, + "sound": { + "type": "null" + }, + "org": { + "type": "object", + "properties": { + "organizationUnit": { + "type": "object", + "properties": {} + } + } + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/reference-schema.json b/flink-formats-kafka/flink-json-debezium/src/test/resources/reference-schema.json new file mode 100644 index 000000000..99e0e79dc --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/reference-schema.json @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +{ + "$schema": "http://json-schema.org/draft-04/schema#", + "definitions": { + "address": { + "type": "object", + "properties": { + "street_address": { + "type": "string" + }, + "city": { + "type": "string" + }, + "state": { + "type": "string" + } + }, + "required": [ + "street_address", + "city", + "state" + ] + } + }, + "type": "object", + "properties": { + "billing_address": { + "$ref": "#/definitions/address" + }, + "shipping_address": { + "$ref": "#/definitions/address" + }, + "optional_address": { + "oneOf": [ + { + "type": "null" + }, + { + "$ref": "#/definitions/address" + } + ] + } + } +} From 231c7b5a3de6255f10d22d573669279021deea98 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 19 Feb 2018 14:04:27 +0100 Subject: [PATCH 015/322] [hotfix] [formats] Dependency and code clean-up --- .../flink-json-debezium/pom.xml | 26 ------------------- .../formats/json/JsonSchemaConverter.java | 7 +++-- 2 files changed, 3 insertions(+), 30 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index d51f4fa02..ccc48f4e4 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -50,32 +50,6 @@ under the License. ${project.version} provided - - - - - org.apache.flink - flink-core - ${project.version} - test - test-jar - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - - - diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java index 58495b0b6..5b39f17a5 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java @@ -49,7 +49,6 @@ *

This converter has been developed for JSON Schema draft-07 but also includes keywords of * older drafts to be as compatible as possible. */ -@SuppressWarnings("OptionalIsPresent") public final class JsonSchemaConverter { private JsonSchemaConverter() { @@ -180,9 +179,9 @@ else if (typeNode.isTextual()) { } } } - // use TYPE of reference as fallback - else if (ref.isPresent() && ref.get().has(TYPE)) { - typeSet.add(convertType(node.get(REF).asText(), ref.get(), root)); + // use TYPE of reference as fallback if present + else { + ref.filter(r -> r.has(TYPE)).ifPresent(r -> typeSet.add(convertType(node.get(REF).asText(), r, root))); } // simple interpretation of ONE_OF for supporting "object or null" From 11e88a26789fe5fb90ed982b7e9dac6832734b2d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 27 Feb 2018 14:44:35 +0100 Subject: [PATCH 016/322] Update version to 1.6-SNAPSHOT --- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index ccc48f4e4..d0f55ab43 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.5-SNAPSHOT + 1.6-SNAPSHOT .. From 8463b4075e32b0afa9b8ed8721a0b9f525de21f2 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 19 Feb 2018 13:35:45 +0100 Subject: [PATCH 017/322] [FLINK-8538] [table] Improve unified table sources This closes #5564. --- .../flink-json-debezium/pom.xml | 29 +++- .../apache/flink/table/descriptors/Json.java | 129 ++++++++++++++++++ .../table/descriptors/JsonValidator.java | 55 ++++++++ .../flink/table/descriptors/JsonTest.java | 124 +++++++++++++++++ 4 files changed, 336 insertions(+), 1 deletion(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index d0f55ab43..3a80b0eaa 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -50,6 +50,33 @@ under the License. ${project.version} provided - + + org.apache.flink + + flink-table_2.11 + ${project.version} + provided + + true + + + + + + org.apache.flink + + flink-table_2.11 + ${project.version} + test-jar + test + + + + + org.scala-lang + scala-compiler + test + + diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java new file mode 100644 index 000000000..9c121916d --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.descriptors; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.typeutils.TypeStringUtils; +import org.apache.flink.util.Preconditions; + +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_JSON_SCHEMA; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_SCHEMA; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_TYPE_VALUE; + +/** + * Format descriptor for JSON. + */ +public class Json extends FormatDescriptor { + + private Boolean failOnMissingField; + private Boolean deriveSchema; + private String jsonSchema; + private String schema; + + /** + * Format descriptor for JSON. + */ + public Json() { + super(FORMAT_TYPE_VALUE, 1); + } + + /** + * Sets flag whether to fail if a field is missing or not. + * + * @param failOnMissingField If set to true, the operation fails if there is a missing field. + * If set to false, a missing field is set to null. + */ + public Json failOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + return this; + } + + /** + * Sets the JSON schema string with field names and the types according to the JSON schema + * specification [[http://json-schema.org/specification.html]]. + * + *

The schema might be nested. + * + * @param jsonSchema JSON schema + */ + public Json jsonSchema(String jsonSchema) { + Preconditions.checkNotNull(jsonSchema); + this.jsonSchema = jsonSchema; + this.schema = null; + this.deriveSchema = null; + return this; + } + + /** + * Sets the schema using type information. + * + *

JSON objects are represented as ROW types. + * + *

The schema might be nested. + * + * @param schemaType type information that describes the schema + */ + public Json schema(TypeInformation schemaType) { + Preconditions.checkNotNull(schemaType); + this.schema = TypeStringUtils.writeTypeInfo(schemaType); + this.jsonSchema = null; + this.deriveSchema = null; + return this; + } + + /** + * Derives the format schema from the table's schema described using {@link Schema}. + * + *

This allows for defining schema information only once. + * + *

The names, types, and field order of the format are determined by the table's + * schema. Time attributes are ignored. A "from" definition is interpreted as a field renaming + * in the format. + */ + public Json deriveSchema() { + this.deriveSchema = true; + this.schema = null; + this.jsonSchema = null; + return this; + } + + /** + * Internal method for format properties conversion. + */ + @Override + public void addFormatProperties(DescriptorProperties properties) { + if (deriveSchema != null) { + properties.putBoolean(FORMAT_DERIVE_SCHEMA(), deriveSchema); + } + + if (jsonSchema != null) { + properties.putString(FORMAT_JSON_SCHEMA, jsonSchema); + } + + if (schema != null) { + properties.putString(FORMAT_SCHEMA, schema); + } + + if (failOnMissingField != null) { + properties.putBoolean(FORMAT_FAIL_ON_MISSING_FIELD, failOnMissingField); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java new file mode 100644 index 000000000..fea7cf55b --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.descriptors; + +import org.apache.flink.table.api.ValidationException; + +/** + * Validator for {@link Json}. + */ +public class JsonValidator extends FormatDescriptorValidator { + + public static final String FORMAT_TYPE_VALUE = "json"; + public static final String FORMAT_SCHEMA = "format.schema"; + public static final String FORMAT_JSON_SCHEMA = "format.json-schema"; + public static final String FORMAT_FAIL_ON_MISSING_FIELD = "format.fail-on-missing-field"; + + @Override + public void validate(DescriptorProperties properties) { + super.validate(properties); + properties.validateBoolean(FORMAT_DERIVE_SCHEMA(), true); + final boolean deriveSchema = properties.getOptionalBoolean(FORMAT_DERIVE_SCHEMA()).orElse(false); + final boolean hasSchema = properties.containsKey(FORMAT_SCHEMA); + final boolean hasSchemaString = properties.containsKey(FORMAT_JSON_SCHEMA); + if (deriveSchema && (hasSchema || hasSchemaString)) { + throw new ValidationException( + "Format cannot define a schema and derive from the table's schema at the same time."); + } else if (!deriveSchema && hasSchema && hasSchemaString) { + throw new ValidationException("A definition of both a schema and JSON schema is not allowed."); + } else if (!deriveSchema && !hasSchema && !hasSchemaString) { + throw new ValidationException("A definition of a schema or JSON schema is required."); + } else if (hasSchema) { + properties.validateType(FORMAT_SCHEMA, false); + } else if (hasSchemaString) { + properties.validateString(FORMAT_JSON_SCHEMA, false, 1); + } + + properties.validateBoolean(FORMAT_FAIL_ON_MISSING_FIELD, true); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java new file mode 100644 index 000000000..6e370a02c --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.descriptors; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.Types; +import org.apache.flink.table.api.ValidationException; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Tests for the {@link Json} descriptor. + */ +public class JsonTest extends DescriptorTestBase { + + private static final String JSON_SCHEMA = + "{" + + " 'title': 'Person'," + + " 'type': 'object'," + + " 'properties': {" + + " 'firstName': {" + + " 'type': 'string'" + + " }," + + " 'lastName': {" + + " 'type': 'string'" + + " }," + + " 'age': {" + + " 'description': 'Age in years'," + + " 'type': 'integer'," + + " 'minimum': 0" + + " }" + + " }," + + " 'required': ['firstName', 'lastName']" + + "}"; + + @Test(expected = ValidationException.class) + public void testInvalidMissingField() { + addPropertyAndVerify(descriptors().get(0), "format.fail-on-missing-field", "DDD"); + } + + @Test(expected = ValidationException.class) + public void testMissingSchema() { + removePropertyAndVerify(descriptors().get(0), "format.json-schema"); + } + + @Test(expected = ValidationException.class) + public void testDuplicateSchema() { + // we add an additional non-json schema + addPropertyAndVerify(descriptors().get(0), "format.schema", "DDD"); + } + + // -------------------------------------------------------------------------------------------- + + @Override + public List descriptors() { + final Descriptor desc1 = new Json().jsonSchema("test"); + + final Descriptor desc2 = new Json().jsonSchema(JSON_SCHEMA).failOnMissingField(true); + + final Descriptor desc3 = new Json() + .schema( + Types.ROW( + new String[]{"test1", "test2"}, + new TypeInformation[]{Types.STRING(), Types.SQL_TIMESTAMP()})) + .failOnMissingField(true); + + final Descriptor desc4 = new Json().deriveSchema(); + + return Arrays.asList(desc1, desc2, desc3, desc4); + } + + @Override + public List> properties() { + final Map props1 = new HashMap<>(); + props1.put("format.type", "json"); + props1.put("format.property-version", "1"); + props1.put("format.json-schema", "test"); + + final Map props2 = new HashMap<>(); + props2.put("format.type", "json"); + props2.put("format.property-version", "1"); + props2.put("format.json-schema", JSON_SCHEMA); + props2.put("format.fail-on-missing-field", "true"); + + final Map props3 = new HashMap<>(); + props3.put("format.type", "json"); + props3.put("format.property-version", "1"); + props3.put("format.schema", "ROW(test1 VARCHAR, test2 TIMESTAMP)"); + props3.put("format.fail-on-missing-field", "true"); + + final Map props4 = new HashMap<>(); + props4.put("format.type", "json"); + props4.put("format.property-version", "1"); + props4.put("format.derive-schema", "true"); + + return Arrays.asList(props1, props2, props3, props4); + } + + @Override + public DescriptorValidator validator() { + return new JsonValidator(); + } +} From 78b0f409c77514b1339cf76d887badc66f43eb94 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 1 Mar 2018 10:58:14 +0100 Subject: [PATCH 018/322] [hotfix] [formats] Make ObjectMapper final in JsonNodeDeserializationSchema --- .../flink/formats/json/JsonNodeDeserializationSchema.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java index 7501cc3a8..e20c83977 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java @@ -35,13 +35,10 @@ public class JsonNodeDeserializationSchema extends AbstractDeserializationSchema private static final long serialVersionUID = -1699854177598621044L; - private ObjectMapper mapper; + private final ObjectMapper mapper = new ObjectMapper(); @Override public ObjectNode deserialize(byte[] message) throws IOException { - if (mapper == null) { - mapper = new ObjectMapper(); - } return mapper.readValue(message, ObjectNode.class); } } From 4a20e47ca1cb1ecadd2eb5328252fc6fad3b9506 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 14 Mar 2018 18:11:34 +0100 Subject: [PATCH 019/322] [FLINK-8833] [sql-client] Create a SQL Client JSON format fat-jar This closes #5700. --- .../flink-json-debezium/pom.xml | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 3a80b0eaa..2c8db2da7 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -63,6 +63,13 @@ under the License. + + org.apache.flink + flink-test-utils-junit + ${project.version} + test + + org.apache.flink @@ -79,4 +86,30 @@ under the License. test + + + + + release + + + + org.apache.maven.plugins + maven-jar-plugin + + + package + + jar + + + sql-jar + + + + + + + + From 1baf8b5e63a87c9df7ff0eeba5b5bfe82f929f99 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 11 May 2018 18:57:26 +0200 Subject: [PATCH 020/322] [FLINK-9338] Implemented RegistryAvroDeserializationSchema & provided implementation for Confluent Schema Registry This closes #5995 --- .../flink-avro-confluent-registry/pom.xml | 94 ++++++++++++ ...uentRegistryAvroDeserializationSchema.java | 136 ++++++++++++++++++ .../ConfluentSchemaRegistryCoder.java | 67 +++++++++ .../ConfluentSchemaRegistryCoderTest.java | 80 +++++++++++ 4 files changed, 377 insertions(+) create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/pom.xml create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml new file mode 100644 index 000000000..b421ebf66 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -0,0 +1,94 @@ + + + + + flink-formats + org.apache.flink + 1.6-SNAPSHOT + + 4.0.0 + + flink-avro-confluent-registry + + + + confluent + http://packages.confluent.io/maven/ + + + + + + io.confluent + kafka-schema-registry-client + 3.3.1 + + + org.apache.avro + avro + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.flink + flink-core + ${project.version} + provided + + + org.apache.flink + flink-avro + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + com.fasterxml.jackson.core + org.apache.flink.formats.avro.registry.confluent.shaded.com.fasterxml.jackson.core + + + + + + + + + + diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java new file mode 100644 index 000000000..1135bb998 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent; + +import org.apache.flink.formats.avro.AvroDeserializationSchema; +import org.apache.flink.formats.avro.RegistryAvroDeserializationSchema; +import org.apache.flink.formats.avro.SchemaCoder; + +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.specific.SpecificRecord; + +import javax.annotation.Nullable; + +/** + * Deserialization schema that deserializes from Avro binary format using {@link SchemaCoder} that uses + * Confluent Schema Registry. + * + * @param type of record it produces + */ +public class ConfluentRegistryAvroDeserializationSchema extends RegistryAvroDeserializationSchema { + + private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; + + private static final long serialVersionUID = -1671641202177852775L; + + /** + * Creates a Avro deserialization schema. + * + * @param recordClazz class to which deserialize. Should be either + * {@link SpecificRecord} or {@link GenericRecord}. + * @param reader reader's Avro schema. Should be provided if recordClazz is + * {@link GenericRecord} + * @param schemaCoderProvider provider for schema coder that reads writer schema from Confluent Schema Registry + */ + private ConfluentRegistryAvroDeserializationSchema(Class recordClazz, @Nullable Schema reader, + SchemaCoder.SchemaCoderProvider schemaCoderProvider) { + super(recordClazz, reader, schemaCoderProvider); + } + + /** + * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link GenericRecord} + * using provided reader schema and looks up writer schema in Confluent Schema Registry. + * + * @param schema schema of produced records + * @param url url of schema registry to connect + * @return deserialized record in form of {@link GenericRecord} + */ + public static ConfluentRegistryAvroDeserializationSchema forGeneric(Schema schema, String url) { + return forGeneric(schema, url, DEFAULT_IDENTITY_MAP_CAPACITY); + } + + /** + * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link GenericRecord} + * using provided reader schema and looks up writer schema in Confluent Schema Registry. + * + * @param schema schema of produced records + * @param url url of schema registry to connect + * @param identityMapCapacity maximum number of cached schema versions (default: 1000) + * @return deserialized record in form of {@link GenericRecord} + */ + public static ConfluentRegistryAvroDeserializationSchema forGeneric(Schema schema, String url, + int identityMapCapacity) { + return new ConfluentRegistryAvroDeserializationSchema<>( + GenericRecord.class, + schema, + new CachedSchemaCoderProvider(url, identityMapCapacity)); + } + + /** + * Creates {@link AvroDeserializationSchema} that produces classes that were generated from avro + * schema and looks up writer schema in Confluent Schema Registry. + * + * @param tClass class of record to be produced + * @param url url of schema registry to connect + * @return deserialized record + */ + public static ConfluentRegistryAvroDeserializationSchema forSpecific(Class tClass, + String url) { + return forSpecific(tClass, url, DEFAULT_IDENTITY_MAP_CAPACITY); + } + + /** + * Creates {@link AvroDeserializationSchema} that produces classes that were generated from avro + * schema and looks up writer schema in Confluent Schema Registry. + * + * @param tClass class of record to be produced + * @param url url of schema registry to connect + * @param identityMapCapacity maximum number of cached schema versions (default: 1000) + * @return deserialized record + */ + public static ConfluentRegistryAvroDeserializationSchema forSpecific(Class tClass, + String url, int identityMapCapacity) { + return new ConfluentRegistryAvroDeserializationSchema<>( + tClass, + null, + new CachedSchemaCoderProvider(url, identityMapCapacity) + ); + } + + private static class CachedSchemaCoderProvider implements SchemaCoder.SchemaCoderProvider { + + private static final long serialVersionUID = 4023134423033312666L; + private final String url; + private final int identityMapCapacity; + + CachedSchemaCoderProvider(String url, int identityMapCapacity) { + this.url = url; + this.identityMapCapacity = identityMapCapacity; + } + + @Override + public SchemaCoder get() { + return new ConfluentSchemaRegistryCoder(new CachedSchemaRegistryClient( + url, + identityMapCapacity)); + } + } +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java new file mode 100644 index 000000000..1f2dc69ca --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent; + +import org.apache.flink.formats.avro.SchemaCoder; + +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import org.apache.avro.Schema; + +import java.io.DataInputStream; +import java.io.IOException; +import java.io.InputStream; + +import static java.lang.String.format; + +/** + * Reads schema using Confluent Schema Registry protocol. + */ +public class ConfluentSchemaRegistryCoder implements SchemaCoder { + + private final SchemaRegistryClient schemaRegistryClient; + + /** + * Creates {@link SchemaCoder} that uses provided {@link SchemaRegistryClient} to connect to + * schema registry. + * + * @param schemaRegistryClient client to connect schema registry + */ + public ConfluentSchemaRegistryCoder(SchemaRegistryClient schemaRegistryClient) { + this.schemaRegistryClient = schemaRegistryClient; + } + + @Override + public Schema readSchema(InputStream in) throws IOException { + DataInputStream dataInputStream = new DataInputStream(in); + + if (dataInputStream.readByte() != 0) { + throw new IOException("Unknown data format. Magic number does not match"); + } else { + int schemaId = dataInputStream.readInt(); + + try { + return schemaRegistryClient.getById(schemaId); + } catch (RestClientException e) { + throw new IOException(format("Could not find schema with id %s in registry", schemaId), e); + } + } + } + +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java new file mode 100644 index 000000000..01e807c7d --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent; + +import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link ConfluentSchemaRegistryCoder}. + */ +public class ConfluentSchemaRegistryCoderTest { + + @Test + public void testSpecificRecordWithConfluentSchemaRegistry() throws Exception { + MockSchemaRegistryClient client = new MockSchemaRegistryClient(); + + Schema schema = SchemaBuilder.record("testRecord") + .fields() + .optionalString("testField") + .endRecord(); + int schemaId = client.register("testTopic", schema); + + ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(client); + ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream(); + DataOutputStream dataOutputStream = new DataOutputStream(byteOutStream); + dataOutputStream.writeByte(0); + dataOutputStream.writeInt(schemaId); + dataOutputStream.flush(); + + ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray()); + Schema readSchema = registryCoder.readSchema(byteInStream); + + assertEquals(schema, readSchema); + assertEquals(0, byteInStream.available()); + } + + @Test(expected = IOException.class) + public void testMagicByteVerification() throws Exception { + MockSchemaRegistryClient client = new MockSchemaRegistryClient(); + int schemaId = client.register("testTopic", Schema.create(Schema.Type.BOOLEAN)); + + ConfluentSchemaRegistryCoder coder = new ConfluentSchemaRegistryCoder(client); + ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream(); + DataOutputStream dataOutputStream = new DataOutputStream(byteOutStream); + dataOutputStream.writeByte(5); + dataOutputStream.writeInt(schemaId); + dataOutputStream.flush(); + + ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray()); + coder.readSchema(byteInStream); + + // exception is thrown + } + +} From 8498a9bd9d9eb8c3a12947008758d8d21667a3dc Mon Sep 17 00:00:00 2001 From: kai-chi Date: Thu, 12 Apr 2018 16:08:26 +0200 Subject: [PATCH 021/322] [FLINK-7850][build] Add activation property to maven profiles This closes #5840. --- flink-formats-kafka/flink-json-debezium/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 2c8db2da7..defa00450 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -91,6 +91,11 @@ under the License. release + + + release + + From da1617ef9f1196c635be31d0a382b847f798ddd3 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 29 May 2018 10:15:56 +0200 Subject: [PATCH 022/322] [FLINK-9464] Remove version and scope from flink-test-utils-junit in flink-json pom.xml --- flink-formats-kafka/flink-json-debezium/pom.xml | 2 -- 1 file changed, 2 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index defa00450..4e3176694 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -66,8 +66,6 @@ under the License. org.apache.flink flink-test-utils-junit - ${project.version} - test From d0e134414ab028694438686605a8c10fb772d121 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 12 Jun 2018 14:51:20 +0200 Subject: [PATCH 023/322] [FLINK-9557] [formats] Parse 'integer' type as BigDecimal This closes #6153. --- .../org/apache/flink/formats/json/JsonSchemaConverter.java | 4 +++- .../formats/json/JsonRowDeserializationSchemaTest.java | 7 +++---- .../flink/formats/json/JsonRowSerializationSchemaTest.java | 7 +++---- .../apache/flink/formats/json/JsonSchemaConverterTest.java | 2 +- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java index 5b39f17a5..7a001f637 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java @@ -165,7 +165,9 @@ else if (typeNode.isTextual()) { typeSet.add(Types.BIG_DEC); break; case TYPE_INTEGER: - typeSet.add(Types.BIG_INT); + // use BigDecimal for easier interoperability + // without affecting the correctness of the result + typeSet.add(Types.BIG_DEC); break; case TYPE_OBJECT: typeSet.add(convertObject(location, node, root)); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index 1c4a17a8a..5e77b801a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -29,7 +29,6 @@ import java.io.IOException; import java.math.BigDecimal; -import java.math.BigInteger; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; @@ -79,12 +78,12 @@ public void testTypeInfoDeserialization() throws Exception { @Test public void testSchemaDeserialization() throws Exception { - final BigInteger id = BigInteger.valueOf(1238123899121L); + final BigDecimal id = BigDecimal.valueOf(1238123899121L); final String name = "asdlkjasjkdla998y1122"; final byte[] bytes = new byte[1024]; ThreadLocalRandom.current().nextBytes(bytes); - final BigInteger[] numbers = new BigInteger[] { - BigInteger.valueOf(1), BigInteger.valueOf(2), BigInteger.valueOf(3)}; + final BigDecimal[] numbers = new BigDecimal[] { + BigDecimal.valueOf(1), BigDecimal.valueOf(2), BigDecimal.valueOf(3)}; final String[] strings = new String[] {"one", "two", "three"}; final ObjectMapper objectMapper = new ObjectMapper(); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java index f956fc640..94a05b3d7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java @@ -25,7 +25,6 @@ import java.io.IOException; import java.math.BigDecimal; -import java.math.BigInteger; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; @@ -137,7 +136,7 @@ public void testSchema() throws IOException { "}"); final Row row = new Row(11); - row.setField(0, BigInteger.valueOf(-333)); + row.setField(0, BigDecimal.valueOf(-333)); row.setField(1, BigDecimal.valueOf(12.2222)); row.setField(2, null); row.setField(3, ""); @@ -148,8 +147,8 @@ public void testSchema() throws IOException { final byte[] bytes = new byte[1024]; ThreadLocalRandom.current().nextBytes(bytes); row.setField(7, bytes); - final BigInteger[] numbers = new BigInteger[] { - BigInteger.valueOf(1), BigInteger.valueOf(2), BigInteger.valueOf(3)}; + final BigDecimal[] numbers = new BigDecimal[] { + BigDecimal.valueOf(1), BigDecimal.valueOf(2), BigDecimal.valueOf(3)}; row.setField(8, numbers); final String[] strings = new String[] {"one", "two", "three"}; row.setField(9, strings); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java index d0726639a..7cf3b7c1e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java @@ -47,7 +47,7 @@ public void testComplexSchema() throws Exception { "email", "tel", "sound", "org"}, Types.STRING, Types.STRING, Types.BOOLEAN, Types.ROW(Types.BIG_DEC, Types.STRING, Types.STRING, Types.STRING), Types.OBJECT_ARRAY(Types.STRING), Types.STRING, Types.ROW_NAMED(new String[] {"type", "value"}, Types.STRING, Types.STRING), - Types.ROW_NAMED(new String[] {"type", "value"}, Types.BIG_INT, Types.STRING), Types.VOID, + Types.ROW_NAMED(new String[] {"type", "value"}, Types.BIG_DEC, Types.STRING), Types.VOID, Types.ROW_NAMED(new String[] {"organizationUnit"}, Types.ROW())); assertEquals(expected, result); From 7995bce848d4583f31a98710e2bae7f3259e50c8 Mon Sep 17 00:00:00 2001 From: "Yadan.JS" Date: Tue, 12 Jun 2018 10:03:58 -0400 Subject: [PATCH 024/322] [FLINK-8983] End-to-end test: Confluent schema registry This closes #6083. --- .../pom.xml | 155 ++++++++++++++++++ .../src/main/avro/user.avsc | 27 +++ .../test/TestAvroConsumerConfluent.java | 86 ++++++++++ 3 files changed, 268 insertions(+) create mode 100644 flink-confluent-schema-registry-e2e-tests/pom.xml create mode 100644 flink-confluent-schema-registry-e2e-tests/src/main/avro/user.avsc create mode 100644 flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml new file mode 100644 index 000000000..576fca6d9 --- /dev/null +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -0,0 +1,155 @@ + + + + flink-end-to-end-tests + org.apache.flink + 1.6-SNAPSHOT + + 4.0.0 + + flink-confluent-schema-registry + + UTF-8 + 4.1.0 + + + + + confluent + http://packages.confluent.io/maven/ + + + + + + + org.apache.flink + flink-core + ${project.version} + + + + org.apache.flink + flink-clients_${scala.binary.version} + ${project.version} + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + + + + org.apache.flink + flink-connector-kafka-0.10_${scala.binary.version} + ${project.version} + + + org.apache.flink + flink-avro + ${project.version} + + + org.apache.flink + flink-avro-confluent-registry + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + package + + shade + + + TestAvroConsumerConfluent + + + com.google.code.findbugs:jsr305 + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + org.apache.flink.schema.registry.test.TestAvroConsumerConfluent + + + + + + + + org.apache.avro + avro-maven-plugin + ${avro.version} + + + generate-sources + + schema + + + ${project.basedir}/src/main/avro/ + ${project.basedir}/src/main/java/ + PRIVATE + + **/*.avsc + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + + + diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/avro/user.avsc b/flink-confluent-schema-registry-e2e-tests/src/main/avro/user.avsc new file mode 100644 index 000000000..aca9b83f9 --- /dev/null +++ b/flink-confluent-schema-registry-e2e-tests/src/main/avro/user.avsc @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + {"namespace": "example.avro", + "type": "record", + "name": "User", + "fields": [ + {"name": "name", "type": "string", "default": ""}, + {"name": "favoriteNumber", "type": "string", "default": ""}, + {"name": "favoriteColor", "type": "string", "default": ""}, + {"name": "eventType","type": {"name": "EventType","type": "enum", "symbols": ["meeting"] }} + ] +} diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java new file mode 100644 index 000000000..9149832a3 --- /dev/null +++ b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.schema.registry.test; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; + +import example.avro.User; + +import java.util.Properties; + +/** + * A simple example that shows how to read from and write to Kafka with Confluent Schema Registry. + * This will read AVRO messages from the input topic, parse them into a POJO type via checking the Schema by calling Schema registry. + * Then this example publish the POJO type to kafka by converting the POJO to AVRO and verifying the schema. + * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --schema-registry-url http://localhost:8081 --group.id myconsumer + */ +public class TestAvroConsumerConfluent { + + public static void main(String[] args) throws Exception { + Properties config = new Properties(); + // parse input arguments + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + + if (parameterTool.getNumberOfParameters() < 6) { + System.out.println("Missing parameters!\n" + + "Usage: Kafka --input-topic --output-topic " + + "--bootstrap.servers " + + "--zookeeper.connect " + + "--schema-registry-url --group.id "); + return; + } + config.setProperty("bootstrap.servers", parameterTool.getRequired("bootstrap.servers")); + config.setProperty("group.id", parameterTool.getRequired("group.id")); + config.setProperty("zookeeper.connect", parameterTool.getRequired("zookeeper.connect")); + String schemaRegistryUrl = parameterTool.getRequired("schema-registry-url"); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().disableSysoutLogging(); + + DataStreamSource input = env + .addSource( + new FlinkKafkaConsumer010( + parameterTool.getRequired("input-topic"), + ConfluentRegistryAvroDeserializationSchema.forSpecific(User.class, schemaRegistryUrl), + config).setStartFromEarliest()); + + SingleOutputStreamOperator mapToString = input + .map(new MapFunction() { + @Override + public String map(User value) throws Exception { + return value.toString(); + } + }); + + FlinkKafkaProducer010 stringFlinkKafkaProducer010 = new FlinkKafkaProducer010( + parameterTool.getRequired("output-topic"), + new SimpleStringSchema(), + config); + + mapToString.addSink(stringFlinkKafkaProducer010); + env.execute("Kafka 0.10 Confluent Schema Registry AVRO Example"); + } +} From d1b4119d87d71d5bff807e8d40cbaadeb70c16eb Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 28 Jun 2018 18:34:24 +0200 Subject: [PATCH 025/322] [FLINK-8983] Integrate test_confluent_schema_registry.sh into run-nightly-tests.sh --- .../.gitignore | 1 + flink-confluent-schema-registry-e2e-tests/pom.xml | 7 +++++++ .../registry/test/TestAvroConsumerConfluent.java | 14 +++++--------- 3 files changed, 13 insertions(+), 9 deletions(-) create mode 100644 flink-confluent-schema-registry-e2e-tests/.gitignore diff --git a/flink-confluent-schema-registry-e2e-tests/.gitignore b/flink-confluent-schema-registry-e2e-tests/.gitignore new file mode 100644 index 000000000..b83a75282 --- /dev/null +++ b/flink-confluent-schema-registry-e2e-tests/.gitignore @@ -0,0 +1 @@ +src/main/java/example/avro diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 576fca6d9..2f63c22b3 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -150,6 +150,13 @@ under the License. + + org.apache.maven.plugins + maven-checkstyle-plugin + + **/example/avro/* + + diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java index 9149832a3..fbbe3c353 100644 --- a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java +++ b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java @@ -28,6 +28,7 @@ import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; import example.avro.User; +import org.apache.avro.specific.SpecificRecordBase; import java.util.Properties; @@ -40,7 +41,6 @@ public class TestAvroConsumerConfluent { public static void main(String[] args) throws Exception { - Properties config = new Properties(); // parse input arguments final ParameterTool parameterTool = ParameterTool.fromArgs(args); @@ -52,6 +52,7 @@ public static void main(String[] args) throws Exception { "--schema-registry-url --group.id "); return; } + Properties config = new Properties(); config.setProperty("bootstrap.servers", parameterTool.getRequired("bootstrap.servers")); config.setProperty("group.id", parameterTool.getRequired("group.id")); config.setProperty("zookeeper.connect", parameterTool.getRequired("zookeeper.connect")); @@ -62,20 +63,15 @@ public static void main(String[] args) throws Exception { DataStreamSource input = env .addSource( - new FlinkKafkaConsumer010( + new FlinkKafkaConsumer010<>( parameterTool.getRequired("input-topic"), ConfluentRegistryAvroDeserializationSchema.forSpecific(User.class, schemaRegistryUrl), config).setStartFromEarliest()); SingleOutputStreamOperator mapToString = input - .map(new MapFunction() { - @Override - public String map(User value) throws Exception { - return value.toString(); - } - }); + .map((MapFunction) SpecificRecordBase::toString); - FlinkKafkaProducer010 stringFlinkKafkaProducer010 = new FlinkKafkaProducer010( + FlinkKafkaProducer010 stringFlinkKafkaProducer010 = new FlinkKafkaProducer010<>( parameterTool.getRequired("output-topic"), new SimpleStringSchema(), config); From 986b3b2dbae5027429e4de8ed67c8f95acc43414 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 26 Jun 2018 11:46:06 +0200 Subject: [PATCH 026/322] [FLINK-9444] [formats] Add full SQL support for Avro formats This PR adds full support of Apache Avro records for the Table API & SQL. It adds (de)serialization schemas to the row type for both specific and generic records. It converts all Avro types to Flink types and vice versa. It supports both physical and logical Avro types. Both an Avro class or a Avro schema string can be used for format initialization. This closes #6218. This closes #6082. --- .../apache/flink/formats/json/JsonRowDeserializationSchema.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index edc4b0115..df5285190 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -87,7 +87,6 @@ public JsonRowDeserializationSchema(String jsonSchema) { this(JsonSchemaConverter.convert(jsonSchema)); } - @SuppressWarnings("unchecked") @Override public Row deserialize(byte[] message) throws IOException { try { From d9ba7228fe537f4d137403708270f8843ae13942 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 27 Jun 2018 13:16:49 +0200 Subject: [PATCH 027/322] [FLINK-8558] [table] Add unified format interfaces and separate formats from connectors This PR introduces a format discovery mechanism based on Java Service Providers. The general `TableFormatFactory` is similar to the existing table source discovery mechanism. However, it allows for arbirary format interfaces that might be introduced in the future. At the moment, a connector can request configured instances of `DeserializationSchema` and `SerializationSchema`. In the future we can add interfaces such as a `Writer` or `KeyedSerializationSchema` without breaking backwards compatibility. This PR deprecates the existing strong coupling of connector and format for the Kafa table sources and table source factories. It introduces descriptor-based alternatives. --- .../json/JsonRowDeserializationSchema.java | 20 ++- .../formats/json/JsonRowFormatFactory.java | 108 +++++++++++++ ...erter.java => JsonRowSchemaConverter.java} | 4 +- .../json/JsonRowSerializationSchema.java | 20 ++- .../apache/flink/table/descriptors/Json.java | 3 +- .../table/descriptors/JsonValidator.java | 2 +- ...che.flink.table.formats.TableFormatFactory | 16 ++ .../json/JsonRowFormatFactoryTest.java | 150 ++++++++++++++++++ ...t.java => JsonRowSchemaConverterTest.java} | 22 +-- .../json/JsonRowSerializationSchemaTest.java | 2 +- 10 files changed, 329 insertions(+), 18 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java rename flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/{JsonSchemaConverter.java => JsonRowSchemaConverter.java} (99%) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java rename flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/{JsonSchemaConverterTest.java => JsonRowSchemaConverterTest.java} (80%) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index df5285190..dc8a116ac 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -38,6 +38,7 @@ import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +import java.util.Objects; /** * Deserialization schema from JSON to Flink types. @@ -84,7 +85,7 @@ public JsonRowDeserializationSchema(TypeInformation typeInfo) { * @see http://json-schema.org/ */ public JsonRowDeserializationSchema(String jsonSchema) { - this(JsonSchemaConverter.convert(jsonSchema)); + this(JsonRowSchemaConverter.convert(jsonSchema)); } @Override @@ -118,6 +119,23 @@ public void setFailOnMissingField(boolean failOnMissingField) { this.failOnMissingField = failOnMissingField; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final JsonRowDeserializationSchema that = (JsonRowDeserializationSchema) o; + return failOnMissingField == that.failOnMissingField && Objects.equals(typeInfo, that.typeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(typeInfo, failOnMissingField); + } + // -------------------------------------------------------------------------------------------- private Object convert(JsonNode node, TypeInformation info) { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java new file mode 100644 index 000000000..fd7bda6c7 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.FormatDescriptorValidator; +import org.apache.flink.table.descriptors.JsonValidator; +import org.apache.flink.table.descriptors.SchemaValidator; +import org.apache.flink.table.formats.DeserializationSchemaFactory; +import org.apache.flink.table.formats.SerializationSchemaFactory; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Table format factory for providing configured instances of JSON-to-row {@link SerializationSchema} + * and {@link DeserializationSchema}. + */ +public class JsonRowFormatFactory implements SerializationSchemaFactory, DeserializationSchemaFactory { + + @Override + public Map requiredContext() { + final Map context = new HashMap<>(); + context.put(FormatDescriptorValidator.FORMAT_TYPE(), JsonValidator.FORMAT_TYPE_VALUE); + context.put(FormatDescriptorValidator.FORMAT_PROPERTY_VERSION(), "1"); + return context; + } + + @Override + public boolean supportsSchemaDerivation() { + return true; + } + + @Override + public List supportedProperties() { + final List properties = new ArrayList<>(); + properties.add(JsonValidator.FORMAT_JSON_SCHEMA); + properties.add(JsonValidator.FORMAT_SCHEMA); + properties.add(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD); + properties.add(FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA()); + properties.addAll(SchemaValidator.getSchemaDerivationKeys()); + return properties; + } + + @Override + public DeserializationSchema createDeserializationSchema(Map properties) { + final DescriptorProperties descriptorProperties = validateAndGetProperties(properties); + + // create and configure + final JsonRowDeserializationSchema schema = new JsonRowDeserializationSchema(createTypeInformation(descriptorProperties)); + + descriptorProperties.getOptionalBoolean(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD) + .ifPresent(schema::setFailOnMissingField); + + return schema; + } + + @Override + public SerializationSchema createSerializationSchema(Map properties) { + final DescriptorProperties descriptorProperties = validateAndGetProperties(properties); + + // create and configure + return new JsonRowSerializationSchema(createTypeInformation(descriptorProperties)); + } + + private static DescriptorProperties validateAndGetProperties(Map propertiesMap) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + descriptorProperties.putProperties(propertiesMap); + + // validate + new JsonValidator().validate(descriptorProperties); + + return descriptorProperties; + } + + private static TypeInformation createTypeInformation(DescriptorProperties descriptorProperties) { + if (descriptorProperties.containsKey(JsonValidator.FORMAT_SCHEMA)) { + return (RowTypeInfo) descriptorProperties.getType(JsonValidator.FORMAT_SCHEMA); + } else if (descriptorProperties.containsKey(JsonValidator.FORMAT_JSON_SCHEMA)) { + return JsonRowSchemaConverter.convert(descriptorProperties.getString(JsonValidator.FORMAT_JSON_SCHEMA)); + } else { + return SchemaValidator.deriveFormatFields(descriptorProperties).toRowType(); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java similarity index 99% rename from flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java rename to flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java index 7a001f637..320ca1f8c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java @@ -49,9 +49,9 @@ *

This converter has been developed for JSON Schema draft-07 but also includes keywords of * older drafts to be as compatible as possible. */ -public final class JsonSchemaConverter { +public final class JsonRowSchemaConverter { - private JsonSchemaConverter() { + private JsonRowSchemaConverter() { // private } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index 8fee6a4d9..d9420629f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -39,6 +39,7 @@ import java.sql.Time; import java.sql.Timestamp; import java.text.SimpleDateFormat; +import java.util.Objects; /** * Serialization schema that serializes an object of Flink types into a JSON bytes. @@ -89,7 +90,7 @@ public JsonRowSerializationSchema(TypeInformation typeInfo) { * @see http://json-schema.org/ */ public JsonRowSerializationSchema(String jsonSchema) { - this(JsonSchemaConverter.convert(jsonSchema)); + this(JsonRowSchemaConverter.convert(jsonSchema)); } @Override @@ -107,6 +108,23 @@ public byte[] serialize(Row row) { } } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final JsonRowSerializationSchema that = (JsonRowSerializationSchema) o; + return Objects.equals(typeInfo, that.typeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(typeInfo); + } + // -------------------------------------------------------------------------------------------- private ObjectNode convertRow(ObjectNode reuse, RowTypeInfo info, Row row) { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java index 9c121916d..035f05f7c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.typeutils.TypeStringUtils; +import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA; @@ -81,7 +82,7 @@ public Json jsonSchema(String jsonSchema) { * * @param schemaType type information that describes the schema */ - public Json schema(TypeInformation schemaType) { + public Json schema(TypeInformation schemaType) { Preconditions.checkNotNull(schemaType); this.schema = TypeStringUtils.writeTypeInfo(schemaType); this.jsonSchema = null; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java index fea7cf55b..49e1abc8a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java @@ -45,7 +45,7 @@ public void validate(DescriptorProperties properties) { } else if (!deriveSchema && !hasSchema && !hasSchemaString) { throw new ValidationException("A definition of a schema or JSON schema is required."); } else if (hasSchema) { - properties.validateType(FORMAT_SCHEMA, false); + properties.validateType(FORMAT_SCHEMA, true, false); } else if (hasSchemaString) { properties.validateString(FORMAT_JSON_SCHEMA, false, 1); } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory new file mode 100644 index 000000000..aec584688 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.formats.json.JsonRowFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java new file mode 100644 index 000000000..d763b9047 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.Types; +import org.apache.flink.table.descriptors.Descriptor; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.Json; +import org.apache.flink.table.descriptors.Schema; +import org.apache.flink.table.formats.DeserializationSchemaFactory; +import org.apache.flink.table.formats.SerializationSchemaFactory; +import org.apache.flink.table.formats.TableFormatFactoryService; +import org.apache.flink.types.Row; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for the {@link JsonRowFormatFactory}. + */ +public class JsonRowFormatFactoryTest extends TestLogger { + + private static final String JSON_SCHEMA = + "{" + + " 'title': 'Fruit'," + + " 'type': 'object'," + + " 'properties': {" + + " 'name': {" + + " 'type': 'string'" + + " }," + + " 'count': {" + + " 'type': 'integer'" + + " }," + + " 'time': {" + + " 'description': 'row time'," + + " 'type': 'string'," + + " 'format': 'date-time'" + + " }" + + " }," + + " 'required': ['name', 'count', 'time']" + + "}"; + + private static final TypeInformation SCHEMA = Types.ROW( + new String[]{"field1", "field2"}, + new TypeInformation[]{Types.BOOLEAN(), Types.INT()}); + + @Test + public void testSchema() { + final Map properties = toMap( + new Json() + .schema(SCHEMA) + .failOnMissingField(false)); + + testSchemaSerializationSchema(properties); + + testSchemaDeserializationSchema(properties); + } + + @Test + public void testJsonSchema() { + final Map properties = toMap( + new Json() + .jsonSchema(JSON_SCHEMA) + .failOnMissingField(true)); + + testJsonSchemaSerializationSchema(properties); + + testJsonSchemaDeserializationSchema(properties); + } + + @Test + public void testSchemaDerivation() { + final Map properties = toMap( + new Schema() + .field("field1", Types.BOOLEAN()) + .field("field2", Types.INT()) + .field("proctime", Types.SQL_TIMESTAMP()).proctime(), + new Json() + .deriveSchema()); + + testSchemaSerializationSchema(properties); + + testSchemaDeserializationSchema(properties); + } + + private void testSchemaDeserializationSchema(Map properties) { + final DeserializationSchema actual2 = TableFormatFactoryService + .find(DeserializationSchemaFactory.class, properties) + .createDeserializationSchema(properties); + final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema(SCHEMA); + expected2.setFailOnMissingField(false); + assertEquals(expected2, actual2); + } + + private void testSchemaSerializationSchema(Map properties) { + final SerializationSchema actual1 = TableFormatFactoryService + .find(SerializationSchemaFactory.class, properties) + .createSerializationSchema(properties); + final SerializationSchema expected1 = new JsonRowSerializationSchema(SCHEMA); + assertEquals(expected1, actual1); + } + + private void testJsonSchemaDeserializationSchema(Map properties) { + final DeserializationSchema actual2 = TableFormatFactoryService + .find(DeserializationSchemaFactory.class, properties) + .createDeserializationSchema(properties); + final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema(JSON_SCHEMA); + expected2.setFailOnMissingField(true); + assertEquals(expected2, actual2); + } + + private void testJsonSchemaSerializationSchema(Map properties) { + final SerializationSchema actual1 = TableFormatFactoryService + .find(SerializationSchemaFactory.class, properties) + .createSerializationSchema(properties); + final SerializationSchema expected1 = new JsonRowSerializationSchema(JSON_SCHEMA); + assertEquals(expected1, actual1); + } + + private static Map toMap(Descriptor... desc) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + for (Descriptor d : desc) { + d.addProperties(descriptorProperties); + } + return descriptorProperties.asMap(); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java similarity index 80% rename from flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java rename to flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java index 7cf3b7c1e..1af45f4e8 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java @@ -31,16 +31,16 @@ import static org.junit.Assert.assertEquals; /** - * Tests for {@link JsonSchemaConverter}. + * Tests for {@link JsonRowSchemaConverter}. */ -public class JsonSchemaConverterTest { +public class JsonRowSchemaConverterTest { @Test public void testComplexSchema() throws Exception { final URL url = getClass().getClassLoader().getResource("complex-schema.json"); Objects.requireNonNull(url); final String schema = FileUtils.readFileUtf8(new File(url.getFile())); - final TypeInformation result = JsonSchemaConverter.convert(schema); + final TypeInformation result = JsonRowSchemaConverter.convert(schema); final TypeInformation expected = Types.ROW_NAMED( new String[] {"fn", "familyName", "additionalName", "tuples", "honorificPrefix", "url", @@ -58,7 +58,7 @@ public void testReferenceSchema() throws Exception { final URL url = getClass().getClassLoader().getResource("reference-schema.json"); Objects.requireNonNull(url); final String schema = FileUtils.readFileUtf8(new File(url.getFile())); - final TypeInformation result = JsonSchemaConverter.convert(schema); + final TypeInformation result = JsonRowSchemaConverter.convert(schema); final TypeInformation expected = Types.ROW_NAMED( new String[] {"billing_address", "shipping_address", "optional_address"}, @@ -71,43 +71,43 @@ public void testReferenceSchema() throws Exception { @Test public void testAtomicType() { - final TypeInformation result = JsonSchemaConverter.convert("{ type: 'number' }"); + final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'number' }"); assertEquals(Types.BIG_DEC, result); } @Test(expected = IllegalArgumentException.class) public void testMissingType() { - JsonSchemaConverter.convert("{ }"); + JsonRowSchemaConverter.convert("{ }"); } @Test(expected = IllegalArgumentException.class) public void testWrongType() { - JsonSchemaConverter.convert("{ type: 'whatever' }"); + JsonRowSchemaConverter.convert("{ type: 'whatever' }"); } @Test(expected = IllegalArgumentException.class) public void testArrayWithAdditionalItems() { - JsonSchemaConverter.convert("{ type: 'array', items: [{type: 'integer'}], additionalItems: true }"); + JsonRowSchemaConverter.convert("{ type: 'array', items: [{type: 'integer'}], additionalItems: true }"); } @Test public void testMissingProperties() { - final TypeInformation result = JsonSchemaConverter.convert("{ type: 'object' }"); + final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'object' }"); assertEquals(Types.ROW(), result); } @Test public void testNullUnionTypes() { - final TypeInformation result = JsonSchemaConverter.convert("{ type: ['string', 'null'] }"); + final TypeInformation result = JsonRowSchemaConverter.convert("{ type: ['string', 'null'] }"); assertEquals(Types.STRING, result); } @Test public void testTimestamp() { - final TypeInformation result = JsonSchemaConverter.convert("{ type: 'string', format: 'date-time' }"); + final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'string', format: 'date-time' }"); assertEquals(Types.SQL_TIMESTAMP, result); } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java index 94a05b3d7..e2410d4a7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java @@ -111,7 +111,7 @@ public void testSerializeRowWithInvalidNumberOfFields() { @Test public void testSchema() throws IOException { - final TypeInformation rowSchema = JsonSchemaConverter.convert( + final TypeInformation rowSchema = JsonRowSchemaConverter.convert( "{" + " type: 'object'," + " properties: {" + From f96fdd0cebdd65d1a13e6fc164321d40884748b8 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 11 Jul 2018 13:29:03 +0200 Subject: [PATCH 028/322] [FLINK-8866] [table] Merge table source/sink/format factories - Rename to TableFactory and move it to org.apache.flink.table.factories package - Unify source/sink/format factories with same logic and exceptions --- .../flink/formats/json/JsonRowFormatFactory.java | 4 ++-- ...pache.flink.table.factories.TableFormatFactory} | 0 .../formats/json/JsonRowFormatFactoryTest.java | 14 +++++++------- 3 files changed, 9 insertions(+), 9 deletions(-) rename flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/{org.apache.flink.table.formats.TableFormatFactory => org.apache.flink.table.factories.TableFormatFactory} (100%) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java index fd7bda6c7..458b94a5d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -26,8 +26,8 @@ import org.apache.flink.table.descriptors.FormatDescriptorValidator; import org.apache.flink.table.descriptors.JsonValidator; import org.apache.flink.table.descriptors.SchemaValidator; -import org.apache.flink.table.formats.DeserializationSchemaFactory; -import org.apache.flink.table.formats.SerializationSchemaFactory; +import org.apache.flink.table.factories.DeserializationSchemaFactory; +import org.apache.flink.table.factories.SerializationSchemaFactory; import org.apache.flink.types.Row; import java.util.ArrayList; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory similarity index 100% rename from flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory rename to flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java index d763b9047..f43234dde 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -26,9 +26,9 @@ import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.Json; import org.apache.flink.table.descriptors.Schema; -import org.apache.flink.table.formats.DeserializationSchemaFactory; -import org.apache.flink.table.formats.SerializationSchemaFactory; -import org.apache.flink.table.formats.TableFormatFactoryService; +import org.apache.flink.table.factories.DeserializationSchemaFactory; +import org.apache.flink.table.factories.SerializationSchemaFactory; +import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.types.Row; import org.apache.flink.util.TestLogger; @@ -107,7 +107,7 @@ public void testSchemaDerivation() { } private void testSchemaDeserializationSchema(Map properties) { - final DeserializationSchema actual2 = TableFormatFactoryService + final DeserializationSchema actual2 = TableFactoryService .find(DeserializationSchemaFactory.class, properties) .createDeserializationSchema(properties); final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema(SCHEMA); @@ -116,7 +116,7 @@ private void testSchemaDeserializationSchema(Map properties) { } private void testSchemaSerializationSchema(Map properties) { - final SerializationSchema actual1 = TableFormatFactoryService + final SerializationSchema actual1 = TableFactoryService .find(SerializationSchemaFactory.class, properties) .createSerializationSchema(properties); final SerializationSchema expected1 = new JsonRowSerializationSchema(SCHEMA); @@ -124,7 +124,7 @@ private void testSchemaSerializationSchema(Map properties) { } private void testJsonSchemaDeserializationSchema(Map properties) { - final DeserializationSchema actual2 = TableFormatFactoryService + final DeserializationSchema actual2 = TableFactoryService .find(DeserializationSchemaFactory.class, properties) .createDeserializationSchema(properties); final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema(JSON_SCHEMA); @@ -133,7 +133,7 @@ private void testJsonSchemaDeserializationSchema(Map properties) } private void testJsonSchemaSerializationSchema(Map properties) { - final SerializationSchema actual1 = TableFormatFactoryService + final SerializationSchema actual1 = TableFactoryService .find(SerializationSchemaFactory.class, properties) .createSerializationSchema(properties); final SerializationSchema expected1 = new JsonRowSerializationSchema(JSON_SCHEMA); From de8c985910267ea83c62a2ba8645ffb73de8a7a9 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 12 Jul 2018 10:50:09 +0200 Subject: [PATCH 029/322] [FLINK-8866] [table] Make source/sink factories environment-dependent Usually it is very uncommon to define both a batch and streaming source in the same factory. Separating by environment is a concept that can be find throughout the entire flink-table module because both sources and sinks behave quite different per environment. This closes #6323. --- ...ormatFactory => org.apache.flink.table.factories.TableFactory} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/{org.apache.flink.table.factories.TableFormatFactory => org.apache.flink.table.factories.TableFactory} (100%) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory rename to flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory From 4557d9498cdb76af9473fd2481ef5e8d930f4b2d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 16 Jul 2018 23:24:31 +0200 Subject: [PATCH 030/322] Update version to 1.7-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 2f63c22b3..cdc3f9593 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.6-SNAPSHOT + 1.7-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index b421ebf66..9d8185196 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.6-SNAPSHOT + 1.7-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 4e3176694..fbe0136e7 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.6-SNAPSHOT + 1.7-SNAPSHOT .. From 3e5ff956931c96df3f44c923eccb7dbf0f1cbb85 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 18 Jul 2018 13:30:28 +0200 Subject: [PATCH 031/322] [FLINK-9886] [sql-client] Build SQL jars with every build This closes #6366. --- flink-formats-kafka/flink-json-debezium/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index fbe0136e7..0cb3f8ddd 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -86,12 +86,12 @@ under the License. + - - release + sql-jars - release + !skipSqlJars From 37c2f210d20dec45b173cb09ded2444bf114f5d6 Mon Sep 17 00:00:00 2001 From: Chesnay Date: Thu, 26 Jul 2018 15:08:30 +0200 Subject: [PATCH 032/322] [FLINK-9944][tests] Cleanup end-to-end test poms --- .../pom.xml | 31 ++----------------- 1 file changed, 2 insertions(+), 29 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index cdc3f9593..ea81997f3 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -40,23 +40,11 @@ under the License. - - - org.apache.flink - flink-core - ${project.version} - - - - org.apache.flink - flink-clients_${scala.binary.version} - ${project.version} - org.apache.flink flink-streaming-java_${scala.binary.version} ${project.version} + provided @@ -81,30 +69,15 @@ under the License. org.apache.maven.plugins maven-shade-plugin - 3.0.0 + TestAvroConsumerConfluent package shade TestAvroConsumerConfluent - - - com.google.code.findbugs:jsr305 - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - org.apache.flink.schema.registry.test.TestAvroConsumerConfluent From 04f0424f6b3ff24f56815c3f921c3842fd9ac8cb Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 26 Jul 2018 15:42:53 +0200 Subject: [PATCH 033/322] [FLINK-9947] [docs] Document unified table sources/sinks/formats Adds documentation for unified table sources, sinks, and formats both for Table API & SQL and SQL Client. - New connect page - Adapted SQL Client page - Adapted Sources & Sinks page This closes #6456. --- .../main/java/org/apache/flink/table/descriptors/Json.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java index 035f05f7c..026f49e61 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -96,8 +96,8 @@ public Json schema(TypeInformation schemaType) { *

This allows for defining schema information only once. * *

The names, types, and field order of the format are determined by the table's - * schema. Time attributes are ignored. A "from" definition is interpreted as a field renaming - * in the format. + * schema. Time attributes are ignored if their origin is not a field. A "from" definition + * is interpreted as a field renaming in the format. */ public Json deriveSchema() { this.deriveSchema = true; From eb0349e4b0db718fe25078995ee5e22fcdff6adc Mon Sep 17 00:00:00 2001 From: jerryjzhang Date: Sun, 19 Aug 2018 22:27:01 +0800 Subject: [PATCH 034/322] [FLINK-10170] [table] Add string representation for all Table & SQL API types Since 1.6 the recommended way of creating source/sink tables is using connector/format/schema descriptors. This commit adds string-based representation for all types supported by the Table & SQL API. We use a syntax similar to Hive and other SQL projects. This closes #6578. --- .../test/java/org/apache/flink/table/descriptors/JsonTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java index 6e370a02c..ac6ff11c3 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java @@ -106,7 +106,7 @@ public List> properties() { final Map props3 = new HashMap<>(); props3.put("format.type", "json"); props3.put("format.property-version", "1"); - props3.put("format.schema", "ROW(test1 VARCHAR, test2 TIMESTAMP)"); + props3.put("format.schema", "ROW"); props3.put("format.fail-on-missing-field", "true"); final Map props4 = new HashMap<>(); From eed2ad5833b80fc49369e1f44bc3f480bcb47168 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 26 Oct 2018 15:57:03 +0200 Subject: [PATCH 035/322] [FLINK-10687] [table] Move TypeStringUtils to flink-table-common --- .../src/main/java/org/apache/flink/table/descriptors/Json.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java index 026f49e61..8e5b6f3ac 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -19,7 +19,7 @@ package org.apache.flink.table.descriptors; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.typeutils.TypeStringUtils; +import org.apache.flink.table.utils.TypeStringUtils; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; From 491d9cae0f3674998b710e8433602505d4982621 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 29 Oct 2018 09:18:39 +0100 Subject: [PATCH 036/322] [FLINK-10687] [table] Move DescriptorProperties to flink-table-common --- .../java/org/apache/flink/table/descriptors/JsonValidator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java index 49e1abc8a..af8721f03 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java @@ -45,7 +45,7 @@ public void validate(DescriptorProperties properties) { } else if (!deriveSchema && !hasSchema && !hasSchemaString) { throw new ValidationException("A definition of a schema or JSON schema is required."); } else if (hasSchema) { - properties.validateType(FORMAT_SCHEMA, true, false); + properties.validateType(FORMAT_SCHEMA, false, true); } else if (hasSchemaString) { properties.validateString(FORMAT_JSON_SCHEMA, false, 1); } From cf523e98f12c50a0ddb46d1a2e7c8349e1bd3828 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Sat, 27 Oct 2018 15:47:22 +0200 Subject: [PATCH 037/322] [FLINK-10687] [table] Move format descriptors and validators to flink-table-common This commit makes the flink-formats module Scala free by introducing a flink-table-common module that is implemented in Java. This module contains all classes that are required across different Maven modules. Additionally, all classes in this module have been annotated with @Internal and @PublicEvolving accordingly. Since the methods in Descriptor were declared with private[flink] visibility, they have been migrated to a new toProperties() method that is public. This closes #6958. --- .../flink-json-debezium/pom.xml | 18 ++++--- .../formats/json/JsonRowFormatFactory.java | 52 +++++++------------ .../apache/flink/table/descriptors/Json.java | 13 +++-- .../table/descriptors/JsonValidator.java | 6 ++- .../json/JsonRowFormatFactoryTest.java | 6 +-- 5 files changed, 47 insertions(+), 48 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 0cb3f8ddd..4b1baa976 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -53,12 +53,9 @@ under the License. org.apache.flink - - flink-table_2.11 + flink-table-common ${project.version} provided - - true @@ -68,16 +65,25 @@ under the License. flink-test-utils-junit + + + org.apache.flink + flink-table-common + ${project.version} + test + test-jar + + + org.apache.flink flink-table_2.11 ${project.version} - test-jar test - + org.scala-lang scala-compiler diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java index 458b94a5d..567bef3a8 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -23,15 +23,13 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.descriptors.FormatDescriptorValidator; import org.apache.flink.table.descriptors.JsonValidator; -import org.apache.flink.table.descriptors.SchemaValidator; import org.apache.flink.table.factories.DeserializationSchemaFactory; import org.apache.flink.table.factories.SerializationSchemaFactory; +import org.apache.flink.table.factories.TableFormatFactoryBase; import org.apache.flink.types.Row; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; @@ -39,35 +37,25 @@ * Table format factory for providing configured instances of JSON-to-row {@link SerializationSchema} * and {@link DeserializationSchema}. */ -public class JsonRowFormatFactory implements SerializationSchemaFactory, DeserializationSchemaFactory { +public class JsonRowFormatFactory extends TableFormatFactoryBase + implements SerializationSchemaFactory, DeserializationSchemaFactory { - @Override - public Map requiredContext() { - final Map context = new HashMap<>(); - context.put(FormatDescriptorValidator.FORMAT_TYPE(), JsonValidator.FORMAT_TYPE_VALUE); - context.put(FormatDescriptorValidator.FORMAT_PROPERTY_VERSION(), "1"); - return context; + public JsonRowFormatFactory() { + super(JsonValidator.FORMAT_TYPE_VALUE, 1, true); } @Override - public boolean supportsSchemaDerivation() { - return true; - } - - @Override - public List supportedProperties() { + protected List supportedFormatProperties() { final List properties = new ArrayList<>(); properties.add(JsonValidator.FORMAT_JSON_SCHEMA); properties.add(JsonValidator.FORMAT_SCHEMA); properties.add(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD); - properties.add(FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA()); - properties.addAll(SchemaValidator.getSchemaDerivationKeys()); return properties; } @Override public DeserializationSchema createDeserializationSchema(Map properties) { - final DescriptorProperties descriptorProperties = validateAndGetProperties(properties); + final DescriptorProperties descriptorProperties = getValidatedProperties(properties); // create and configure final JsonRowDeserializationSchema schema = new JsonRowDeserializationSchema(createTypeInformation(descriptorProperties)); @@ -80,29 +68,29 @@ public DeserializationSchema createDeserializationSchema(Map createSerializationSchema(Map properties) { - final DescriptorProperties descriptorProperties = validateAndGetProperties(properties); + final DescriptorProperties descriptorProperties = getValidatedProperties(properties); // create and configure return new JsonRowSerializationSchema(createTypeInformation(descriptorProperties)); } - private static DescriptorProperties validateAndGetProperties(Map propertiesMap) { - final DescriptorProperties descriptorProperties = new DescriptorProperties(true); - descriptorProperties.putProperties(propertiesMap); - - // validate - new JsonValidator().validate(descriptorProperties); - - return descriptorProperties; - } - - private static TypeInformation createTypeInformation(DescriptorProperties descriptorProperties) { + private TypeInformation createTypeInformation(DescriptorProperties descriptorProperties) { if (descriptorProperties.containsKey(JsonValidator.FORMAT_SCHEMA)) { return (RowTypeInfo) descriptorProperties.getType(JsonValidator.FORMAT_SCHEMA); } else if (descriptorProperties.containsKey(JsonValidator.FORMAT_JSON_SCHEMA)) { return JsonRowSchemaConverter.convert(descriptorProperties.getString(JsonValidator.FORMAT_JSON_SCHEMA)); } else { - return SchemaValidator.deriveFormatFields(descriptorProperties).toRowType(); + return deriveSchema(descriptorProperties.asMap()).toRowType(); } } + + private static DescriptorProperties getValidatedProperties(Map propertiesMap) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(); + descriptorProperties.putProperties(propertiesMap); + + // validate + new JsonValidator().validate(descriptorProperties); + + return descriptorProperties; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java index 8e5b6f3ac..3c4888c96 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -23,6 +23,8 @@ import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import java.util.Map; + import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_JSON_SCHEMA; @@ -106,13 +108,12 @@ public Json deriveSchema() { return this; } - /** - * Internal method for format properties conversion. - */ @Override - public void addFormatProperties(DescriptorProperties properties) { + protected Map toFormatProperties() { + final DescriptorProperties properties = new DescriptorProperties(); + if (deriveSchema != null) { - properties.putBoolean(FORMAT_DERIVE_SCHEMA(), deriveSchema); + properties.putBoolean(FORMAT_DERIVE_SCHEMA, deriveSchema); } if (jsonSchema != null) { @@ -126,5 +127,7 @@ public void addFormatProperties(DescriptorProperties properties) { if (failOnMissingField != null) { properties.putBoolean(FORMAT_FAIL_ON_MISSING_FIELD, failOnMissingField); } + + return properties.asMap(); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java index af8721f03..a520811d1 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java @@ -18,11 +18,13 @@ package org.apache.flink.table.descriptors; +import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.ValidationException; /** * Validator for {@link Json}. */ +@Internal public class JsonValidator extends FormatDescriptorValidator { public static final String FORMAT_TYPE_VALUE = "json"; @@ -33,8 +35,8 @@ public class JsonValidator extends FormatDescriptorValidator { @Override public void validate(DescriptorProperties properties) { super.validate(properties); - properties.validateBoolean(FORMAT_DERIVE_SCHEMA(), true); - final boolean deriveSchema = properties.getOptionalBoolean(FORMAT_DERIVE_SCHEMA()).orElse(false); + properties.validateBoolean(FORMAT_DERIVE_SCHEMA, true); + final boolean deriveSchema = properties.getOptionalBoolean(FORMAT_DERIVE_SCHEMA).orElse(false); final boolean hasSchema = properties.containsKey(FORMAT_SCHEMA); final boolean hasSchemaString = properties.containsKey(FORMAT_JSON_SCHEMA); if (deriveSchema && (hasSchema || hasSchemaString)) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java index f43234dde..caf99f482 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -119,7 +119,7 @@ private void testSchemaSerializationSchema(Map properties) { final SerializationSchema actual1 = TableFactoryService .find(SerializationSchemaFactory.class, properties) .createSerializationSchema(properties); - final SerializationSchema expected1 = new JsonRowSerializationSchema(SCHEMA); + final SerializationSchema expected1 = new JsonRowSerializationSchema(SCHEMA); assertEquals(expected1, actual1); } @@ -141,9 +141,9 @@ private void testJsonSchemaSerializationSchema(Map properties) { } private static Map toMap(Descriptor... desc) { - final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + final DescriptorProperties descriptorProperties = new DescriptorProperties(); for (Descriptor d : desc) { - d.addProperties(descriptorProperties); + descriptorProperties.putProperties(d.toProperties()); } return descriptorProperties.asMap(); } From ede44abdce928f128451a6fcdfd0b9de3560816b Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 4 Oct 2018 10:44:30 +0200 Subject: [PATCH 038/322] [FLINK-7811] Add Scala 2.12 profile This changes some modules that had a _2.11 dependency but didn't expose it in their module name to instead depend on the ${scala.binary.version} dependency. The main reason for this is to make the build self contained, before, with the hard-dependency on 2.11, when buildig for 2.12 it would not be clear where the dependency would come from because it is not created as part of the build. This could lead to inconsistencies. For example, when adding a new class in flink-runtime but not recompiling on 2.11 but only on 2.12, the 2.12 tests would fail when using that new class because they would use 2.11 dependencies that weren't rebuilt with the new class. We also don't build flink-scala-shell and flink-connector-kafka-0.8 because they don't work with Scala 2.12. This also includes $PROFILE in dependency convergence check script. This is in preparation for building with "-Dscala-212", where we hace to exclude certain things. We also exclude Kafka 0.8 from stages when building for Scala 2.12 And add Scala 2.12 to the release scripts --- flink-formats-kafka/flink-json-debezium/pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 4b1baa976..1d6428903 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -77,8 +77,7 @@ under the License. org.apache.flink - - flink-table_2.11 + flink-table_${scala.binary.version} ${project.version} test From 08451b924e73a2eee80472785b45c30cd372ff7f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 3 Nov 2018 12:08:28 +0100 Subject: [PATCH 039/322] Update version to 1.8-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index ea81997f3..0dc586943 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.7-SNAPSHOT + 1.8-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 9d8185196..72a74ac11 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.7-SNAPSHOT + 1.8-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 1d6428903..43af92c09 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.7-SNAPSHOT + 1.8-SNAPSHOT .. From 37e0d14a2a174a24152ed490d37233be3ecda405 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 23 Nov 2018 15:57:02 +0100 Subject: [PATCH 040/322] [FLINK-10997][formats] Bundle kafka-scheme-registry-client --- .../flink-avro-confluent-registry/pom.xml | 25 +++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 72a74ac11..4c99f71e5 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -78,10 +78,31 @@ under the License. shade + false + + + io.confluent:* + com.fasterxml.jackson.core:* + org.apache.zookeeper:zookeeper + com.101tec:zkclient + + - com.fasterxml.jackson.core - org.apache.flink.formats.avro.registry.confluent.shaded.com.fasterxml.jackson.core + com.fasterxml.jackson + org.apache.flink.formats.avro.registry.confluent.shaded.com.fasterxml.jackson + + + org.apache.zookeeper + org.apache.flink.formats.avro.registry.confluent.shaded.org.apache.zookeeper + + + org.apache.jute + org.apache.flink.formats.avro.registry.confluent.shaded.org.apache.jute + + + org.I0Itec.zkclient + org.apache.flink.formats.avro.registry.confluent.shaded.org.101tec From e159e6bac7798c59569db2c90d7e075a414bb8b5 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 27 Nov 2018 16:07:33 +0100 Subject: [PATCH 041/322] [FLINK-10987] Add LICENSE & NOTICE files for flink-avro-confluent-registry --- .../src/main/resources/META-INF/NOTICE | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE new file mode 100644 index 000000000..24f2ff846 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -0,0 +1,15 @@ +flink-avro-confluent-registry +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.101tec:zkclient:0.10 +- com.fasterxml.jackson.core:jackson-databind:2.8.4 +- com.fasterxml.jackson.core:jackson-annotations:2.8.0 +- com.fasterxml.jackson.core:jackson-core:2.8.4 +- io.confluent:common-utils:3.3.1 +- io.confluent:kafka-schema-registry-client:3.3.1 +- org.apache.zookeeper:zookeeper:3.4.10 From 95f8e9561de4d84c88f1f020107dda1d6534b7bf Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 28 Jan 2019 07:48:59 +0100 Subject: [PATCH 042/322] [FLINK-11064] [table] Setup a new flink-table module structure This commit splits the flink-table module into multiple submodules in accordance with FLIP-32 (step 1). The new module structure looks as follows: flink-table-common ^ | flink-table-api-java <------- flink-table-api-scala ^ ^ | | flink-table-api-java-bridge flink-table-api-scala-bridge ^ | flink-table-planner The module structure assumes that the type system has been reworked such that only one table environment exists for both Java and Scala users. The module `flink-table-planner` contains the content of the old `flink-table` module. From there we can distribute ported classes to their final module without breaking backwards compatibility or force users to update their dependencies again. For example, if a user wants to implement a pure table program in Scala, `flink-table-api-scala` and `flink-table-planner` need to be added to the project. Until we support pure table programs, `flink-table-api-scala/java-bridge` and `flink-table-planner` need to be added to the project. This closes #7587. --- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 43af92c09..dd5283674 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -77,7 +77,7 @@ under the License. org.apache.flink - flink-table_${scala.binary.version} + flink-table-planner_${scala.binary.version} ${project.version} test From f0a4e343d16c747f81987fb2de58424fd2d14b6a Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 26 Nov 2018 16:11:04 +0100 Subject: [PATCH 043/322] [FLINK-9964][table] Add a full CSV table format factory It adds CsvRowSerializationSchema, CsvRowDeserializationSchema, a new CSV descriptor, CsvRowTableFormatFactory, documentation and tests The format integrates nicely with most SQL types. It deprecates the "old CSV" descriptor stack and prepares also for FLINK-7050 (#4660). The old CSV descriptor is still available under as "OldCsv". This closes #7777. --- flink-formats-kafka/flink-json-debezium/pom.xml | 5 ----- .../flink/formats/json/JsonRowDeserializationSchema.java | 2 +- .../main/java/org/apache/flink/table/descriptors/Json.java | 4 ++-- 3 files changed, 3 insertions(+), 8 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index dd5283674..f8191b166 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -60,11 +60,6 @@ under the License. - - org.apache.flink - flink-test-utils-junit - - org.apache.flink diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index dc8a116ac..df1fbc5d8 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -46,7 +46,7 @@ *

Deserializes a byte[] message as a JSON object and reads * the specified fields. * - *

Failure during deserialization are forwarded as wrapped IOExceptions. + *

Failures during deserialization are forwarded as wrapped IOExceptions. */ @PublicEvolving public class JsonRowDeserializationSchema implements DeserializationSchema { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java index 3c4888c96..7bc7cfbf0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -93,11 +93,11 @@ public Json schema(TypeInformation schemaType) { } /** - * Derives the format schema from the table's schema described using {@link Schema}. + * Derives the format schema from the table's schema described. * *

This allows for defining schema information only once. * - *

The names, types, and field order of the format are determined by the table's + *

The names, types, and fields' order of the format are determined by the table's * schema. Time attributes are ignored if their origin is not a field. A "from" definition * is interpreted as a field renaming in the format. */ From be2bdf7e2ac52223c41faa8d32538ca3d7cd105a Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 25 Feb 2019 10:21:06 +0100 Subject: [PATCH 044/322] Update version to 1.9-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 0dc586943..71ffb9881 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.8-SNAPSHOT + 1.9-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 4c99f71e5..9c89167f4 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.8-SNAPSHOT + 1.9-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index f8191b166..8fe48c8f9 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.8-SNAPSHOT + 1.9-SNAPSHOT .. From 34632351912cfa1a4a87a278f6f099122398bf4b Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 13 Mar 2019 14:52:58 +0100 Subject: [PATCH 045/322] [FLINK-11901][build] Update NOTICE files with year 2019 This closes #7975. --- .../src/main/resources/META-INF/NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 24f2ff846..8a6da1e04 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-avro-confluent-registry -Copyright 2014-2018 The Apache Software Foundation +Copyright 2014-2019 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From 3862d9ab4e175d8e9fb82c5557be131ef0962e03 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Wed, 22 May 2019 19:16:02 +0900 Subject: [PATCH 046/322] [FLINK-12578][build] Use https URL for Maven repositories MapR repositories are excluded from this change as there appears to be some certificate issue on the MapR side. The latest documentation (https://mapr.com/docs/61/DevelopmentGuide/MavenArtifacts.html) also recommends using the http url. --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 71ffb9881..12d495089 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -35,7 +35,7 @@ under the License. confluent - http://packages.confluent.io/maven/ + https://packages.confluent.io/maven/ diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 9c89167f4..aa6382fee 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -32,7 +32,7 @@ under the License. confluent - http://packages.confluent.io/maven/ + https://packages.confluent.io/maven/ From 47e90f25014967d335c8a121bb9d92a61b553517 Mon Sep 17 00:00:00 2001 From: Kurt Young Date: Fri, 12 Jul 2019 14:25:23 +0800 Subject: [PATCH 047/322] Update version to 1.10-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 12d495089..85fd21d82 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.9-SNAPSHOT + 1.10-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index aa6382fee..5366cbd57 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.9-SNAPSHOT + 1.10-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 8fe48c8f9..eb9e3472f 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.9-SNAPSHOT + 1.10-SNAPSHOT .. From b75bed69b586c6499c8281c1209b6fcfa57e52a1 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 5 Mar 2019 10:45:20 +0100 Subject: [PATCH 048/322] [FLINK-11727][formats] Fixed JSON format issues after serialization This commit reworks JSON format to use a runtime converter created based on given TypeInformation. Pre this commit conversion logic was based on reference comparison of TypeInformation which was not working after serialization of the format. This also introduces a builder pattern for ensuring future immutability of schemas. This closes #7932. --- .../json/JsonRowDeserializationSchema.java | 394 +++++++++++++----- .../formats/json/JsonRowFormatFactory.java | 13 +- .../json/JsonRowSerializationSchema.java | 332 ++++++++++----- .../flink/formats/json/TimeFormats.java | 46 ++ .../JsonRowDeserializationSchemaTest.java | 66 +-- .../json/JsonRowFormatFactoryTest.java | 12 +- .../json/JsonRowSerializationSchemaTest.java | 63 +-- .../utils/DeserializationSchemaMatcher.java | 164 ++++++++ .../utils/SerializationSchemaMatcher.java | 192 +++++++++ 9 files changed, 1010 insertions(+), 272 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index df1fbc5d8..4a1ff274b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -27,18 +27,39 @@ import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; +import org.apache.flink.util.WrappingRuntimeException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import java.io.IOException; -import java.lang.reflect.Array; +import java.io.Serializable; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.time.temporal.TemporalAccessor; +import java.time.temporal.TemporalQueries; +import java.util.Arrays; +import java.util.List; import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +import static java.lang.String.format; +import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; +import static java.util.Spliterators.spliterator; +import static java.util.stream.StreamSupport.stream; +import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * Deserialization schema from JSON to Flink types. @@ -54,45 +75,56 @@ public class JsonRowDeserializationSchema implements DeserializationSchema private static final long serialVersionUID = -228294330688809195L; /** Type information describing the result type. */ - private final TypeInformation typeInfo; + private final RowTypeInfo typeInfo; + + private boolean failOnMissingField; /** Object mapper for parsing the JSON. */ private final ObjectMapper objectMapper = new ObjectMapper(); - /** Flag indicating whether to fail on a missing field. */ - private boolean failOnMissingField; + private DeserializationRuntimeConverter runtimeConverter; + + private JsonRowDeserializationSchema( + TypeInformation typeInfo, + boolean failOnMissingField) { + checkNotNull(typeInfo, "Type information"); + checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + this.typeInfo = (RowTypeInfo) typeInfo; + this.failOnMissingField = failOnMissingField; + this.runtimeConverter = createConverter(this.typeInfo); + } /** - * Creates a JSON deserialization schema for the given type information. - * - * @param typeInfo Type information describing the result type. The field names of {@link Row} - * are used to parse the JSON properties. + * @deprecated Use the provided {@link Builder} instead. */ + @Deprecated public JsonRowDeserializationSchema(TypeInformation typeInfo) { - Preconditions.checkNotNull(typeInfo, "Type information"); - this.typeInfo = typeInfo; - - if (!(typeInfo instanceof RowTypeInfo)) { - throw new IllegalArgumentException("Row type information expected."); - } + this(typeInfo, false); } /** - * Creates a JSON deserialization schema for the given JSON schema. - * - * @param jsonSchema JSON schema describing the result type - * - * @see http://json-schema.org/ + * @deprecated Use the provided {@link Builder} instead. */ + @Deprecated public JsonRowDeserializationSchema(String jsonSchema) { - this(JsonRowSchemaConverter.convert(jsonSchema)); + this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)), false); + } + + /** + * @deprecated Use the provided {@link Builder} instead. + */ + @Deprecated + public void setFailOnMissingField(boolean failOnMissingField) { + // TODO make this class immutable once we drop this method + this.failOnMissingField = failOnMissingField; + this.runtimeConverter = createConverter(this.typeInfo); } @Override public Row deserialize(byte[] message) throws IOException { try { final JsonNode root = objectMapper.readTree(message); - return convertRow(root, (RowTypeInfo) typeInfo); + return (Row) runtimeConverter.convert(objectMapper, root); } catch (Throwable t) { throw new IOException("Failed to deserialize JSON object.", t); } @@ -109,14 +141,48 @@ public TypeInformation getProducedType() { } /** - * Configures the failure behaviour if a JSON field is missing. - * - *

By default, a missing field is ignored and the field is set to null. - * - * @param failOnMissingField Flag indicating whether to fail or not on a missing field. + * Builder for {@link JsonRowDeserializationSchema}. */ - public void setFailOnMissingField(boolean failOnMissingField) { - this.failOnMissingField = failOnMissingField; + public static class Builder { + + private final RowTypeInfo typeInfo; + private boolean failOnMissingField = false; + + /** + * Creates a JSON deserialization schema for the given type information. + * + * @param typeInfo Type information describing the result type. The field names of {@link Row} + * are used to parse the JSON properties. + */ + public Builder(TypeInformation typeInfo) { + checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + this.typeInfo = (RowTypeInfo) typeInfo; + } + + /** + * Creates a JSON deserialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * + * @see http://json-schema.org/ + */ + public Builder(String jsonSchema) { + this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema))); + } + + /** + * Configures schema to fail if a JSON field is missing. + * + *

By default, a missing field is ignored and the field is set to null. + */ + public Builder failOnMissingField() { + this.failOnMissingField = true; + return this; + } + + public JsonRowDeserializationSchema build() { + return new JsonRowDeserializationSchema(typeInfo, failOnMissingField); + } } @Override @@ -128,7 +194,8 @@ public boolean equals(Object o) { return false; } final JsonRowDeserializationSchema that = (JsonRowDeserializationSchema) o; - return failOnMissingField == that.failOnMissingField && Objects.equals(typeInfo, that.typeInfo); + return Objects.equals(typeInfo, that.typeInfo) && + Objects.equals(failOnMissingField, that.failOnMissingField); } @Override @@ -136,99 +203,214 @@ public int hashCode() { return Objects.hash(typeInfo, failOnMissingField); } - // -------------------------------------------------------------------------------------------- - - private Object convert(JsonNode node, TypeInformation info) { - if (info == Types.VOID || node.isNull()) { - return null; - } else if (info == Types.BOOLEAN) { - return node.asBoolean(); - } else if (info == Types.STRING) { - return node.asText(); - } else if (info == Types.BIG_DEC) { - return node.decimalValue(); - } else if (info == Types.BIG_INT) { - return node.bigIntegerValue(); - } else if (info == Types.SQL_DATE) { - return Date.valueOf(node.asText()); - } else if (info == Types.SQL_TIME) { - // according to RFC 3339 every full-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - final String time = node.asText(); - if (time.indexOf('Z') < 0 || time.indexOf('.') >= 0) { - throw new IllegalStateException( - "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet. " + - "Format: HH:mm:ss'Z'"); + /* + Runtime converter + */ + + /** + * Runtime converter that maps between {@link JsonNode}s and Java objects. + */ + @FunctionalInterface + private interface DeserializationRuntimeConverter extends Serializable { + Object convert(ObjectMapper mapper, JsonNode jsonNode); + } + + private DeserializationRuntimeConverter createConverter(TypeInformation typeInfo) { + DeserializationRuntimeConverter baseConverter = createConverterForSimpleType(typeInfo) + .orElseGet(() -> + createContainerConverter(typeInfo) + .orElseGet(() -> createFallbackConverter(typeInfo.getTypeClass()))); + return wrapIntoNullableConverter(baseConverter); + } + + private DeserializationRuntimeConverter wrapIntoNullableConverter(DeserializationRuntimeConverter converter) { + return (mapper, jsonNode) -> { + if (jsonNode.isNull()) { + return null; } - return Time.valueOf(time.substring(0, time.length() - 1)); - } else if (info == Types.SQL_TIMESTAMP) { + + return converter.convert(mapper, jsonNode); + }; + } + + private Optional createContainerConverter(TypeInformation typeInfo) { + if (typeInfo instanceof RowTypeInfo) { + return Optional.of(createRowConverter((RowTypeInfo) typeInfo)); + } else if (typeInfo instanceof ObjectArrayTypeInfo) { + return Optional.of(createObjectArrayConverter(((ObjectArrayTypeInfo) typeInfo).getComponentInfo())); + } else if (typeInfo instanceof BasicArrayTypeInfo) { + return Optional.of(createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo())); + } else if (isPrimitiveByteArray(typeInfo)) { + return Optional.of(createByteArrayConverter()); + } else { + return Optional.empty(); + } + } + + private DeserializationRuntimeConverter createByteArrayConverter() { + return (mapper, jsonNode) -> { + try { + return jsonNode.binaryValue(); + } catch (IOException e) { + throw new WrappingRuntimeException("Unable to deserialize byte array.", e); + } + }; + } + + private boolean isPrimitiveByteArray(TypeInformation typeInfo) { + return typeInfo instanceof PrimitiveArrayTypeInfo && + ((PrimitiveArrayTypeInfo) typeInfo).getComponentType() == Types.BYTE; + } + + private DeserializationRuntimeConverter createObjectArrayConverter(TypeInformation elementTypeInfo) { + DeserializationRuntimeConverter elementConverter = createConverter(elementTypeInfo); + return assembleArrayConverter(elementConverter); + } + + private DeserializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) { + List fieldConverters = Arrays.stream(typeInfo.getFieldTypes()) + .map(this::createConverter) + .collect(Collectors.toList()); + + return assembleRowConverter(typeInfo.getFieldNames(), fieldConverters); + } + + private DeserializationRuntimeConverter createFallbackConverter(Class valueType) { + return (mapper, jsonNode) -> { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.treeToValue(jsonNode, valueType); + } catch (JsonProcessingException e) { + throw new WrappingRuntimeException(format("Could not convert node: %s", jsonNode), e); + } + }; + } + + private Optional createConverterForSimpleType(TypeInformation simpleTypeInfo) { + if (simpleTypeInfo == Types.VOID) { + return Optional.of((mapper, jsonNode) -> null); + } else if (simpleTypeInfo == Types.BOOLEAN) { + return Optional.of((mapper, jsonNode) -> jsonNode.asBoolean()); + } else if (simpleTypeInfo == Types.STRING) { + return Optional.of((mapper, jsonNode) -> jsonNode.asText()); + } else if (simpleTypeInfo == Types.INT) { + return Optional.of((mapper, jsonNode) -> jsonNode.asInt()); + } else if (simpleTypeInfo == Types.LONG) { + return Optional.of((mapper, jsonNode) -> jsonNode.asLong()); + } else if (simpleTypeInfo == Types.DOUBLE) { + return Optional.of((mapper, jsonNode) -> jsonNode.asDouble()); + } else if (simpleTypeInfo == Types.FLOAT) { + return Optional.of((mapper, jsonNode) -> Float.parseFloat(jsonNode.asText().trim())); + } else if (simpleTypeInfo == Types.SHORT) { + return Optional.of((mapper, jsonNode) -> Short.parseShort(jsonNode.asText().trim())); + } else if (simpleTypeInfo == Types.BYTE) { + return Optional.of((mapper, jsonNode) -> Byte.parseByte(jsonNode.asText().trim())); + } else if (simpleTypeInfo == Types.BIG_DEC) { + return Optional.of((mapper, jsonNode) -> jsonNode.decimalValue()); + } else if (simpleTypeInfo == Types.BIG_INT) { + return Optional.of((mapper, jsonNode) -> jsonNode.bigIntegerValue()); + } else if (simpleTypeInfo == Types.SQL_DATE) { + return Optional.of(createDateConverter()); + } else if (simpleTypeInfo == Types.SQL_TIME) { + return Optional.of(createTimeConverter()); + } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) { + return Optional.of(createTimestampConverter()); + } else { + return Optional.empty(); + } + } + + private DeserializationRuntimeConverter createDateConverter() { + return (mapper, jsonNode) -> Date.valueOf(ISO_LOCAL_DATE.parse(jsonNode.asText()) + .query(TemporalQueries.localDate())); + } + + private DeserializationRuntimeConverter createTimestampConverter() { + return (mapper, jsonNode) -> { // according to RFC 3339 every date-time must have a timezone; // until we have full timezone support, we only support UTC; // users can parse their time as string as a workaround - final String timestamp = node.asText(); - if (timestamp.indexOf('Z') < 0) { + TemporalAccessor parsedTimestamp = RFC3339_TIMESTAMP_FORMAT.parse(jsonNode.asText()); + + ZoneOffset zoneOffset = parsedTimestamp.query(TemporalQueries.offset()); + + if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0) { throw new IllegalStateException( "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); } - return Timestamp.valueOf(timestamp.substring(0, timestamp.length() - 1).replace('T', ' ')); - } else if (info instanceof RowTypeInfo) { - return convertRow(node, (RowTypeInfo) info); - } else if (info instanceof ObjectArrayTypeInfo) { - return convertObjectArray(node, ((ObjectArrayTypeInfo) info).getComponentInfo()); - } else if (info instanceof BasicArrayTypeInfo) { - return convertObjectArray(node, ((BasicArrayTypeInfo) info).getComponentInfo()); - } else if (info instanceof PrimitiveArrayTypeInfo && - ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { - return convertByteArray(node); - } else { - // for types that were specified without JSON schema - // e.g. POJOs - try { - return objectMapper.treeToValue(node, info.getTypeClass()); - } catch (JsonProcessingException e) { - throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + + LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); + LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); + + return Timestamp.valueOf(LocalDateTime.of(localDate, localTime)); + }; + } + + private DeserializationRuntimeConverter createTimeConverter() { + return (mapper, jsonNode) -> { + + // according to RFC 3339 every full-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText()); + + ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset()); + LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); + + if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0 || localTime.getNano() != 0) { + throw new IllegalStateException( + "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet."); } - } + + return Time.valueOf(localTime); + }; } - private Row convertRow(JsonNode node, RowTypeInfo info) { - final String[] names = info.getFieldNames(); - final TypeInformation[] types = info.getFieldTypes(); - - final Row row = new Row(names.length); - for (int i = 0; i < names.length; i++) { - final String name = names[i]; - final JsonNode subNode = node.get(name); - if (subNode == null) { - if (failOnMissingField) { - throw new IllegalStateException( - "Could not find field with name '" + name + "'."); - } else { - row.setField(i, null); - } - } else { - row.setField(i, convert(subNode, types[i])); + private DeserializationRuntimeConverter assembleRowConverter( + String[] fieldNames, + List fieldConverters) { + return (mapper, jsonNode) -> { + ObjectNode node = (ObjectNode) jsonNode; + + int arity = fieldNames.length; + Row row = new Row(arity); + for (int i = 0; i < arity; i++) { + String fieldName = fieldNames[i]; + JsonNode field = node.get(fieldName); + Object convertField = convertField(mapper, fieldConverters.get(i), fieldName, field); + row.setField(i, convertField); } - } - return row; + return row; + }; } - private Object convertObjectArray(JsonNode node, TypeInformation elementType) { - final Object[] array = (Object[]) Array.newInstance(elementType.getTypeClass(), node.size()); - for (int i = 0; i < node.size(); i++) { - array[i] = convert(node.get(i), elementType); + private Object convertField( + ObjectMapper mapper, + DeserializationRuntimeConverter fieldConverter, + String fieldName, + JsonNode field) { + if (field == null) { + if (failOnMissingField) { + throw new IllegalStateException( + "Could not find field with name '" + fieldName + "'."); + } else { + return null; + } + } else { + return fieldConverter.convert(mapper, field); } - return array; } - private Object convertByteArray(JsonNode node) { - try { - return node.binaryValue(); - } catch (IOException e) { - throw new RuntimeException("Unable to deserialize byte array.", e); - } + private DeserializationRuntimeConverter assembleArrayConverter(DeserializationRuntimeConverter elementConverter) { + return (mapper, jsonNode) -> { + ArrayNode node = (ArrayNode) jsonNode; + + return stream(spliterator(node.elements(), node.size(), 0), false) + .map(innerNode -> elementConverter.convert(mapper, innerNode)) + .toArray(); + }; } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java index 567bef3a8..af758b63c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -58,12 +58,17 @@ public DeserializationSchema createDeserializationSchema(Map { + if (flag) { + schema.failOnMissingField(); + } + }); - return schema; + return schema.build(); } @Override @@ -71,7 +76,7 @@ public SerializationSchema createSerializationSchema(Map pr final DescriptorProperties descriptorProperties = getValidatedProperties(properties); // create and configure - return new JsonRowSerializationSchema(createTypeInformation(descriptorProperties)); + return new JsonRowSerializationSchema.Builder(createTypeInformation(descriptorProperties)).build(); } private TypeInformation createTypeInformation(DescriptorProperties descriptorProperties) { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index d9420629f..0aa7151fc 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -27,19 +27,32 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.WrappingRuntimeException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import java.io.Serializable; import java.math.BigDecimal; import java.math.BigInteger; +import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; -import java.text.SimpleDateFormat; +import java.util.Arrays; +import java.util.List; import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +import static java.lang.String.format; +import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; +import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * Serialization schema that serializes an object of Flink types into a JSON bytes. @@ -55,42 +68,61 @@ public class JsonRowSerializationSchema implements SerializationSchema { private static final long serialVersionUID = -2885556750743978636L; /** Type information describing the input type. */ - private final TypeInformation typeInfo; + private final RowTypeInfo typeInfo; /** Object mapper that is used to create output JSON objects. */ private final ObjectMapper mapper = new ObjectMapper(); - /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone, without milliseconds). */ - private SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm:ss'Z'"); - - /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone). */ - private SimpleDateFormat timeFormatWithMillis = new SimpleDateFormat("HH:mm:ss.SSS'Z'"); - - /** Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC timezone). */ - private SimpleDateFormat timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + private final SerializationRuntimeConverter runtimeConverter; /** Reusable object node. */ private transient ObjectNode node; /** - * Creates a JSON serialization schema for the given type information. - * - * @param typeInfo The field names of {@link Row} are used to map to JSON properties. + * @deprecated Use the provided {@link Builder} instead. */ + @Deprecated public JsonRowSerializationSchema(TypeInformation typeInfo) { + // TODO make this constructor private in the future Preconditions.checkNotNull(typeInfo, "Type information"); - this.typeInfo = typeInfo; + Preconditions.checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + this.typeInfo = (RowTypeInfo) typeInfo; + this.runtimeConverter = createConverter(typeInfo); } /** - * Creates a JSON serialization schema for the given JSON schema. - * - * @param jsonSchema JSON schema describing the result type - * - * @see http://json-schema.org/ + * Builder for {@link JsonRowSerializationSchema}. */ - public JsonRowSerializationSchema(String jsonSchema) { - this(JsonRowSchemaConverter.convert(jsonSchema)); + @PublicEvolving + public static class Builder { + + private final RowTypeInfo typeInfo; + + /** + * Creates a JSON serialization schema for the given type information. + * + * @param typeInfo Type information describing the result type. The field names of {@link Row} + * are used to parse the JSON properties. + */ + public Builder(TypeInformation typeInfo) { + checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + this.typeInfo = (RowTypeInfo) typeInfo; + } + + /** + * Creates a JSON serialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * + * @see http://json-schema.org/ + */ + public Builder(String jsonSchema) { + this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema))); + } + + public JsonRowSerializationSchema build() { + return new JsonRowSerializationSchema(typeInfo); + } } @Override @@ -100,7 +132,7 @@ public byte[] serialize(Row row) { } try { - convertRow(node, (RowTypeInfo) typeInfo, row); + runtimeConverter.convert(mapper, node, row); return mapper.writeValueAsBytes(node); } catch (Throwable t) { throw new RuntimeException("Could not serialize row '" + row + "'. " + @@ -125,102 +157,204 @@ public int hashCode() { return Objects.hash(typeInfo); } - // -------------------------------------------------------------------------------------------- + /* + Runtime converters + */ - private ObjectNode convertRow(ObjectNode reuse, RowTypeInfo info, Row row) { - if (reuse == null) { - reuse = mapper.createObjectNode(); - } - final String[] fieldNames = info.getFieldNames(); - final TypeInformation[] fieldTypes = info.getFieldTypes(); + /** + * Runtime converter that maps between Java objects and corresponding {@link JsonNode}s. + */ + @FunctionalInterface + private interface SerializationRuntimeConverter extends Serializable { + JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object object); + } + + private SerializationRuntimeConverter createConverter(TypeInformation typeInfo) { + SerializationRuntimeConverter baseConverter = createConverterForSimpleType(typeInfo) + .orElseGet(() -> + createContainerConverter(typeInfo) + .orElseGet(this::createFallbackConverter)); + return wrapIntoNullableConverter(baseConverter); + } + + private SerializationRuntimeConverter wrapIntoNullableConverter(SerializationRuntimeConverter converter) { + return (mapper, reuse, object) -> { + if (object == null) { + return mapper.getNodeFactory().nullNode(); + } - // validate the row - if (row.getArity() != fieldNames.length) { - throw new IllegalStateException(String.format( - "Number of elements in the row '%s' is different from number of field names: %d", row, fieldNames.length)); + return converter.convert(mapper, reuse, object); + }; + } + + private Optional createContainerConverter(TypeInformation typeInfo) { + if (typeInfo instanceof RowTypeInfo) { + return Optional.of(createRowConverter((RowTypeInfo) typeInfo)); + } else if (typeInfo instanceof ObjectArrayTypeInfo) { + return Optional.of(createObjectArrayConverter(((ObjectArrayTypeInfo) typeInfo).getComponentInfo())); + } else if (typeInfo instanceof BasicArrayTypeInfo) { + return Optional.of(createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo())); + } else if (isPrimitiveByteArray(typeInfo)) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().binaryNode((byte[]) object)); + } else { + return Optional.empty(); } + } + + private boolean isPrimitiveByteArray(TypeInformation typeInfo) { + return typeInfo instanceof PrimitiveArrayTypeInfo && + ((PrimitiveArrayTypeInfo) typeInfo).getComponentType() == Types.BYTE; + } + + private SerializationRuntimeConverter createObjectArrayConverter(TypeInformation elementTypeInfo) { + SerializationRuntimeConverter elementConverter = createConverter(elementTypeInfo); + return assembleArrayConverter(elementConverter); + } - for (int i = 0; i < fieldNames.length; i++) { - final String name = fieldNames[i]; + private SerializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) { + List fieldConverters = Arrays.stream(typeInfo.getFieldTypes()) + .map(this::createConverter) + .collect(Collectors.toList()); + + return assembleRowConverter(typeInfo.getFieldNames(), fieldConverters); + } - final JsonNode fieldConverted = convert(reuse, reuse.get(name), fieldTypes[i], row.getField(i)); - reuse.set(name, fieldConverted); + private SerializationRuntimeConverter createFallbackConverter() { + return (mapper, reuse, object) -> { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.valueToTree(object); + } catch (IllegalArgumentException e) { + throw new WrappingRuntimeException(format("Could not convert object: %s", object), e); + } + }; + } + + private Optional createConverterForSimpleType(TypeInformation simpleTypeInfo) { + if (simpleTypeInfo == Types.VOID) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().nullNode()); + } else if (simpleTypeInfo == Types.BOOLEAN) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().booleanNode((Boolean) object)); + } else if (simpleTypeInfo == Types.STRING) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().textNode((String) object)); + } else if (simpleTypeInfo == Types.INT) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Integer) object)); + } else if (simpleTypeInfo == Types.LONG) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Long) object)); + } else if (simpleTypeInfo == Types.DOUBLE) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Double) object)); + } else if (simpleTypeInfo == Types.FLOAT) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Float) object)); + } else if (simpleTypeInfo == Types.SHORT) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Short) object)); + } else if (simpleTypeInfo == Types.BYTE) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Byte) object)); + } else if (simpleTypeInfo == Types.BIG_DEC) { + return Optional.of(createBigDecimalConverter()); + } else if (simpleTypeInfo == Types.BIG_INT) { + return Optional.of(createBigIntegerConverter()); + } else if (simpleTypeInfo == Types.SQL_DATE) { + return Optional.of(createDateConverter()); + } else if (simpleTypeInfo == Types.SQL_TIME) { + return Optional.of(createTimeConverter()); + } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) { + return Optional.of(createTimestampConverter()); + } else { + return Optional.empty(); } + } + + private SerializationRuntimeConverter createDateConverter() { + return (mapper, reuse, object) -> { + Date date = (Date) object; + + return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format(date.toLocalDate())); + }; + } + + private SerializationRuntimeConverter createTimestampConverter() { + return (mapper, reuse, object) -> { + Timestamp timestamp = (Timestamp) object; - return reuse; + return mapper.getNodeFactory() + .textNode(RFC3339_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); + }; } - private JsonNode convert(ContainerNode container, JsonNode reuse, TypeInformation info, Object object) { - if (info == Types.VOID || object == null) { - return container.nullNode(); - } else if (info == Types.BOOLEAN) { - return container.booleanNode((Boolean) object); - } else if (info == Types.STRING) { - return container.textNode((String) object); - } else if (info == Types.BIG_DEC) { + private SerializationRuntimeConverter createTimeConverter() { + return (mapper, reuse, object) -> { + final Time time = (Time) object; + + JsonNodeFactory nodeFactory = mapper.getNodeFactory(); + return nodeFactory.textNode(RFC3339_TIME_FORMAT.format(time.toLocalTime())); + }; + } + + private SerializationRuntimeConverter createBigDecimalConverter() { + return (mapper, reuse, object) -> { // convert decimal if necessary + JsonNodeFactory nodeFactory = mapper.getNodeFactory(); if (object instanceof BigDecimal) { - return container.numberNode((BigDecimal) object); + return nodeFactory.numberNode((BigDecimal) object); } - return container.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); - } else if (info == Types.BIG_INT) { - // convert integer if necessary + return nodeFactory.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); + }; + } + + private SerializationRuntimeConverter createBigIntegerConverter() { + return (mapper, reuse, object) -> { + // convert decimal if necessary + JsonNodeFactory nodeFactory = mapper.getNodeFactory(); if (object instanceof BigInteger) { - return container.numberNode((BigInteger) object); + return nodeFactory.numberNode((BigInteger) object); } - return container.numberNode(BigInteger.valueOf(((Number) object).longValue())); - } else if (info == Types.SQL_DATE) { - return container.textNode(object.toString()); - } else if (info == Types.SQL_TIME) { - final Time time = (Time) object; - // strip milliseconds if possible - if (time.getTime() % 1000 > 0) { - return container.textNode(timeFormatWithMillis.format(time)); - } - return container.textNode(timeFormat.format(time)); - } else if (info == Types.SQL_TIMESTAMP) { - return container.textNode(timestampFormat.format((Timestamp) object)); - } else if (info instanceof RowTypeInfo) { - if (reuse != null && reuse instanceof ObjectNode) { - return convertRow((ObjectNode) reuse, (RowTypeInfo) info, (Row) object); + return nodeFactory.numberNode(BigInteger.valueOf(((Number) object).longValue())); + }; + } + + private SerializationRuntimeConverter assembleRowConverter( + String[] fieldNames, + List fieldConverters) { + return (mapper, reuse, object) -> { + ObjectNode node; + + if (reuse == null) { + node = mapper.createObjectNode(); } else { - return convertRow(null, (RowTypeInfo) info, (Row) object); + node = (ObjectNode) reuse; } - } else if (info instanceof ObjectArrayTypeInfo) { - if (reuse != null && reuse instanceof ArrayNode) { - return convertObjectArray((ArrayNode) reuse, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); - } else { - return convertObjectArray(null, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + + Row row = (Row) object; + + for (int i = 0; i < fieldNames.length; i++) { + String fieldName = fieldNames[i]; + node.set(fieldName, + fieldConverters.get(i).convert(mapper, node.get(fieldNames[i]), row.getField(i))); } - } else if (info instanceof BasicArrayTypeInfo) { - if (reuse != null && reuse instanceof ArrayNode) { - return convertObjectArray((ArrayNode) reuse, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + + return node; + }; + } + + private SerializationRuntimeConverter assembleArrayConverter(SerializationRuntimeConverter elementConverter) { + return (mapper, reuse, object) -> { + ArrayNode node; + + if (reuse == null) { + node = mapper.createArrayNode(); } else { - return convertObjectArray(null, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + node = (ArrayNode) reuse; + node.removeAll(); } - } else if (info instanceof PrimitiveArrayTypeInfo && ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { - return container.binaryNode((byte[]) object); - } else { - // for types that were specified without JSON schema - // e.g. POJOs - try { - return mapper.valueToTree(object); - } catch (IllegalArgumentException e) { - throw new IllegalStateException("Unsupported type information '" + info + "' for object: " + object, e); - } - } - } - private ArrayNode convertObjectArray(ArrayNode reuse, TypeInformation info, Object[] array) { - if (reuse == null) { - reuse = mapper.createArrayNode(); - } else { - reuse.removeAll(); - } + Object[] array = (Object[]) object; - for (Object object : array) { - reuse.add(convert(reuse, null, info, object)); - } - return reuse; + for (Object element : array) { + node.add(elementConverter.convert(mapper, null, element)); + } + + return node; + }; } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java new file mode 100644 index 000000000..c946c5db7 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; + +/** + * Time formats respecting the RFC3339 specification. + */ +class TimeFormats { + + /** Formatter for RFC 3339-compliant string representation of a time value. */ + static final DateTimeFormatter RFC3339_TIME_FORMAT = new DateTimeFormatterBuilder() + .appendPattern("HH:mm:ss") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) + .appendPattern("'Z'") + .toFormatter(); + + /** Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC timezone). */ + static final DateTimeFormatter RFC3339_TIMESTAMP_FORMAT = new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral('T') + .append(RFC3339_TIME_FORMAT) + .toFormatter(); + + private TimeFormats() { + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index 5e77b801a..057301909 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.types.Row; @@ -27,15 +28,16 @@ import org.junit.Assert; import org.junit.Test; -import java.io.IOException; import java.math.BigDecimal; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; import java.util.concurrent.ThreadLocalRandom; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; +import static org.apache.flink.formats.utils.DeserializationSchemaMatcher.whenDeserializedWith; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertThat; +import static org.junit.internal.matchers.ThrowableCauseMatcher.hasCause; /** * Tests for the {@link JsonRowDeserializationSchema}. @@ -62,18 +64,18 @@ public void testTypeInfoDeserialization() throws Exception { byte[] serializedJson = objectMapper.writeValueAsBytes(root); - JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema( + JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder( Types.ROW_NAMED( - new String[] { "id", "name", "bytes" }, + new String[]{"id", "name", "bytes"}, Types.LONG, Types.STRING, Types.PRIMITIVE_ARRAY(Types.BYTE)) - ); + ).build(); - Row deserialized = deserializationSchema.deserialize(serializedJson); + Row row = new Row(3); + row.setField(0, id); + row.setField(1, name); + row.setField(2, bytes); - assertEquals(3, deserialized.getArity()); - assertEquals(id, deserialized.getField(0)); - assertEquals(name, deserialized.getField(1)); - assertArrayEquals(bytes, (byte[]) deserialized.getField(2)); + assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); } @Test @@ -103,7 +105,7 @@ public void testSchemaDeserialization() throws Exception { final byte[] serializedJson = objectMapper.writeValueAsBytes(root); - JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema( + JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder( "{" + " type: 'object'," + " properties: {" + @@ -124,9 +126,7 @@ public void testSchemaDeserialization() throws Exception { " }" + " }" + " }" + - "}"); - - final Row deserialized = deserializationSchema.deserialize(serializedJson); + "}").build(); final Row expected = new Row(10); expected.setField(0, id); @@ -143,7 +143,7 @@ public void testSchemaDeserialization() throws Exception { nestedRow.setField(1, BigDecimal.valueOf(12)); expected.setField(9, nestedRow); - assertEquals(expected, deserialized); + assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(expected)); } /** @@ -158,25 +158,25 @@ public void testMissingNode() throws Exception { root.put("id", 123123123); byte[] serializedJson = objectMapper.writeValueAsBytes(root); - JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema( - Types.ROW_NAMED( - new String[] { "name" }, - Types.STRING) - ); + TypeInformation rowTypeInformation = Types.ROW_NAMED( + new String[]{"name"}, + Types.STRING); - Row row = deserializationSchema.deserialize(serializedJson); + JsonRowDeserializationSchema deserializationSchema = + new JsonRowDeserializationSchema.Builder(rowTypeInformation) + .build(); - assertEquals(1, row.getArity()); - Assert.assertNull("Missing field not null", row.getField(0)); + Row row = new Row(1); + assertThat(serializedJson, + whenDeserializedWith(deserializationSchema).equalsTo(row)); - deserializationSchema.setFailOnMissingField(true); + deserializationSchema = new JsonRowDeserializationSchema.Builder(rowTypeInformation) + .failOnMissingField() + .build(); - try { - deserializationSchema.deserialize(serializedJson); - Assert.fail("Did not throw expected Exception"); - } catch (IOException e) { - Assert.assertTrue(e.getCause() instanceof IllegalStateException); - } + assertThat(serializedJson, + whenDeserializedWith(deserializationSchema) + .failsWithException(hasCause(instanceOf(IllegalStateException.class)))); } /** @@ -185,10 +185,10 @@ public void testMissingNode() throws Exception { @Test public void testNumberOfFieldNamesAndTypesMismatch() { try { - new JsonRowDeserializationSchema( + new JsonRowDeserializationSchema.Builder( Types.ROW_NAMED( new String[]{"one", "two", "three"}, - Types.LONG)); + Types.LONG)).build(); Assert.fail("Did not throw expected Exception"); } catch (IllegalArgumentException ignored) { // Expected diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java index caf99f482..47b06c49b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -110,8 +110,7 @@ private void testSchemaDeserializationSchema(Map properties) { final DeserializationSchema actual2 = TableFactoryService .find(DeserializationSchemaFactory.class, properties) .createDeserializationSchema(properties); - final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema(SCHEMA); - expected2.setFailOnMissingField(false); + final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema.Builder(SCHEMA).build(); assertEquals(expected2, actual2); } @@ -119,7 +118,7 @@ private void testSchemaSerializationSchema(Map properties) { final SerializationSchema actual1 = TableFactoryService .find(SerializationSchemaFactory.class, properties) .createSerializationSchema(properties); - final SerializationSchema expected1 = new JsonRowSerializationSchema(SCHEMA); + final SerializationSchema expected1 = new JsonRowSerializationSchema.Builder(SCHEMA).build(); assertEquals(expected1, actual1); } @@ -127,8 +126,9 @@ private void testJsonSchemaDeserializationSchema(Map properties) final DeserializationSchema actual2 = TableFactoryService .find(DeserializationSchemaFactory.class, properties) .createDeserializationSchema(properties); - final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema(JSON_SCHEMA); - expected2.setFailOnMissingField(true); + final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema.Builder(JSON_SCHEMA) + .failOnMissingField() + .build(); assertEquals(expected2, actual2); } @@ -136,7 +136,7 @@ private void testJsonSchemaSerializationSchema(Map properties) { final SerializationSchema actual1 = TableFactoryService .find(SerializationSchemaFactory.class, properties) .createSerializationSchema(properties); - final SerializationSchema expected1 = new JsonRowSerializationSchema(JSON_SCHEMA); + final SerializationSchema expected1 = new JsonRowSerializationSchema.Builder(JSON_SCHEMA).build(); assertEquals(expected1, actual1); } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java index e2410d4a7..cc1f5bf37 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java @@ -30,7 +30,10 @@ import java.sql.Timestamp; import java.util.concurrent.ThreadLocalRandom; +import static org.apache.flink.formats.utils.SerializationSchemaMatcher.whenSerializedWith; +import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; /** * Tests for the {@link JsonRowSerializationSchema}. @@ -38,7 +41,7 @@ public class JsonRowSerializationSchemaTest { @Test - public void testRowSerialization() throws IOException { + public void testRowSerialization() { final TypeInformation rowSchema = Types.ROW_NAMED( new String[] {"f1", "f2", "f3"}, Types.INT, Types.BOOLEAN, Types.STRING); @@ -48,8 +51,14 @@ public void testRowSerialization() throws IOException { row.setField(1, true); row.setField(2, "str"); - final Row resultRow = serializeAndDeserialize(rowSchema, row); - assertEquals(row, resultRow); + final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) + .build(); + final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(rowSchema) + .build(); + + assertThat(row, whenSerializedWith(serializationSchema) + .andDeserializedWith(deserializationSchema) + .equalsTo(row)); } @Test @@ -63,8 +72,10 @@ public void testSerializationOfTwoRows() throws IOException { row1.setField(1, true); row1.setField(2, "str"); - final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema); - final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema); + final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) + .build(); + final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(rowSchema) + .build(); byte[] bytes = serializationSchema.serialize(row1); assertEquals(row1, deserializationSchema.deserialize(bytes)); @@ -79,7 +90,7 @@ public void testSerializationOfTwoRows() throws IOException { } @Test - public void testNestedSchema() throws IOException { + public void testNestedSchema() { final TypeInformation rowSchema = Types.ROW_NAMED( new String[] {"f1", "f2", "f3"}, Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.DOUBLE)); @@ -92,25 +103,32 @@ public void testNestedSchema() throws IOException { nested.setField(1, 2.3); row.setField(2, nested); - final Row resultRow = serializeAndDeserialize(rowSchema, row); - assertEquals(row, resultRow); + final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) + .build(); + final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(rowSchema) + .build(); + + assertThat(row, whenSerializedWith(serializationSchema) + .andDeserializedWith(deserializationSchema) + .equalsTo(row)); } - @Test(expected = RuntimeException.class) + @Test public void testSerializeRowWithInvalidNumberOfFields() { final TypeInformation rowSchema = Types.ROW_NAMED( - new String[] {"f1", "f2", "f3"}, + new String[]{"f1", "f2", "f3"}, Types.INT, Types.BOOLEAN, Types.STRING); final Row row = new Row(1); row.setField(0, 1); - final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema); - serializationSchema.serialize(row); + final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) + .build(); + assertThat(row, whenSerializedWith(serializationSchema).failsWithException(instanceOf(RuntimeException.class))); } @Test - public void testSchema() throws IOException { + public void testSchema() { final TypeInformation rowSchema = JsonRowSchemaConverter.convert( "{" + " type: 'object'," + @@ -157,17 +175,14 @@ public void testSchema() throws IOException { nestedRow.setField(1, BigDecimal.valueOf(12)); row.setField(10, nestedRow); - final Row resultRow = serializeAndDeserialize(rowSchema, row); - assertEquals(row, resultRow); - } - - // -------------------------------------------------------------------------------------------- + final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) + .build(); + final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(rowSchema) + .build(); - private Row serializeAndDeserialize(TypeInformation rowSchema, Row row) throws IOException { - final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema); - final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema); - - final byte[] bytes = serializationSchema.serialize(row); - return deserializationSchema.deserialize(bytes); + assertThat(row, whenSerializedWith(serializationSchema) + .andDeserializedWith(deserializationSchema) + .equalsTo(row)); } + } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java new file mode 100644 index 000000000..c5b19e7f0 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.utils; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.types.Row; + +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; + +import java.io.IOException; +import java.util.Objects; + +import static org.apache.flink.util.InstantiationUtil.deserializeObject; +import static org.apache.flink.util.InstantiationUtil.serializeObject; + +/** + * Matcher that provides a common way for asserting results of {@link DeserializationSchema}. It takes into account + * e.g. the fact that serialization schema during runtime might be used after serializing it over a wire. Usage: + * + *

    + *
  • when asserting for result after deserializing a row + *
    {@code
    + *      byte[] jsonBytes = ...
    + *      Row expectedRow = ...
    + *      final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema);
    + *
    + *      assertThat(jsonBytes, whenDeserializedWith(deserializationSchema)
    + *          .matches(expectedRow));
    + * }
    + *
  • + * + *
  • to check if an exception is thrown during serialization: + *
    {@code
    + *      assertThat(serializedJson,
    + *          whenDeserializedWith(deserializationSchema)
    + *              .failsWithException(hasCause(instanceOf(IllegalStateException.class))));
    + * }
    + *
  • + *
+ */ +public abstract class DeserializationSchemaMatcher extends TypeSafeMatcher { + + final DeserializationSchema deserializationSchema; + + private DeserializationSchemaMatcher(DeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + } + + public static DeserializationSchemaMatcherBuilder whenDeserializedWith(DeserializationSchema deserializationSchema) { + return new DeserializationSchemaMatcherBuilder(deserializationSchema); + } + + private static class DeserializationSchemaResultMatcher extends DeserializationSchemaMatcher { + + private final Row expected; + + private DeserializationSchemaResultMatcher( + DeserializationSchema deserializationSchema, + Row expected) { + super(deserializationSchema); + + this.expected = expected; + } + + @Override + protected boolean matchesSafely(byte[] item) { + try { + return Objects.deepEquals(deserializationSchema.deserialize(item), expected); + } catch (IOException e) { + throw new AssertionError("Could not deserialize", e); + } + } + + @Override + public void describeTo(Description description) { + description.appendValue(expected); + } + } + + private static class DeserializationSchemaExceptionMatcher extends DeserializationSchemaMatcher { + + private final Matcher exceptionMatcher; + private Throwable thrownException = null; + + private DeserializationSchemaExceptionMatcher( + DeserializationSchema deserializationSchema, + Matcher exceptionMatcher) { + super(deserializationSchema); + this.exceptionMatcher = exceptionMatcher; + } + + @Override + protected boolean matchesSafely(byte[] item) { + try { + deserializationSchema.deserialize(item); + } catch (IOException e) { + thrownException = e; + } + return exceptionMatcher.matches(thrownException); + } + + @Override + public void describeTo(Description description) { + exceptionMatcher.describeTo(description); + } + + @Override + protected void describeMismatchSafely(byte[] item, Description mismatchDescription) { + exceptionMatcher.describeMismatch(thrownException, mismatchDescription); + } + } + + /** + * Builder for {@link DeserializationSchemaMatcher}. + */ + public static class DeserializationSchemaMatcherBuilder { + + private DeserializationSchema deserializationSchema; + + private DeserializationSchemaMatcherBuilder(DeserializationSchema deserializationSchema) { + try { + // we serialize and deserialize the schema to test runtime behavior + // when the schema is shipped to the cluster + this.deserializationSchema = deserializeObject( + serializeObject(deserializationSchema), + this.getClass().getClassLoader()); + } catch (IOException | ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + public DeserializationSchemaMatcher equalsTo(Row row) { + return new DeserializationSchemaResultMatcher( + deserializationSchema, + row + ); + } + + public DeserializationSchemaMatcher failsWithException(Matcher exceptionMatcher) { + return new DeserializationSchemaExceptionMatcher( + deserializationSchema, + exceptionMatcher + ); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java new file mode 100644 index 000000000..4cda5dd37 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.utils; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.types.Row; + +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; + +import java.io.IOException; +import java.util.Objects; + +import static org.apache.flink.util.InstantiationUtil.deserializeObject; +import static org.apache.flink.util.InstantiationUtil.serializeObject; + +/** + * Matcher that provides a common way for asserting results of {@link SerializationSchema}. It takes into account + * e.g. the fact that serialization schema during runtime might be used after serializing and deserializing it over + * a wire. Usage: + * + *
    + *
  • when asserting for result after serializing and deserializing a row + *
    {@code
    + *      final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema);
    + *      final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema);
    + *
    + *      assertThat(row, whenSerializedWith(serializationSchema)
    + *          .andDeserializedWith(deserializationSchema)
    + *          .matches(row));
    + * }
    + *
  • + * + *
  • to check if an exception is thrown during serialization: + *
    {@code
    + *      assertThat(row, whenSerializedWith(serializationSchema).failsWithException(instanceOf(RuntimeException.class)));
    + * }
    + *
  • + *
+ */ +public abstract class SerializationSchemaMatcher extends TypeSafeMatcher { + + final SerializationSchema serializationSchema; + + private SerializationSchemaMatcher(SerializationSchema serializationSchema) { + this.serializationSchema = serializationSchema; + } + + public static SerializationSchemaMatcherBuilder whenSerializedWith(SerializationSchema serializationSchema) { + return new SerializationSchemaMatcherBuilder(serializationSchema); + } + + private static class SerializationSchemaResultMatcher extends SerializationSchemaMatcher { + + private final Row expected; + private final DeserializationSchema deserializationSchema; + + private SerializationSchemaResultMatcher( + SerializationSchema serializationSchema, + DeserializationSchema deserializationSchema, + Row expected) { + super(serializationSchema); + + this.expected = expected; + this.deserializationSchema = deserializationSchema; + } + + @Override + protected boolean matchesSafely(Row item) { + try { + return Objects.deepEquals( + deserializationSchema.deserialize(serializationSchema.serialize(item)), + expected); + } catch (IOException e) { + throw new AssertionError("Could not deserialize", e); + } + } + + @Override + public void describeTo(Description description) { + description.appendValue(expected); + } + } + + private static class SerializationSchemaExceptionMatcher extends SerializationSchemaMatcher { + + private final Matcher exceptionMatcher; + private Throwable thrownException = null; + + private SerializationSchemaExceptionMatcher( + SerializationSchema serializationSchema, + Matcher exceptionMatcher) { + super(serializationSchema); + this.exceptionMatcher = exceptionMatcher; + } + + @Override + protected boolean matchesSafely(Row item) { + try { + serializationSchema.serialize(item); + } catch (Exception e) { + thrownException = e; + } + return exceptionMatcher.matches(thrownException); + } + + @Override + public void describeTo(Description description) { + exceptionMatcher.describeTo(description); + } + + @Override + protected void describeMismatchSafely(Row item, Description mismatchDescription) { + exceptionMatcher.describeMismatch(thrownException, mismatchDescription); + } + } + + /** + * Builder for {@link SerializationSchemaMatcher} that can assert results after serialize and deserialize. + */ + public static class SerializationWithDeserializationSchemaMatcherBuilder { + + private SerializationSchema serializationSchema; + private DeserializationSchema deserializationSchema; + + private SerializationWithDeserializationSchemaMatcherBuilder( + SerializationSchema serializationSchema, + DeserializationSchema deserializationSchema) { + try { + // we serialize and deserialize the schema to test runtime behavior + // when the schema is shipped to the cluster + this.serializationSchema = deserializeObject( + serializeObject(serializationSchema), + this.getClass().getClassLoader()); + this.deserializationSchema = deserializeObject( + serializeObject(deserializationSchema), + this.getClass().getClassLoader()); + } catch (IOException | ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + public SerializationSchemaMatcher equalsTo(Row expected) { + return new SerializationSchemaResultMatcher( + serializationSchema, + deserializationSchema, + expected + ); + } + } + + /** + * Builder for {@link SerializationSchemaMatcher}. + */ + public static class SerializationSchemaMatcherBuilder { + + private SerializationSchema serializationSchema; + + private SerializationSchemaMatcherBuilder(SerializationSchema serializationSchema) { + this.serializationSchema = serializationSchema; + } + + public SerializationWithDeserializationSchemaMatcherBuilder andDeserializedWith(DeserializationSchema deserializationSchema) { + return new SerializationWithDeserializationSchemaMatcherBuilder(serializationSchema, deserializationSchema); + } + + public SerializationSchemaMatcher failsWithException(Matcher exceptionMatcher) { + return new SerializationSchemaExceptionMatcher( + serializationSchema, + exceptionMatcher + ); + } + } +} From 51dd12da662a585eabd61edd3e36848996412f2d Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 21 Aug 2019 16:38:00 +0800 Subject: [PATCH 049/322] [FLINK-13794][client] Remove unused logic of printStatusDuringExecution --- .../flink/schema/registry/test/TestAvroConsumerConfluent.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java index fbbe3c353..c24b0f996 100644 --- a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java +++ b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java @@ -59,7 +59,6 @@ public static void main(String[] args) throws Exception { String schemaRegistryUrl = parameterTool.getRequired("schema-registry-url"); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().disableSysoutLogging(); DataStreamSource input = env .addSource( From 477eb06dec82eb7590269d9f2e0eb83aa7a4cb6a Mon Sep 17 00:00:00 2001 From: Benchao Li Date: Wed, 6 Nov 2019 08:49:19 +0800 Subject: [PATCH 050/322] [FLINK-14546][formats] Support map type in JSON format This closes #10060 --- .../json/JsonRowDeserializationSchema.java | 25 +++++++++++++++++++ .../JsonRowDeserializationSchemaTest.java | 20 ++++++++++++--- 2 files changed, 42 insertions(+), 3 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index 4a1ff274b..49fb58850 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.MapTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.types.Row; @@ -34,6 +35,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; import java.io.IOException; import java.io.Serializable; @@ -47,7 +49,10 @@ import java.time.temporal.TemporalAccessor; import java.time.temporal.TemporalQueries; import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; @@ -242,11 +247,31 @@ private Optional createContainerConverter(TypeI return Optional.of(createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo())); } else if (isPrimitiveByteArray(typeInfo)) { return Optional.of(createByteArrayConverter()); + } else if (typeInfo instanceof MapTypeInfo) { + MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo; + return Optional.of(createMapConverter(mapTypeInfo.getKeyTypeInfo(), mapTypeInfo.getValueTypeInfo())); } else { return Optional.empty(); } } + private DeserializationRuntimeConverter createMapConverter(TypeInformation keyType, TypeInformation valueType) { + DeserializationRuntimeConverter valueConverter = createConverter(valueType); + DeserializationRuntimeConverter keyConverter = createConverter(keyType); + + return (mapper, jsonNode) -> { + Iterator> fields = jsonNode.fields(); + Map result = new HashMap<>(); + while (fields.hasNext()) { + Map.Entry entry = fields.next(); + Object key = keyConverter.convert(mapper, TextNode.valueOf(entry.getKey())); + Object value = valueConverter.convert(mapper, entry.getValue()); + result.put(key, value); + } + return result; + }; + } + private DeserializationRuntimeConverter createByteArrayConverter() { return (mapper, jsonNode) -> { try { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index 057301909..8314b6381 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -32,6 +32,8 @@ import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import static org.apache.flink.formats.utils.DeserializationSchemaMatcher.whenDeserializedWith; @@ -53,6 +55,13 @@ public void testTypeInfoDeserialization() throws Exception { String name = "asdlkjasjkdla998y1122"; byte[] bytes = new byte[1024]; ThreadLocalRandom.current().nextBytes(bytes); + Map map = new HashMap<>(); + map.put("flink", 123L); + + Map> nestedMap = new HashMap<>(); + Map innerMap = new HashMap<>(); + innerMap.put("key", 234); + nestedMap.put("inner_map", innerMap); ObjectMapper objectMapper = new ObjectMapper(); @@ -61,19 +70,24 @@ public void testTypeInfoDeserialization() throws Exception { root.put("id", id); root.put("name", name); root.put("bytes", bytes); + root.putObject("map").put("flink", 123); + root.putObject("map2map").putObject("inner_map").put("key", 234); byte[] serializedJson = objectMapper.writeValueAsBytes(root); JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder( Types.ROW_NAMED( - new String[]{"id", "name", "bytes"}, - Types.LONG, Types.STRING, Types.PRIMITIVE_ARRAY(Types.BYTE)) + new String[]{"id", "name", "bytes", "map", "map2map"}, + Types.LONG, Types.STRING, Types.PRIMITIVE_ARRAY(Types.BYTE), Types.MAP(Types.STRING, Types.LONG), + Types.MAP(Types.STRING, Types.MAP(Types.STRING, Types.INT))) ).build(); - Row row = new Row(3); + Row row = new Row(5); row.setField(0, id); row.setField(1, name); row.setField(2, bytes); + row.setField(3, map); + row.setField(4, nestedMap); assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); } From 32e62b6fa74bf56ce09e1d7af30fdb1721a8460b Mon Sep 17 00:00:00 2001 From: Jiayi Liao Date: Thu, 7 Nov 2019 13:28:12 +0800 Subject: [PATCH 051/322] [FLINK-14608][formats] Remove java stream from JsonRowDeserializationSchema This closes #10107 --- .../json/JsonRowDeserializationSchema.java | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index 49fb58850..b38934fca 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -39,6 +39,7 @@ import java.io.IOException; import java.io.Serializable; +import java.lang.reflect.Array; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; @@ -59,8 +60,6 @@ import static java.lang.String.format; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static java.util.Spliterators.spliterator; -import static java.util.stream.StreamSupport.stream; import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; import static org.apache.flink.util.Preconditions.checkArgument; @@ -289,7 +288,7 @@ private boolean isPrimitiveByteArray(TypeInformation typeInfo) { private DeserializationRuntimeConverter createObjectArrayConverter(TypeInformation elementTypeInfo) { DeserializationRuntimeConverter elementConverter = createConverter(elementTypeInfo); - return assembleArrayConverter(elementConverter); + return assembleArrayConverter(elementTypeInfo, elementConverter); } private DeserializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) { @@ -429,13 +428,21 @@ private Object convertField( } } - private DeserializationRuntimeConverter assembleArrayConverter(DeserializationRuntimeConverter elementConverter) { + private DeserializationRuntimeConverter assembleArrayConverter( + TypeInformation elementType, + DeserializationRuntimeConverter elementConverter) { + + final Class elementClass = elementType.getTypeClass(); + return (mapper, jsonNode) -> { - ArrayNode node = (ArrayNode) jsonNode; + final ArrayNode node = (ArrayNode) jsonNode; + final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); + for (int i = 0; i < node.size(); i++) { + final JsonNode innerNode = node.get(i); + array[i] = elementConverter.convert(mapper, innerNode); + } - return stream(spliterator(node.elements(), node.size(), 0), false) - .map(innerNode -> elementConverter.convert(mapper, innerNode)) - .toArray(); + return array; }; } } From 1f43fd3a632915dabbbb1c0780c8128775168ace Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 20 Nov 2019 10:35:03 +0100 Subject: [PATCH 052/322] [FLINK-14104][build] Pin jackson version to 2.10.1 --- .../src/main/resources/META-INF/NOTICE | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 8a6da1e04..26541c32e 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -7,9 +7,9 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - com.101tec:zkclient:0.10 -- com.fasterxml.jackson.core:jackson-databind:2.8.4 -- com.fasterxml.jackson.core:jackson-annotations:2.8.0 -- com.fasterxml.jackson.core:jackson-core:2.8.4 +- com.fasterxml.jackson.core:jackson-databind:2.10.1 +- com.fasterxml.jackson.core:jackson-annotations:2.10.1 +- com.fasterxml.jackson.core:jackson-core:2.10.1 - io.confluent:common-utils:3.3.1 - io.confluent:kafka-schema-registry-client:3.3.1 - org.apache.zookeeper:zookeeper:3.4.10 From b8c87f3109ee173e1021b15be09cf57aacbc58e2 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 3 Dec 2019 20:51:06 +0800 Subject: [PATCH 053/322] [FLINK-14645][json] Support LocalTimestamp/LocalDate/LocalTime for JSON format --- .../json/JsonRowDeserializationSchema.java | 84 +++++++++++-------- .../json/JsonRowSerializationSchema.java | 52 +++++++----- .../JsonRowDeserializationSchemaTest.java | 30 +++++-- .../json/JsonRowSerializationSchemaTest.java | 8 +- 4 files changed, 110 insertions(+), 64 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index b38934fca..4cdd53807 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -335,61 +335,73 @@ private Optional createConverterForSimpleType(T } else if (simpleTypeInfo == Types.BIG_INT) { return Optional.of((mapper, jsonNode) -> jsonNode.bigIntegerValue()); } else if (simpleTypeInfo == Types.SQL_DATE) { - return Optional.of(createDateConverter()); + return Optional.of(this::convertToDate); } else if (simpleTypeInfo == Types.SQL_TIME) { - return Optional.of(createTimeConverter()); + return Optional.of(this::convertToTime); } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) { - return Optional.of(createTimestampConverter()); + return Optional.of(this::convertToTimestamp); + } else if (simpleTypeInfo == Types.LOCAL_DATE) { + return Optional.of(this::convertToLocalDate); + } else if (simpleTypeInfo == Types.LOCAL_TIME) { + return Optional.of(this::convertToLocalTime); + } else if (simpleTypeInfo == Types.LOCAL_DATE_TIME) { + return Optional.of(this::convertToLocalDateTime); } else { return Optional.empty(); } } - private DeserializationRuntimeConverter createDateConverter() { - return (mapper, jsonNode) -> Date.valueOf(ISO_LOCAL_DATE.parse(jsonNode.asText()) - .query(TemporalQueries.localDate())); + private LocalDate convertToLocalDate(ObjectMapper mapper, JsonNode jsonNode) { + return ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); } - private DeserializationRuntimeConverter createTimestampConverter() { - return (mapper, jsonNode) -> { - // according to RFC 3339 every date-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - TemporalAccessor parsedTimestamp = RFC3339_TIMESTAMP_FORMAT.parse(jsonNode.asText()); + private Date convertToDate(ObjectMapper mapper, JsonNode jsonNode) { + return Date.valueOf(convertToLocalDate(mapper, jsonNode)); + } - ZoneOffset zoneOffset = parsedTimestamp.query(TemporalQueries.offset()); + private LocalDateTime convertToLocalDateTime(ObjectMapper mapper, JsonNode jsonNode) { + // according to RFC 3339 every date-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + TemporalAccessor parsedTimestamp = RFC3339_TIMESTAMP_FORMAT.parse(jsonNode.asText()); - if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0) { - throw new IllegalStateException( - "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + - "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); - } + ZoneOffset zoneOffset = parsedTimestamp.query(TemporalQueries.offset()); - LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); - LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); + if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0) { + throw new IllegalStateException( + "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + + "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + } - return Timestamp.valueOf(LocalDateTime.of(localDate, localTime)); - }; + LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); + LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); + + return LocalDateTime.of(localDate, localTime); } - private DeserializationRuntimeConverter createTimeConverter() { - return (mapper, jsonNode) -> { + private Timestamp convertToTimestamp(ObjectMapper mapper, JsonNode jsonNode) { + return Timestamp.valueOf(convertToLocalDateTime(mapper, jsonNode)); + } - // according to RFC 3339 every full-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText()); + private LocalTime convertToLocalTime(ObjectMapper mapper, JsonNode jsonNode) { + // according to RFC 3339 every full-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText()); - ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset()); - LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); + ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset()); + LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); - if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0 || localTime.getNano() != 0) { - throw new IllegalStateException( - "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet."); - } + if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0 || localTime.getNano() != 0) { + throw new IllegalStateException( + "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet."); + } - return Time.valueOf(localTime); - }; + return localTime; + } + + private Time convertToTime(ObjectMapper mapper, JsonNode jsonNode) { + return Time.valueOf(convertToLocalTime(mapper, jsonNode)); } private DeserializationRuntimeConverter assembleRowConverter( diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index 0aa7151fc..5e89b42a0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -41,6 +41,9 @@ import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; import java.util.Arrays; import java.util.List; import java.util.Objects; @@ -255,40 +258,49 @@ private Optional createConverterForSimpleType(Typ } else if (simpleTypeInfo == Types.BIG_INT) { return Optional.of(createBigIntegerConverter()); } else if (simpleTypeInfo == Types.SQL_DATE) { - return Optional.of(createDateConverter()); + return Optional.of(this::convertDate); } else if (simpleTypeInfo == Types.SQL_TIME) { - return Optional.of(createTimeConverter()); + return Optional.of(this::convertTime); } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) { - return Optional.of(createTimestampConverter()); + return Optional.of(this::convertTimestamp); + } else if (simpleTypeInfo == Types.LOCAL_DATE) { + return Optional.of(this::convertLocalDate); + } else if (simpleTypeInfo == Types.LOCAL_TIME) { + return Optional.of(this::convertLocalTime); + } else if (simpleTypeInfo == Types.LOCAL_DATE_TIME) { + return Optional.of(this::convertLocalDateTime); } else { return Optional.empty(); } } - private SerializationRuntimeConverter createDateConverter() { - return (mapper, reuse, object) -> { - Date date = (Date) object; + private JsonNode convertLocalDate(ObjectMapper mapper, JsonNode reuse, Object object) { + return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format((LocalDate) object)); + } - return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format(date.toLocalDate())); - }; + private JsonNode convertDate(ObjectMapper mapper, JsonNode reuse, Object object) { + Date date = (Date) object; + return convertLocalDate(mapper, reuse, date.toLocalDate()); } - private SerializationRuntimeConverter createTimestampConverter() { - return (mapper, reuse, object) -> { - Timestamp timestamp = (Timestamp) object; + private JsonNode convertLocalDateTime(ObjectMapper mapper, JsonNode reuse, Object object) { + return mapper.getNodeFactory() + .textNode(RFC3339_TIMESTAMP_FORMAT.format((LocalDateTime) object)); + } - return mapper.getNodeFactory() - .textNode(RFC3339_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); - }; + private JsonNode convertTimestamp(ObjectMapper mapper, JsonNode reuse, Object object) { + Timestamp timestamp = (Timestamp) object; + return convertLocalDateTime(mapper, reuse, timestamp.toLocalDateTime()); } - private SerializationRuntimeConverter createTimeConverter() { - return (mapper, reuse, object) -> { - final Time time = (Time) object; + private JsonNode convertLocalTime(ObjectMapper mapper, JsonNode reuse, Object object) { + JsonNodeFactory nodeFactory = mapper.getNodeFactory(); + return nodeFactory.textNode(RFC3339_TIME_FORMAT.format((LocalTime) object)); + } - JsonNodeFactory nodeFactory = mapper.getNodeFactory(); - return nodeFactory.textNode(RFC3339_TIME_FORMAT.format(time.toLocalTime())); - }; + private JsonNode convertTime(ObjectMapper mapper, JsonNode reuse, Object object) { + final Time time = (Time) object; + return convertLocalTime(mapper, reuse, time.toLocalTime()); } private SerializationRuntimeConverter createBigDecimalConverter() { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index 8314b6381..717898770 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -55,6 +55,10 @@ public void testTypeInfoDeserialization() throws Exception { String name = "asdlkjasjkdla998y1122"; byte[] bytes = new byte[1024]; ThreadLocalRandom.current().nextBytes(bytes); + Timestamp timestamp = Timestamp.valueOf("1990-10-14 12:12:43"); + Date date = Date.valueOf("1990-10-14"); + Time time = Time.valueOf("12:12:43"); + Map map = new HashMap<>(); map.put("flink", 123L); @@ -70,6 +74,12 @@ public void testTypeInfoDeserialization() throws Exception { root.put("id", id); root.put("name", name); root.put("bytes", bytes); + root.put("date1", "1990-10-14"); + root.put("date2", "1990-10-14"); + root.put("time1", "12:12:43Z"); + root.put("time2", "12:12:43Z"); + root.put("timestamp1", "1990-10-14T12:12:43Z"); + root.put("timestamp2", "1990-10-14T12:12:43Z"); root.putObject("map").put("flink", 123); root.putObject("map2map").putObject("inner_map").put("key", 234); @@ -77,17 +87,27 @@ public void testTypeInfoDeserialization() throws Exception { JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder( Types.ROW_NAMED( - new String[]{"id", "name", "bytes", "map", "map2map"}, - Types.LONG, Types.STRING, Types.PRIMITIVE_ARRAY(Types.BYTE), Types.MAP(Types.STRING, Types.LONG), + new String[]{"id", "name", "bytes", "date1", "date2", + "time1", "time2", "timestamp1", "timestamp2", "map", "map2map"}, + Types.LONG, Types.STRING, Types.PRIMITIVE_ARRAY(Types.BYTE), + Types.SQL_DATE, Types.LOCAL_DATE, Types.SQL_TIME, Types.LOCAL_TIME, + Types.SQL_TIMESTAMP, Types.LOCAL_DATE_TIME, + Types.MAP(Types.STRING, Types.LONG), Types.MAP(Types.STRING, Types.MAP(Types.STRING, Types.INT))) ).build(); - Row row = new Row(5); + Row row = new Row(11); row.setField(0, id); row.setField(1, name); row.setField(2, bytes); - row.setField(3, map); - row.setField(4, nestedMap); + row.setField(3, date); + row.setField(4, date.toLocalDate()); + row.setField(5, time); + row.setField(6, time.toLocalTime()); + row.setField(7, timestamp); + row.setField(8, timestamp.toLocalDateTime()); + row.setField(9, map); + row.setField(10, nestedMap); assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java index cc1f5bf37..4a0706fe2 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java @@ -43,13 +43,15 @@ public class JsonRowSerializationSchemaTest { @Test public void testRowSerialization() { final TypeInformation rowSchema = Types.ROW_NAMED( - new String[] {"f1", "f2", "f3"}, - Types.INT, Types.BOOLEAN, Types.STRING); + new String[] {"f1", "f2", "f3", "f4", "f5"}, + Types.INT, Types.BOOLEAN, Types.STRING, Types.SQL_TIMESTAMP, Types.LOCAL_DATE_TIME); - final Row row = new Row(3); + final Row row = new Row(5); row.setField(0, 1); row.setField(1, true); row.setField(2, "str"); + row.setField(3, Timestamp.valueOf("1990-10-14 12:12:43")); + row.setField(4, Timestamp.valueOf("1990-10-14 12:12:43").toLocalDateTime()); final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) .build(); From 737cd9ad2e84764172157e8373af50a842c9c9a5 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Sun, 8 Dec 2019 00:10:32 +0800 Subject: [PATCH 054/322] [FLINK-14824][csv][json] Make derivation schema as default option for CSV and JSON formats --- .../org/apache/flink/table/descriptors/Json.java | 4 ++++ .../flink/table/descriptors/JsonValidator.java | 11 +++++------ .../formats/json/JsonRowFormatFactoryTest.java | 14 ++++++++++++++ .../apache/flink/table/descriptors/JsonTest.java | 16 +++++++++------- 4 files changed, 32 insertions(+), 13 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java index 7bc7cfbf0..6abaa3f54 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -100,7 +100,11 @@ public Json schema(TypeInformation schemaType) { *

The names, types, and fields' order of the format are determined by the table's * schema. Time attributes are ignored if their origin is not a field. A "from" definition * is interpreted as a field renaming in the format. + * + * @deprecated Derivation format schema from table's schema is the default behavior now. + * So there is no need to explicitly declare to derive schema. */ + @Deprecated public Json deriveSchema() { this.deriveSchema = true; this.schema = null; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java index a520811d1..061d7fd16 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java @@ -36,16 +36,15 @@ public class JsonValidator extends FormatDescriptorValidator { public void validate(DescriptorProperties properties) { super.validate(properties); properties.validateBoolean(FORMAT_DERIVE_SCHEMA, true); - final boolean deriveSchema = properties.getOptionalBoolean(FORMAT_DERIVE_SCHEMA).orElse(false); + final boolean deriveSchema = properties.getOptionalBoolean(FORMAT_DERIVE_SCHEMA).orElse(true); final boolean hasSchema = properties.containsKey(FORMAT_SCHEMA); final boolean hasSchemaString = properties.containsKey(FORMAT_JSON_SCHEMA); - if (deriveSchema && (hasSchema || hasSchemaString)) { - throw new ValidationException( - "Format cannot define a schema and derive from the table's schema at the same time."); - } else if (!deriveSchema && hasSchema && hasSchemaString) { + // if a schema is defined, no matter derive schema is set or not, will use the defined schema + if (!deriveSchema && hasSchema && hasSchemaString) { throw new ValidationException("A definition of both a schema and JSON schema is not allowed."); } else if (!deriveSchema && !hasSchema && !hasSchemaString) { - throw new ValidationException("A definition of a schema or JSON schema is required."); + throw new ValidationException("A definition of a schema or JSON schema is required " + + "if derivation from table's schema is disabled."); } else if (hasSchema) { properties.validateType(FORMAT_SCHEMA, false, true); } else if (hasSchemaString) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java index 47b06c49b..3951c1586 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -106,6 +106,20 @@ public void testSchemaDerivation() { testSchemaDeserializationSchema(properties); } + @Test + public void testSchemaDerivationByDefault() { + final Map properties = toMap( + new Schema() + .field("field1", Types.BOOLEAN()) + .field("field2", Types.INT()) + .field("proctime", Types.SQL_TIMESTAMP()).proctime(), + new Json()); + + testSchemaSerializationSchema(properties); + + testSchemaDeserializationSchema(properties); + } + private void testSchemaDeserializationSchema(Map properties) { final DeserializationSchema actual2 = TableFactoryService .find(DeserializationSchemaFactory.class, properties) diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java index ac6ff11c3..2bc58987d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java @@ -59,11 +59,6 @@ public void testInvalidMissingField() { addPropertyAndVerify(descriptors().get(0), "format.fail-on-missing-field", "DDD"); } - @Test(expected = ValidationException.class) - public void testMissingSchema() { - removePropertyAndVerify(descriptors().get(0), "format.json-schema"); - } - @Test(expected = ValidationException.class) public void testDuplicateSchema() { // we add an additional non-json schema @@ -87,7 +82,9 @@ public List descriptors() { final Descriptor desc4 = new Json().deriveSchema(); - return Arrays.asList(desc1, desc2, desc3, desc4); + final Descriptor desc5 = new Json().failOnMissingField(false); + + return Arrays.asList(desc1, desc2, desc3, desc4, desc5); } @Override @@ -114,7 +111,12 @@ public List> properties() { props4.put("format.property-version", "1"); props4.put("format.derive-schema", "true"); - return Arrays.asList(props1, props2, props3, props4); + final Map props5 = new HashMap<>(); + props5.put("format.type", "json"); + props5.put("format.property-version", "1"); + props5.put("format.fail-on-missing-field", "false"); + + return Arrays.asList(props1, props2, props3, props4, props5); } @Override From 8c40806010d9f0d196b856c9a3d24f69ec2f45e8 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 9 Dec 2019 17:28:45 +0100 Subject: [PATCH 055/322] Update version to 1.11-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 85fd21d82..6e96bff0c 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.10-SNAPSHOT + 1.11-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 5366cbd57..3bdda1d20 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.10-SNAPSHOT + 1.11-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index eb9e3472f..3bc69aa83 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.10-SNAPSHOT + 1.11-SNAPSHOT .. From 080c5fe6bbf2e5a6cfc3524d5784fc965726aff7 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 31 Dec 2019 17:43:13 +0800 Subject: [PATCH 056/322] [hotfix][csv][json] Deprecate schema methods for CSV and JSON formats Deprecate schema methods for CSV and JSON formats because these formats can derive schema from table schema by default. It is no longer necessary to explicitly declare format schema. --- .../java/org/apache/flink/table/descriptors/Json.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java index 6abaa3f54..0f4df09ed 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -66,7 +66,11 @@ public Json failOnMissingField(boolean failOnMissingField) { *

The schema might be nested. * * @param jsonSchema JSON schema + * @deprecated {@link Json} supports derive schema from table schema by default, + * it is no longer necessary to explicitly declare the format schema. + * This method will be removed in the future. */ + @Deprecated public Json jsonSchema(String jsonSchema) { Preconditions.checkNotNull(jsonSchema); this.jsonSchema = jsonSchema; @@ -83,7 +87,11 @@ public Json jsonSchema(String jsonSchema) { *

The schema might be nested. * * @param schemaType type information that describes the schema + * @deprecated {@link Json} supports derive schema from table schema by default, + * it is no longer necessary to explicitly declare the format schema. + * This method will be removed in the future. */ + @Deprecated public Json schema(TypeInformation schemaType) { Preconditions.checkNotNull(schemaType); this.schema = TypeStringUtils.writeTypeInfo(schemaType); From bce75700009523118004d1b881deb91b4fc5c71b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dominik=20Wosi=C5=84ski?= Date: Fri, 2 Aug 2019 00:19:05 +0200 Subject: [PATCH 057/322] [FLINK-9679] Add Schema Registry version as parameter + update version --- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 3bdda1d20..289d10003 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -29,6 +29,10 @@ under the License. flink-avro-confluent-registry + + 4.1.0 + + confluent @@ -40,7 +44,7 @@ under the License. io.confluent kafka-schema-registry-client - 3.3.1 + ${confluent.schema.registry.version} org.apache.avro From cbca0cf1c5e3bafd7ebd4348983570e534917ead Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dominik=20Wosi=C5=84ski?= Date: Fri, 2 Aug 2019 00:29:02 +0200 Subject: [PATCH 058/322] [FLINK-9679] Add (Confluent)(Registry)AvroSerializationSchema --- ...fluentRegistryAvroSerializationSchema.java | 116 ++++++++++++++++++ .../ConfluentSchemaRegistryCoder.java | 30 ++++- 2 files changed, 145 insertions(+), 1 deletion(-) create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java new file mode 100644 index 000000000..b5680b0d8 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent; + +import org.apache.flink.formats.avro.AvroSerializationSchema; +import org.apache.flink.formats.avro.RegistryAvroSerializationSchema; +import org.apache.flink.formats.avro.SchemaCoder; + +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.specific.SpecificRecord; + +/** + * Serialization schema that serializes to Avro binary format that uses + * Confluent Schema Registry. + * + * @param the type to be serialized + */ +public class ConfluentRegistryAvroSerializationSchema extends RegistryAvroSerializationSchema { + + private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; + + private static final long serialVersionUID = -1771641202177852775L; + + /** + * Creates a Avro serialization schema. + * + * @param recordClazz class to serialize. Should be either + * {@link SpecificRecord} or {@link GenericRecord}. + * @param schema writer's Avro schema. Should be provided if recordClazz is + * {@link GenericRecord} + * @param schemaCoderProvider provider for schema coder that writes the writer schema to Confluent Schema Registry + */ + private ConfluentRegistryAvroSerializationSchema(Class recordClazz, Schema schema, + SchemaCoder.SchemaCoderProvider schemaCoderProvider) { + super(recordClazz, schema, schemaCoderProvider); + } + + /** + * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from avro + * schema and writes the writer schema to Confluent Schema Registry. + * + * @param tClass the type to be serialized + * @param subject subject of schema registry to produce + * @param schemaRegistryUrl url of schema registry to connect + * + * @return Serialized record + */ + public static ConfluentRegistryAvroSerializationSchema forSpecific(Class tClass, + String subject, + String schemaRegistryUrl) { + return new ConfluentRegistryAvroSerializationSchema<>( + tClass, + null, + new CachedSchemaCoderProvider(subject, schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY) + ); + } + + /** + * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from avro + * schema and writes the writer schema to Confluent Schema Registry. + * + * @param subject subject of schema registry to produce + * @param schema schema that will be used for serialization + * @param schemaRegistryUrl url of schema registry to connect + * + * @return Serialized record in form of byte array + */ + public static ConfluentRegistryAvroSerializationSchema forGeneric(String subject, + Schema schema, + String schemaRegistryUrl) { + return new ConfluentRegistryAvroSerializationSchema<>( + GenericRecord.class, + schema, + new CachedSchemaCoderProvider(subject, schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY) + ); + } + + private static class CachedSchemaCoderProvider implements SchemaCoder.SchemaCoderProvider { + + private static final long serialVersionUID = 4023134423033312666L; + private final String subject; + private final String url; + private final int identityMapCapacity; + + CachedSchemaCoderProvider(String subject, String url, int identityMapCapacity) { + this.subject = subject; + this.url = url; + this.identityMapCapacity = identityMapCapacity; + } + + @Override + public SchemaCoder get() { + return new ConfluentSchemaRegistryCoder(subject, new CachedSchemaRegistryClient( + url, + identityMapCapacity)); + } + } +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java index 1f2dc69ca..36b973ea6 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java @@ -27,15 +27,31 @@ import java.io.DataInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; import static java.lang.String.format; /** - * Reads schema using Confluent Schema Registry protocol. + * Reads and Writes schema using Confluent Schema Registry protocol. */ public class ConfluentSchemaRegistryCoder implements SchemaCoder { private final SchemaRegistryClient schemaRegistryClient; + private String subject; + private static final int CONFLUENT_MAGIC_BYTE = 0; + + /** + * Creates {@link SchemaCoder} that uses provided {@link SchemaRegistryClient} to connect to + * schema registry. + * + * @param schemaRegistryClient client to connect schema registry + * @param subject subject of schema registry to produce + */ + public ConfluentSchemaRegistryCoder(String subject, SchemaRegistryClient schemaRegistryClient) { + this.schemaRegistryClient = schemaRegistryClient; + this.subject = subject; + } /** * Creates {@link SchemaCoder} that uses provided {@link SchemaRegistryClient} to connect to @@ -64,4 +80,16 @@ public Schema readSchema(InputStream in) throws IOException { } } + @Override + public void writeSchema(Schema schema, OutputStream out) throws IOException { + try { + int registeredId = schemaRegistryClient.register(subject, schema); + out.write(CONFLUENT_MAGIC_BYTE); + byte[] schemaIdBytes = ByteBuffer.allocate(4).putInt(registeredId).array(); + out.write(schemaIdBytes); + } catch (RestClientException e) { + throw new IOException("Could not register schema in registry", e); + } + } + } From 9d291d2331b2f817a355ee90960082514972079c Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 2 Jan 2020 14:25:20 +0100 Subject: [PATCH 059/322] [FLINK-9679] In Confluent Schema Registry e2e, also test write path --- .../registry/test/TestAvroConsumerConfluent.java | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java index c24b0f996..55549dee9 100644 --- a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java +++ b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema; +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -36,7 +37,7 @@ * A simple example that shows how to read from and write to Kafka with Confluent Schema Registry. * This will read AVRO messages from the input topic, parse them into a POJO type via checking the Schema by calling Schema registry. * Then this example publish the POJO type to kafka by converting the POJO to AVRO and verifying the schema. - * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --schema-registry-url http://localhost:8081 --group.id myconsumer + * --input-topic test-input --output-string-topic test-output --output-avro-topic test-avro-output --output-subject --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --schema-registry-url http://localhost:8081 --group.id myconsumer */ public class TestAvroConsumerConfluent { @@ -46,7 +47,7 @@ public static void main(String[] args) throws Exception { if (parameterTool.getNumberOfParameters() < 6) { System.out.println("Missing parameters!\n" + - "Usage: Kafka --input-topic --output-topic " + + "Usage: Kafka --input-topic --output-string-topic --output-avro-topic " + "--bootstrap.servers " + "--zookeeper.connect " + "--schema-registry-url --group.id "); @@ -71,11 +72,17 @@ public static void main(String[] args) throws Exception { .map((MapFunction) SpecificRecordBase::toString); FlinkKafkaProducer010 stringFlinkKafkaProducer010 = new FlinkKafkaProducer010<>( - parameterTool.getRequired("output-topic"), + parameterTool.getRequired("output-string-topic"), new SimpleStringSchema(), config); - mapToString.addSink(stringFlinkKafkaProducer010); + + FlinkKafkaProducer010 avroFlinkKafkaProducer010 = new FlinkKafkaProducer010<>( + parameterTool.getRequired("output-avro-topic"), + ConfluentRegistryAvroSerializationSchema.forSpecific(User.class, parameterTool.getRequired("output-subject"), schemaRegistryUrl), + config); + input.addSink(avroFlinkKafkaProducer010); + env.execute("Kafka 0.10 Confluent Schema Registry AVRO Example"); } } From 1c6636527aaf0274fd3a70e959f7ef0837285e16 Mon Sep 17 00:00:00 2001 From: Benchao Li Date: Tue, 18 Feb 2020 14:54:41 +0800 Subject: [PATCH 060/322] [FLINK-15988][json] Make JsonRowSerializationSchema's constructor private (#11080) --- .../json/JsonRowSerializationSchema.java | 43 ++++++++++++++++--- .../json/JsonRowFormatFactoryTest.java | 4 +- 2 files changed, 39 insertions(+), 8 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index 5e89b42a0..284739534 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -81,12 +81,7 @@ public class JsonRowSerializationSchema implements SerializationSchema { /** Reusable object node. */ private transient ObjectNode node; - /** - * @deprecated Use the provided {@link Builder} instead. - */ - @Deprecated - public JsonRowSerializationSchema(TypeInformation typeInfo) { - // TODO make this constructor private in the future + private JsonRowSerializationSchema(TypeInformation typeInfo) { Preconditions.checkNotNull(typeInfo, "Type information"); Preconditions.checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); this.typeInfo = (RowTypeInfo) typeInfo; @@ -99,14 +94,21 @@ public JsonRowSerializationSchema(TypeInformation typeInfo) { @PublicEvolving public static class Builder { - private final RowTypeInfo typeInfo; + private RowTypeInfo typeInfo; + + private Builder() { + // private constructor + } /** * Creates a JSON serialization schema for the given type information. * * @param typeInfo Type information describing the result type. The field names of {@link Row} * are used to parse the JSON properties. + * + * @deprecated Use {@link JsonRowSerializationSchema#builder()} instead. */ + @Deprecated public Builder(TypeInformation typeInfo) { checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); this.typeInfo = (RowTypeInfo) typeInfo; @@ -118,16 +120,43 @@ public Builder(TypeInformation typeInfo) { * @param jsonSchema JSON schema describing the result type * * @see http://json-schema.org/ + * + * @deprecated Use {@link JsonRowSerializationSchema#builder()} instead. */ + @Deprecated public Builder(String jsonSchema) { this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema))); } + /** + * Sets type information for JSON serialization schema. + * + * @param typeInfo Type information describing the result type. The field names of {@link Row} + * are used to parse the JSON properties. + */ + public Builder withTypeInfo(TypeInformation typeInfo) { + checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + this.typeInfo = (RowTypeInfo) typeInfo; + return this; + } + + /** + * Finalizes the configuration and checks validity. + * @return Configured {@link JsonRowSerializationSchema} + */ public JsonRowSerializationSchema build() { + checkArgument(typeInfo != null, "typeInfo should be set."); return new JsonRowSerializationSchema(typeInfo); } } + /** + * Creates a builder for {@link JsonRowSerializationSchema.Builder}. + */ + public static Builder builder() { + return new Builder(); + } + @Override public byte[] serialize(Row row) { if (node == null) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java index 3951c1586..6f94094e8 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -150,7 +150,9 @@ private void testJsonSchemaSerializationSchema(Map properties) { final SerializationSchema actual1 = TableFactoryService .find(SerializationSchemaFactory.class, properties) .createSerializationSchema(properties); - final SerializationSchema expected1 = new JsonRowSerializationSchema.Builder(JSON_SCHEMA).build(); + final SerializationSchema expected1 = JsonRowSerializationSchema.builder() + .withTypeInfo(JsonRowSchemaConverter.convert(JSON_SCHEMA)) + .build(); assertEquals(expected1, actual1); } From 908c28ae8ba5ed6314b6827cd2b14e8bdf475bb9 Mon Sep 17 00:00:00 2001 From: zoudan Date: Tue, 18 Feb 2020 16:32:28 +0800 Subject: [PATCH 061/322] [FLINK-15396][json] Support to ignore parse errors for JSON format This closes #11119 --- .../json/JsonRowDeserializationSchema.java | 134 +++++++++++-- .../formats/json/JsonRowFormatFactory.java | 7 +- .../apache/flink/table/descriptors/Json.java | 17 ++ .../table/descriptors/JsonValidator.java | 7 + .../JsonRowDeserializationSchemaTest.java | 177 ++++++++++++++++++ .../flink/table/descriptors/JsonTest.java | 24 ++- 6 files changed, 343 insertions(+), 23 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index 4cdd53807..cc9b55d58 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -28,7 +28,6 @@ import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.types.Row; -import org.apache.flink.util.WrappingRuntimeException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; @@ -40,6 +39,8 @@ import java.io.IOException; import java.io.Serializable; import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.math.BigInteger; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; @@ -88,14 +89,23 @@ public class JsonRowDeserializationSchema implements DeserializationSchema private DeserializationRuntimeConverter runtimeConverter; + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + private JsonRowDeserializationSchema( TypeInformation typeInfo, - boolean failOnMissingField) { + boolean failOnMissingField, + boolean ignoreParseErrors) { checkNotNull(typeInfo, "Type information"); checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + if (ignoreParseErrors && failOnMissingField) { + throw new IllegalArgumentException( + "JSON format doesn't support failOnMissingField and ignoreParseErrors are both true."); + } this.typeInfo = (RowTypeInfo) typeInfo; this.failOnMissingField = failOnMissingField; this.runtimeConverter = createConverter(this.typeInfo); + this.ignoreParseErrors = ignoreParseErrors; } /** @@ -103,7 +113,7 @@ private JsonRowDeserializationSchema( */ @Deprecated public JsonRowDeserializationSchema(TypeInformation typeInfo) { - this(typeInfo, false); + this(typeInfo, false, false); } /** @@ -111,7 +121,7 @@ public JsonRowDeserializationSchema(TypeInformation typeInfo) { */ @Deprecated public JsonRowDeserializationSchema(String jsonSchema) { - this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)), false); + this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)), false, false); } /** @@ -130,7 +140,10 @@ public Row deserialize(byte[] message) throws IOException { final JsonNode root = objectMapper.readTree(message); return (Row) runtimeConverter.convert(objectMapper, root); } catch (Throwable t) { - throw new IOException("Failed to deserialize JSON object.", t); + if (ignoreParseErrors) { + return null; + } + throw new IOException(format("Failed to deserialize JSON '%s'.", new String(message)), t); } } @@ -151,6 +164,7 @@ public static class Builder { private final RowTypeInfo typeInfo; private boolean failOnMissingField = false; + private boolean ignoreParseErrors = false; /** * Creates a JSON deserialization schema for the given type information. @@ -184,8 +198,18 @@ public Builder failOnMissingField() { return this; } + /** + * Configures schema to fail when parsing json failed. + * + *

By default, an exception will be thrown when parsing json fails. + */ + public Builder ignoreParseErrors() { + this.ignoreParseErrors = true; + return this; + } + public JsonRowDeserializationSchema build() { - return new JsonRowDeserializationSchema(typeInfo, failOnMissingField); + return new JsonRowDeserializationSchema(typeInfo, failOnMissingField, ignoreParseErrors); } } @@ -199,12 +223,13 @@ public boolean equals(Object o) { } final JsonRowDeserializationSchema that = (JsonRowDeserializationSchema) o; return Objects.equals(typeInfo, that.typeInfo) && - Objects.equals(failOnMissingField, that.failOnMissingField); + Objects.equals(failOnMissingField, that.failOnMissingField) && + Objects.equals(ignoreParseErrors, that.ignoreParseErrors); } @Override public int hashCode() { - return Objects.hash(typeInfo, failOnMissingField); + return Objects.hash(typeInfo, failOnMissingField, ignoreParseErrors); } /* @@ -232,8 +257,14 @@ private DeserializationRuntimeConverter wrapIntoNullableConverter(Deserializatio if (jsonNode.isNull()) { return null; } - - return converter.convert(mapper, jsonNode); + try { + return converter.convert(mapper, jsonNode); + } catch (Throwable t) { + if (!ignoreParseErrors) { + throw t; + } + return null; + } }; } @@ -257,7 +288,6 @@ private Optional createContainerConverter(TypeI private DeserializationRuntimeConverter createMapConverter(TypeInformation keyType, TypeInformation valueType) { DeserializationRuntimeConverter valueConverter = createConverter(valueType); DeserializationRuntimeConverter keyConverter = createConverter(keyType); - return (mapper, jsonNode) -> { Iterator> fields = jsonNode.fields(); Map result = new HashMap<>(); @@ -276,7 +306,7 @@ private DeserializationRuntimeConverter createByteArrayConverter() { try { return jsonNode.binaryValue(); } catch (IOException e) { - throw new WrappingRuntimeException("Unable to deserialize byte array.", e); + throw new JsonParseException("Unable to deserialize byte array.", e); } }; } @@ -306,7 +336,7 @@ private DeserializationRuntimeConverter createFallbackConverter(Class valueTy try { return mapper.treeToValue(jsonNode, valueType); } catch (JsonProcessingException e) { - throw new WrappingRuntimeException(format("Could not convert node: %s", jsonNode), e); + throw new JsonParseException(format("Could not convert node: %s", jsonNode), e); } }; } @@ -315,15 +345,15 @@ private Optional createConverterForSimpleType(T if (simpleTypeInfo == Types.VOID) { return Optional.of((mapper, jsonNode) -> null); } else if (simpleTypeInfo == Types.BOOLEAN) { - return Optional.of((mapper, jsonNode) -> jsonNode.asBoolean()); + return Optional.of(this::convertToBoolean); } else if (simpleTypeInfo == Types.STRING) { return Optional.of((mapper, jsonNode) -> jsonNode.asText()); } else if (simpleTypeInfo == Types.INT) { - return Optional.of((mapper, jsonNode) -> jsonNode.asInt()); + return Optional.of(this::convertToInt); } else if (simpleTypeInfo == Types.LONG) { - return Optional.of((mapper, jsonNode) -> jsonNode.asLong()); + return Optional.of(this::convertToLong); } else if (simpleTypeInfo == Types.DOUBLE) { - return Optional.of((mapper, jsonNode) -> jsonNode.asDouble()); + return Optional.of(this::convertToDouble); } else if (simpleTypeInfo == Types.FLOAT) { return Optional.of((mapper, jsonNode) -> Float.parseFloat(jsonNode.asText().trim())); } else if (simpleTypeInfo == Types.SHORT) { @@ -331,9 +361,9 @@ private Optional createConverterForSimpleType(T } else if (simpleTypeInfo == Types.BYTE) { return Optional.of((mapper, jsonNode) -> Byte.parseByte(jsonNode.asText().trim())); } else if (simpleTypeInfo == Types.BIG_DEC) { - return Optional.of((mapper, jsonNode) -> jsonNode.decimalValue()); + return Optional.of(this::convertToBigDecimal); } else if (simpleTypeInfo == Types.BIG_INT) { - return Optional.of((mapper, jsonNode) -> jsonNode.bigIntegerValue()); + return Optional.of(this::convertToBigInteger); } else if (simpleTypeInfo == Types.SQL_DATE) { return Optional.of(this::convertToDate); } else if (simpleTypeInfo == Types.SQL_TIME) { @@ -351,6 +381,60 @@ private Optional createConverterForSimpleType(T } } + private boolean convertToBoolean(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isBoolean()) { + // avoid redundant toString and parseBoolean, for better performance + return jsonNode.asBoolean(); + } else { + return Boolean.parseBoolean(jsonNode.asText().trim()); + } + } + + private int convertToInt(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.canConvertToInt()) { + // avoid redundant toString and parseInt, for better performance + return jsonNode.asInt(); + } else { + return Integer.parseInt(jsonNode.asText().trim()); + } + } + + private long convertToLong(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.canConvertToLong()) { + // avoid redundant toString and parseLong, for better performance + return jsonNode.asLong(); + } else { + return Long.parseLong(jsonNode.asText().trim()); + } + } + + private double convertToDouble(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isDouble()) { + // avoid redundant toString and parseDouble, for better performance + return jsonNode.asDouble(); + } else { + return Double.parseDouble(jsonNode.asText().trim()); + } + } + + private BigDecimal convertToBigDecimal(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isBigDecimal()) { + // avoid redundant toString and toDecimal, for better performance + return jsonNode.decimalValue(); + } else { + return new BigDecimal(jsonNode.asText().trim()); + } + } + + private BigInteger convertToBigInteger(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isBigInteger()) { + // avoid redundant toString and toBigInteger, for better performance + return jsonNode.bigIntegerValue(); + } else { + return new BigInteger(jsonNode.asText().trim()); + } + } + private LocalDate convertToLocalDate(ObjectMapper mapper, JsonNode jsonNode) { return ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); } @@ -387,6 +471,7 @@ private LocalTime convertToLocalTime(ObjectMapper mapper, JsonNode jsonNode) { // according to RFC 3339 every full-time must have a timezone; // until we have full timezone support, we only support UTC; // users can parse their time as string as a workaround + TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText()); ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset()); @@ -409,7 +494,6 @@ private DeserializationRuntimeConverter assembleRowConverter( List fieldConverters) { return (mapper, jsonNode) -> { ObjectNode node = (ObjectNode) jsonNode; - int arity = fieldNames.length; Row row = new Row(arity); for (int i = 0; i < arity; i++) { @@ -457,4 +541,14 @@ private DeserializationRuntimeConverter assembleArrayConverter( return array; }; } + + /** + * Exception which refers to parse errors in converters. + * */ + private static final class JsonParseException extends RuntimeException { + private static final long serialVersionUID = 1L; + public JsonParseException(String message, Throwable cause) { + super(message, cause); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java index af758b63c..b855d94e7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -67,7 +67,12 @@ public DeserializationSchema createDeserializationSchema(Map { + if (flag) { + schema.ignoreParseErrors(); + } + }); return schema.build(); } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java index 0f4df09ed..564c88b01 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -27,6 +27,7 @@ import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_IGNORE_PARSE_ERRORS; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_JSON_SCHEMA; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_SCHEMA; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_TYPE_VALUE; @@ -38,6 +39,7 @@ public class Json extends FormatDescriptor { private Boolean failOnMissingField; private Boolean deriveSchema; + private Boolean ignoreParseErrors; private String jsonSchema; private String schema; @@ -59,6 +61,17 @@ public Json failOnMissingField(boolean failOnMissingField) { return this; } + /** + * Sets flag whether to fail when parsing json fails. + * + * @param ignoreParseErrors If set to true, the operation will ignore parse errors. + * If set to false, the operation fails when parsing json fails. + */ + public Json ignoreParseErrors(boolean ignoreParseErrors) { + this.ignoreParseErrors = ignoreParseErrors; + return this; + } + /** * Sets the JSON schema string with field names and the types according to the JSON schema * specification [[http://json-schema.org/specification.html]]. @@ -140,6 +153,10 @@ protected Map toFormatProperties() { properties.putBoolean(FORMAT_FAIL_ON_MISSING_FIELD, failOnMissingField); } + if (ignoreParseErrors != null) { + properties.putBoolean(FORMAT_IGNORE_PARSE_ERRORS, ignoreParseErrors); + } + return properties.asMap(); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java index 061d7fd16..2d0b04e11 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java @@ -31,6 +31,7 @@ public class JsonValidator extends FormatDescriptorValidator { public static final String FORMAT_SCHEMA = "format.schema"; public static final String FORMAT_JSON_SCHEMA = "format.json-schema"; public static final String FORMAT_FAIL_ON_MISSING_FIELD = "format.fail-on-missing-field"; + public static final String FORMAT_IGNORE_PARSE_ERRORS = "format.ignore-parse-errors"; @Override public void validate(DescriptorProperties properties) { @@ -52,5 +53,11 @@ public void validate(DescriptorProperties properties) { } properties.validateBoolean(FORMAT_FAIL_ON_MISSING_FIELD, true); + properties.validateBoolean(FORMAT_IGNORE_PARSE_ERRORS, true); + boolean failOnMissingField = properties.getOptionalBoolean(FORMAT_FAIL_ON_MISSING_FIELD).orElse(false); + boolean ignoreParseErrors = properties.getOptionalBoolean(FORMAT_IGNORE_PARSE_ERRORS).orElse(false); + if (ignoreParseErrors && failOnMissingField) { + throw new ValidationException(FORMAT_FAIL_ON_MISSING_FIELD + " and " + FORMAT_IGNORE_PARSE_ERRORS + " shouldn't both be true."); + } } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index 717898770..cba7fcefd 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -26,26 +26,37 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; + +import javax.annotation.Nullable; import java.math.BigDecimal; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import static org.apache.flink.formats.utils.DeserializationSchemaMatcher.whenDeserializedWith; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertThat; import static org.junit.internal.matchers.ThrowableCauseMatcher.hasCause; +import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; /** * Tests for the {@link JsonRowDeserializationSchema}. */ public class JsonRowDeserializationSchemaTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + /** * Tests simple deserialization using type information. */ @@ -211,6 +222,20 @@ public void testMissingNode() throws Exception { assertThat(serializedJson, whenDeserializedWith(deserializationSchema) .failsWithException(hasCause(instanceOf(IllegalStateException.class)))); + + // ignore-parse-errors ignores missing field exception too + deserializationSchema = new JsonRowDeserializationSchema.Builder(rowTypeInformation) + .ignoreParseErrors() + .build(); + assertThat(serializedJson, + whenDeserializedWith(deserializationSchema).equalsTo(row)); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("JSON format doesn't support failOnMissingField and ignoreParseErrors are both true"); + new JsonRowDeserializationSchema.Builder(rowTypeInformation) + .failOnMissingField() + .ignoreParseErrors() + .build(); } /** @@ -228,4 +253,156 @@ public void testNumberOfFieldNamesAndTypesMismatch() { // Expected } } + + @Test + public void testJsonParse() { + for (TestSpec spec : testData) { + testIgnoreParseErrors(spec); + if (spec.errorMessage != null) { + testParseErrors(spec); + } + } + } + + private void testIgnoreParseErrors(TestSpec spec) { + // the parsing field should be null and no exception is thrown + JsonRowDeserializationSchema ignoreErrorsSchema = + new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation) + .ignoreParseErrors() + .build(); + Row expected; + if (spec.expected != null) { + expected = spec.expected; + } else { + expected = new Row(1); + } + assertThat("Test Ignore Parse Error: " + spec.json, + spec.json.getBytes(), + whenDeserializedWith(ignoreErrorsSchema).equalsTo(expected)); + } + + private void testParseErrors(TestSpec spec) { + // expect exception if parse error is not ignored + JsonRowDeserializationSchema failingSchema = + new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation) + .build(); + assertThat("Test Parse Error: " + spec.json, + spec.json.getBytes(), + whenDeserializedWith(failingSchema) + .failsWithException(hasMessage(containsString(spec.errorMessage)))); + } + + private static List testData = Arrays.asList( + TestSpec + .json("{\"id\": \"trueA\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.BOOLEAN)) + .expect(Row.of(false)), + + TestSpec + .json("{\"id\": true}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.BOOLEAN)) + .expect(Row.of(true)), + + TestSpec + .json("{\"id\":\"abc\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.INT)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), + + TestSpec + .json("{\"id\":112.013}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.LONG)) + .expect(Row.of(112L)), + + TestSpec + .json("{\"id\":\"long\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.LONG)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'"), + + TestSpec + .json("{\"id\":\"112.013.123\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.FLOAT)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), + + TestSpec + .json("{\"id\":\"112.013.123\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.DOUBLE)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), + + TestSpec + .json("{\"id\":\"18:00:243\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.SQL_TIME)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"), + + TestSpec + .json("{\"id\":\"20191112\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.SQL_DATE)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'"), + + TestSpec + .json("{\"id\":\"2019-11-12 18:00:12\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.SQL_TIMESTAMP)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'"), + + TestSpec + .json("{\"id\":\"abc\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.BIG_DEC)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), + + TestSpec + .json("{\"row\":{\"id\":\"abc\"}}") + .typeInfo(Types.ROW_NAMED(new String[]{"row"}, Types.ROW_NAMED(new String[]{"id"}, Types.INT))) + .expect(Row.of(new Row(1))) + .expectErrorMessage("Failed to deserialize JSON '{\"row\":{\"id\":\"abc\"}}'"), + + TestSpec + .json("{\"array\":[123, \"abc\"]}") + .typeInfo(Types.ROW_NAMED(new String[]{"array"}, Types.OBJECT_ARRAY(Types.INT))) + .expect(Row.of((Object) new Integer[]{123, null})) + .expectErrorMessage("Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'"), + + TestSpec + .json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") + .typeInfo(Types.ROW_NAMED(new String[]{"map"}, Types.MAP(Types.STRING, Types.INT))) + .expect(Row.of(createHashMap("key1", 123, "key2", null))) + .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'") + + + ); + + private static Map createHashMap(String k1, Integer v1, String k2, Integer v2) { + Map map = new HashMap<>(); + map.put(k1, v1); + map.put(k2, v2); + return map; + } + + private static class TestSpec { + private final String json; + private @Nullable TypeInformation rowTypeInformation; + private @Nullable Row expected; + private @Nullable String errorMessage; + + private TestSpec(String json) { + this.json = json; + } + + public static TestSpec json(String json) { + return new TestSpec(json); + } + + TestSpec expect(Row row) { + this.expected = row; + return this; + } + + TestSpec typeInfo(TypeInformation rowTypeInformation) { + this.rowTypeInformation = rowTypeInformation; + return this; + } + + TestSpec expectErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + return this; + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java index 2bc58987d..5436b2d2d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java @@ -65,6 +65,11 @@ public void testDuplicateSchema() { addPropertyAndVerify(descriptors().get(0), "format.schema", "DDD"); } + @Test(expected = ValidationException.class) + public void testInvalidIgnoreParseErrors() { + addPropertyAndVerify(descriptors().get(0), "format.ignore-parse-errors", "DDD"); + } + // -------------------------------------------------------------------------------------------- @Override @@ -84,7 +89,11 @@ public List descriptors() { final Descriptor desc5 = new Json().failOnMissingField(false); - return Arrays.asList(desc1, desc2, desc3, desc4, desc5); + final Descriptor desc6 = new Json().jsonSchema(JSON_SCHEMA).ignoreParseErrors(false); + + final Descriptor desc7 = new Json().ignoreParseErrors(true); + + return Arrays.asList(desc1, desc2, desc3, desc4, desc5, desc6, desc7); } @Override @@ -116,7 +125,18 @@ public List> properties() { props5.put("format.property-version", "1"); props5.put("format.fail-on-missing-field", "false"); - return Arrays.asList(props1, props2, props3, props4, props5); + final Map props6 = new HashMap<>(); + props6.put("format.type", "json"); + props6.put("format.property-version", "1"); + props6.put("format.json-schema", JSON_SCHEMA); + props6.put("format.ignore-parse-errors", "false"); + + final Map props7 = new HashMap<>(); + props7.put("format.type", "json"); + props7.put("format.property-version", "1"); + props7.put("format.ignore-parse-errors", "true"); + + return Arrays.asList(props1, props2, props3, props4, props5, props6, props7); } @Override From a3f88500dc6f3c0e718df2df772aaf08b73c4d62 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 16 Mar 2020 13:31:51 +0800 Subject: [PATCH 062/322] [FLINK-16454][build] Update the copyright with 2020 year in NOTICE files --- .../src/main/resources/META-INF/NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 26541c32e..9417323f4 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-avro-confluent-registry -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From 2294b67dce94d22aa9aa2c8c5c48f0e49d642c8d Mon Sep 17 00:00:00 2001 From: Benchao Li Date: Fri, 20 Mar 2020 14:44:19 +0800 Subject: [PATCH 063/322] [FLINK-16220][json] Fix cast exception in JsonRowSerializationSchema when serializing null fields This closes #11180 --- .../json/JsonRowSerializationSchema.java | 6 ++-- .../json/JsonRowSerializationSchemaTest.java | 35 +++++++++++++++++++ 2 files changed, 39 insertions(+), 2 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index 284739534..74418767d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -360,7 +360,8 @@ private SerializationRuntimeConverter assembleRowConverter( return (mapper, reuse, object) -> { ObjectNode node; - if (reuse == null) { + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { node = mapper.createObjectNode(); } else { node = (ObjectNode) reuse; @@ -382,7 +383,8 @@ private SerializationRuntimeConverter assembleArrayConverter(SerializationRuntim return (mapper, reuse, object) -> { ArrayNode node; - if (reuse == null) { + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { node = mapper.createArrayNode(); } else { node = (ArrayNode) reuse; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java index 4a0706fe2..378f92b89 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java @@ -91,6 +91,41 @@ public void testSerializationOfTwoRows() throws IOException { assertEquals(row2, deserializationSchema.deserialize(bytes)); } + @Test + public void testMultiRowsWithNullValues() throws IOException { + String[] jsons = new String[] { + "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, " + + "\"ids\":[1, 2, 3]}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}", + }; + + String[] expected = new String[] { + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}," + + "\"ids\":[1,2,3]}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}", + }; + + TypeInformation schema = Types.ROW_NAMED( + new String[]{"svt", "ops", "ids"}, + Types.STRING, + Types.ROW_NAMED(new String[]{"id"}, Types.STRING), + Types.PRIMITIVE_ARRAY(Types.INT)); + JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(schema) + .build(); + JsonRowSerializationSchema serializationSchema = JsonRowSerializationSchema.builder() + .withTypeInfo(schema) + .build(); + + for (int i = 0; i < jsons.length; i++) { + String json = jsons[i]; + Row row = deserializationSchema.deserialize(json.getBytes()); + String result = new String(serializationSchema.serialize(row)); + assertEquals(expected[i], result); + } + } + @Test public void testNestedSchema() { final TypeInformation rowSchema = Types.ROW_NAMED( From 55543f8e874d7bba8bd8cad8febb2e00b262d1a1 Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Tue, 17 Mar 2020 12:26:50 +0800 Subject: [PATCH 064/322] [FLINK-16125][connecotr/kafka] Remove Kafka connector property zookeeper.connect and clear documentation because Kafka 0.8 connector has been removed. --- .../flink/schema/registry/test/TestAvroConsumerConfluent.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java index 55549dee9..dda461797 100644 --- a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java +++ b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java @@ -37,7 +37,7 @@ * A simple example that shows how to read from and write to Kafka with Confluent Schema Registry. * This will read AVRO messages from the input topic, parse them into a POJO type via checking the Schema by calling Schema registry. * Then this example publish the POJO type to kafka by converting the POJO to AVRO and verifying the schema. - * --input-topic test-input --output-string-topic test-output --output-avro-topic test-avro-output --output-subject --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --schema-registry-url http://localhost:8081 --group.id myconsumer + * --input-topic test-input --output-string-topic test-output --output-avro-topic test-avro-output --output-subject --bootstrap.servers localhost:9092 --schema-registry-url http://localhost:8081 --group.id myconsumer */ public class TestAvroConsumerConfluent { @@ -49,14 +49,12 @@ public static void main(String[] args) throws Exception { System.out.println("Missing parameters!\n" + "Usage: Kafka --input-topic --output-string-topic --output-avro-topic " + "--bootstrap.servers " + - "--zookeeper.connect " + "--schema-registry-url --group.id "); return; } Properties config = new Properties(); config.setProperty("bootstrap.servers", parameterTool.getRequired("bootstrap.servers")); config.setProperty("group.id", parameterTool.getRequired("group.id")); - config.setProperty("zookeeper.connect", parameterTool.getRequired("zookeeper.connect")); String schemaRegistryUrl = parameterTool.getRequired("schema-registry-url"); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); From 80c31f0b0f31dc0eed598d681acf38def6c6de7f Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Thu, 23 Apr 2020 10:55:41 +0200 Subject: [PATCH 065/322] [hotfix] Set kafka-schema-registry-client to 4.1.0 in NOTICE --- .../src/main/resources/META-INF/NOTICE | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 9417323f4..82acd60fc 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -10,6 +10,6 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-databind:2.10.1 - com.fasterxml.jackson.core:jackson-annotations:2.10.1 - com.fasterxml.jackson.core:jackson-core:2.10.1 -- io.confluent:common-utils:3.3.1 -- io.confluent:kafka-schema-registry-client:3.3.1 +- io.confluent:common-utils:4.1.0 +- io.confluent:kafka-schema-registry-client:4.1.0 - org.apache.zookeeper:zookeeper:3.4.10 From 5c6819b7b910292ed1ebd9cfa2ebf09566abb946 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 30 Apr 2020 17:46:34 +0800 Subject: [PATCH 066/322] [FLINK-17461][formats][json] Support JSON serialization and deseriazation schema for RowData type This closes #11944 --- .../flink-json-debezium/pom.xml | 6 +- .../JsonRowDataDeserializationSchema.java | 423 +++++++++++++++ .../json/JsonRowDataSerializationSchema.java | 287 ++++++++++ .../json/JsonRowDataSerDeSchemaTest.java | 500 ++++++++++++++++++ 4 files changed, 1213 insertions(+), 3 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 3bc69aa83..bff7fc1a0 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -69,15 +69,15 @@ under the License. test-jar - + org.apache.flink - flink-table-planner_${scala.binary.version} + flink-table-planner-blink_${scala.binary.version} ${project.version} test - + org.scala-lang scala-compiler diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java new file mode 100644 index 000000000..974b8dde3 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -0,0 +1,423 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.table.types.logical.utils.LogicalTypeUtils; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; + +import java.io.IOException; +import java.io.Serializable; +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.time.temporal.TemporalAccessor; +import java.time.temporal.TemporalQueries; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import static java.lang.String.format; +import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; +import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Deserialization schema from JSON to Flink Table/SQL internal data structure {@link RowData}. + * + *

Deserializes a byte[] message as a JSON object and reads + * the specified fields. + * + *

Failures during deserialization are forwarded as wrapped IOExceptions. + */ +@Internal +public class JsonRowDataDeserializationSchema implements DeserializationSchema { + private static final long serialVersionUID = 1L; + + /** Flag indicating whether to fail if a field is missing. */ + private final boolean failOnMissingField; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + /** TypeInformation of the produced {@link RowData}. **/ + private final TypeInformation resultTypeInfo; + + /** + * Runtime converter that converts {@link JsonNode}s into + * objects of Flink SQL internal data structures. **/ + private final DeserializationRuntimeConverter runtimeConverter; + + /** Object mapper for parsing the JSON. */ + private final ObjectMapper objectMapper = new ObjectMapper(); + + public JsonRowDataDeserializationSchema( + RowType rowType, + TypeInformation resultTypeInfo, + boolean failOnMissingField, + boolean ignoreParseErrors) { + if (ignoreParseErrors && failOnMissingField) { + throw new IllegalArgumentException( + "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."); + } + this.resultTypeInfo = checkNotNull(resultTypeInfo); + this.failOnMissingField = failOnMissingField; + this.ignoreParseErrors = ignoreParseErrors; + this.runtimeConverter = createRowConverter(checkNotNull(rowType)); + } + + @Override + public RowData deserialize(byte[] message) throws IOException { + try { + final JsonNode root = objectMapper.readTree(message); + return (RowData) runtimeConverter.convert(root); + } catch (Throwable t) { + if (ignoreParseErrors) { + return null; + } + throw new IOException(format("Failed to deserialize JSON '%s'.", new String(message)), t); + } + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return resultTypeInfo; + } + + // ------------------------------------------------------------------------------------- + // Runtime Converters + // ------------------------------------------------------------------------------------- + + /** + * Runtime converter that converts {@link JsonNode}s into objects of Flink Table & SQL + * internal data structures. + */ + @FunctionalInterface + private interface DeserializationRuntimeConverter extends Serializable { + Object convert(JsonNode jsonNode); + } + + /** + * Creates a runtime converter which is null safe. + */ + private DeserializationRuntimeConverter createConverter(LogicalType type) { + return wrapIntoNullableConverter(createNotNullConverter(type)); + } + + /** + * Creates a runtime converter which assuming input object is not null. + */ + private DeserializationRuntimeConverter createNotNullConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return jsonNode -> null; + case BOOLEAN: + return this::convertToBoolean; + case TINYINT: + return jsonNode -> Byte.parseByte(jsonNode.asText().trim()); + case SMALLINT: + return jsonNode -> Short.parseShort(jsonNode.asText().trim()); + case INTEGER: + case INTERVAL_YEAR_MONTH: + return this::convertToInt; + case BIGINT: + case INTERVAL_DAY_TIME: + return this::convertToLong; + case DATE: + return this::convertToDate; + case TIME_WITHOUT_TIME_ZONE: + return this::convertToTime; + case TIMESTAMP_WITH_TIME_ZONE: + case TIMESTAMP_WITHOUT_TIME_ZONE: + return this::convertToTimestamp; + case FLOAT: + return this::convertToFloat; + case DOUBLE: + return this::convertToDouble; + case CHAR: + case VARCHAR: + return this::convertToString; + case BINARY: + case VARBINARY: + return this::convertToBytes; + case DECIMAL: + return createDecimalConverter((DecimalType) type); + case ARRAY: + return createArrayConverter((ArrayType) type); + case MAP: + case MULTISET: + return createMapConverter((MapType) type); + case ROW: + return createRowConverter((RowType) type); + case RAW: + default: + throw new UnsupportedOperationException("Unsupported type: " + type); + } + } + + private boolean convertToBoolean(JsonNode jsonNode) { + if (jsonNode.isBoolean()) { + // avoid redundant toString and parseBoolean, for better performance + return jsonNode.asBoolean(); + } else { + return Boolean.parseBoolean(jsonNode.asText().trim()); + } + } + + private int convertToInt(JsonNode jsonNode) { + if (jsonNode.canConvertToInt()) { + // avoid redundant toString and parseInt, for better performance + return jsonNode.asInt(); + } else { + return Integer.parseInt(jsonNode.asText().trim()); + } + } + + private long convertToLong(JsonNode jsonNode) { + if (jsonNode.canConvertToLong()) { + // avoid redundant toString and parseLong, for better performance + return jsonNode.asLong(); + } else { + return Long.parseLong(jsonNode.asText().trim()); + } + } + + private double convertToDouble(JsonNode jsonNode) { + if (jsonNode.isDouble()) { + // avoid redundant toString and parseDouble, for better performance + return jsonNode.asDouble(); + } else { + return Double.parseDouble(jsonNode.asText().trim()); + } + } + + private float convertToFloat(JsonNode jsonNode) { + if (jsonNode.isDouble()) { + // avoid redundant toString and parseDouble, for better performance + return (float) jsonNode.asDouble(); + } else { + return Float.parseFloat(jsonNode.asText().trim()); + } + } + + private int convertToDate(JsonNode jsonNode) { + LocalDate date = ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); + return (int) date.toEpochDay(); + } + + private int convertToTime(JsonNode jsonNode) { + // according to RFC 3339 every full-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText()); + + ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset()); + LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); + + if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0 || localTime.getNano() != 0) { + throw new JsonParseException( + "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet."); + } + + // get number of milliseconds of the day + return localTime.toSecondOfDay() * 1000; + } + + private TimestampData convertToTimestamp(JsonNode jsonNode) { + // according to RFC 3339 every date-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + TemporalAccessor parsedTimestamp = RFC3339_TIMESTAMP_FORMAT.parse(jsonNode.asText()); + + ZoneOffset zoneOffset = parsedTimestamp.query(TemporalQueries.offset()); + + if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0) { + throw new JsonParseException( + "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + + "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + } + + LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); + LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); + + return TimestampData.fromLocalDateTime(LocalDateTime.of(localDate, localTime)); + } + + private StringData convertToString(JsonNode jsonNode) { + return StringData.fromString(jsonNode.asText()); + } + + private byte[] convertToBytes(JsonNode jsonNode) { + try { + return jsonNode.binaryValue(); + } catch (IOException e) { + throw new JsonParseException("Unable to deserialize byte array.", e); + } + } + + private DeserializationRuntimeConverter createDecimalConverter(DecimalType decimalType) { + final int precision = decimalType.getPrecision(); + final int scale = decimalType.getScale(); + return jsonNode -> { + BigDecimal bigDecimal; + if (jsonNode.isBigDecimal()) { + bigDecimal = jsonNode.decimalValue(); + } else { + bigDecimal = new BigDecimal(jsonNode.asText()); + } + return DecimalData.fromBigDecimal(bigDecimal, precision, scale); + }; + } + + private DeserializationRuntimeConverter createArrayConverter(ArrayType arrayType) { + DeserializationRuntimeConverter elementConverter = createConverter(arrayType.getElementType()); + final Class elementClass = LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType()); + return jsonNode -> { + final ArrayNode node = (ArrayNode) jsonNode; + final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); + for (int i = 0; i < node.size(); i++) { + final JsonNode innerNode = node.get(i); + array[i] = elementConverter.convert(innerNode); + } + return new GenericArrayData(array); + }; + } + + private DeserializationRuntimeConverter createMapConverter(MapType mapType) { + LogicalType keyType = mapType.getKeyType(); + if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { + throw new UnsupportedOperationException( + "JSON format doesn't support non-string as key type of map. " + + "The map type is: " + mapType.asSummaryString()); + } + final DeserializationRuntimeConverter keyConverter = createConverter(keyType); + final DeserializationRuntimeConverter valueConverter = createConverter(mapType.getValueType()); + + return jsonNode -> { + Iterator> fields = jsonNode.fields(); + Map result = new HashMap<>(); + while (fields.hasNext()) { + Map.Entry entry = fields.next(); + Object key = keyConverter.convert(TextNode.valueOf(entry.getKey())); + Object value = valueConverter.convert(entry.getValue()); + result.put(key, value); + } + return new GenericMapData(result); + }; + } + + private DeserializationRuntimeConverter createRowConverter(RowType rowType) { + final DeserializationRuntimeConverter[] fieldConverters = rowType.getFields().stream() + .map(RowType.RowField::getType) + .map(this::createConverter) + .toArray(DeserializationRuntimeConverter[]::new); + final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]); + + return jsonNode -> { + ObjectNode node = (ObjectNode) jsonNode; + int arity = fieldNames.length; + GenericRowData row = new GenericRowData(arity); + for (int i = 0; i < arity; i++) { + String fieldName = fieldNames[i]; + JsonNode field = node.get(fieldName); + Object convertedField = convertField(fieldConverters[i], fieldName, field); + row.setField(i, convertedField); + } + return row; + }; + } + + private Object convertField( + DeserializationRuntimeConverter fieldConverter, + String fieldName, + JsonNode field) { + if (field == null) { + if (failOnMissingField) { + throw new JsonParseException( + "Could not find field with name '" + fieldName + "'."); + } else { + return null; + } + } else { + return fieldConverter.convert(field); + } + } + + private DeserializationRuntimeConverter wrapIntoNullableConverter( + DeserializationRuntimeConverter converter) { + return jsonNode -> { + if (jsonNode == null || jsonNode.isNull()) { + return null; + } + try { + return converter.convert(jsonNode); + } catch (Throwable t) { + if (!ignoreParseErrors) { + throw t; + } + return null; + } + }; + } + + /** + * Exception which refers to parse errors in converters. + * */ + private static final class JsonParseException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public JsonParseException(String message) { + super(message); + } + + public JsonParseException(String message, Throwable cause) { + super(message, cause); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java new file mode 100644 index 000000000..7c177386a --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.time.LocalDate; +import java.time.LocalTime; +import java.util.Arrays; + +import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; +import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; + +/** + * Serialization schema that serializes an object of Flink internal data structure into a JSON bytes. + * + *

Serializes the input Flink object into a JSON string and + * converts it into byte[]. + * + *

Result byte[] messages can be deserialized using {@link JsonRowDataDeserializationSchema}. + */ +@Internal +public class JsonRowDataSerializationSchema implements SerializationSchema { + private static final long serialVersionUID = 1L; + + /** The converter that converts internal data formats to JsonNode. */ + private final SerializationRuntimeConverter runtimeConverter; + + /** Object mapper that is used to create output JSON objects. */ + private final ObjectMapper mapper = new ObjectMapper(); + + /** Reusable object node. */ + private transient ObjectNode node; + + public JsonRowDataSerializationSchema(RowType rowType) { + this.runtimeConverter = createConverter(rowType); + } + + @Override + public byte[] serialize(RowData row) { + if (node == null) { + node = mapper.createObjectNode(); + } + + try { + runtimeConverter.convert(mapper, node, row); + return mapper.writeValueAsBytes(node); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'. " + + "Make sure that the schema matches the input.", t); + } + } + + // -------------------------------------------------------------------------------- + // Runtime Converters + // -------------------------------------------------------------------------------- + + /** + * Runtime converter that converts objects of Flink Table & SQL internal data structures + * to corresponding {@link JsonNode}s. + */ + private interface SerializationRuntimeConverter extends Serializable { + JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value); + } + + /** + * Creates a runtime converter which is null safe. + */ + private SerializationRuntimeConverter createConverter(LogicalType type) { + return wrapIntoNullableConverter(createNotNullConverter(type)); + } + + /** + * Creates a runtime converter which assuming input object is not null. + */ + private SerializationRuntimeConverter createNotNullConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return (mapper, reuse, value) -> mapper.getNodeFactory().nullNode(); + case BOOLEAN: + return (mapper, reuse, value) -> mapper.getNodeFactory().booleanNode((boolean) value); + case TINYINT: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((byte) value); + case SMALLINT: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((short) value); + case INTEGER: + case INTERVAL_YEAR_MONTH: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((int) value); + case BIGINT: + case INTERVAL_DAY_TIME: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((long) value); + case FLOAT: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((float) value); + case DOUBLE: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((double) value); + case CHAR: + case VARCHAR: + // value is BinaryString + return (mapper, reuse, value) -> mapper.getNodeFactory().textNode(value.toString()); + case BINARY: + case VARBINARY: + return (mapper, reuse, value) -> mapper.getNodeFactory().binaryNode((byte[]) value); + case DATE: + return createDateConverter(); + case TIME_WITHOUT_TIME_ZONE: + return createTimeConverter(); + case TIMESTAMP_WITH_TIME_ZONE: + case TIMESTAMP_WITHOUT_TIME_ZONE: + return createTimestampConverter(); + case DECIMAL: + return createDecimalConverter(); + case ARRAY: + return createArrayConverter((ArrayType) type); + case MAP: + case MULTISET: + return createMapConverter((MapType) type); + case ROW: + return createRowConverter((RowType) type); + case RAW: + default: + throw new UnsupportedOperationException("Not support to parse type: " + type); + } + } + + private SerializationRuntimeConverter createDecimalConverter() { + return (mapper, reuse, value) -> { + BigDecimal bd = ((DecimalData) value).toBigDecimal(); + return mapper.getNodeFactory().numberNode(bd); + }; + } + + private SerializationRuntimeConverter createDateConverter() { + return (mapper, reuse, value) -> { + int days = (int) value; + LocalDate date = LocalDate.ofEpochDay(days); + return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format(date)); + }; + } + + private SerializationRuntimeConverter createTimeConverter() { + return (mapper, reuse, value) -> { + int millisecond = (int) value; + LocalTime time = LocalTime.ofSecondOfDay(millisecond / 1000L); + return mapper.getNodeFactory().textNode(RFC3339_TIME_FORMAT.format(time)); + }; + } + + private SerializationRuntimeConverter createTimestampConverter() { + return (mapper, reuse, value) -> { + TimestampData timestamp = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(RFC3339_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); + }; + } + + private SerializationRuntimeConverter createArrayConverter(ArrayType type) { + final LogicalType elementType = type.getElementType(); + final SerializationRuntimeConverter elementConverter = createConverter(elementType); + return (mapper, reuse, value) -> { + ArrayNode node; + + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createArrayNode(); + } else { + node = (ArrayNode) reuse; + node.removeAll(); + } + + ArrayData array = (ArrayData) value; + int numElements = array.size(); + for (int i = 0; i < numElements; i++) { + Object element = ArrayData.get(array, i, elementType); + node.add(elementConverter.convert(mapper, null, element)); + } + + return node; + }; + } + + private SerializationRuntimeConverter createMapConverter(MapType type) { + LogicalType keyType = type.getKeyType(); + if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { + throw new UnsupportedOperationException( + "JSON format doesn't support non-string as key type of map. " + + "The map type is: " + type.asSummaryString()); + } + final LogicalType valueType = type.getValueType(); + final SerializationRuntimeConverter valueConverter = createConverter(valueType); + return (mapper, reuse, object) -> { + ObjectNode node; + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createObjectNode(); + } else { + node = (ObjectNode) reuse; + } + + MapData map = (MapData) object; + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + int numElements = map.size(); + for (int i = 0; i < numElements; i++) { + String fieldName = keyArray.getString(i).toString(); // key must be string + Object value = ArrayData.get(valueArray, i, valueType); + node.set(fieldName, valueConverter.convert(mapper, node.get(fieldName), value)); + } + + return node; + }; + } + + private SerializationRuntimeConverter createRowConverter(RowType type) { + final String[] fieldNames = type.getFieldNames().toArray(new String[0]); + final LogicalType[] fieldTypes = type.getFields().stream() + .map(RowType.RowField::getType) + .toArray(LogicalType[]::new); + final SerializationRuntimeConverter[] fieldConverters = Arrays.stream(fieldTypes) + .map(this::createConverter) + .toArray(SerializationRuntimeConverter[]::new); + final int fieldCount = type.getFieldCount(); + + return (mapper, reuse, value) -> { + ObjectNode node; + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createObjectNode(); + } else { + node = (ObjectNode) reuse; + } + RowData row = (RowData) value; + for (int i = 0; i < fieldCount; i++) { + String fieldName = fieldNames[i]; + Object field = RowData.get(row, i, fieldTypes[i]); + node.set(fieldName, fieldConverters[i].convert(mapper, node.get(fieldName), field)); + } + return node; + }; + } + + private SerializationRuntimeConverter wrapIntoNullableConverter( + SerializationRuntimeConverter converter) { + return (mapper, reuse, object) -> { + if (object == null) { + return mapper.getNodeFactory().nullNode(); + } + + return converter.convert(mapper, reuse, object); + }; + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java new file mode 100644 index 000000000..dcfae9c26 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -0,0 +1,500 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.flink.table.api.DataTypes.ARRAY; +import static org.apache.flink.table.api.DataTypes.BIGINT; +import static org.apache.flink.table.api.DataTypes.BOOLEAN; +import static org.apache.flink.table.api.DataTypes.BYTES; +import static org.apache.flink.table.api.DataTypes.DATE; +import static org.apache.flink.table.api.DataTypes.DECIMAL; +import static org.apache.flink.table.api.DataTypes.DOUBLE; +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.FLOAT; +import static org.apache.flink.table.api.DataTypes.INT; +import static org.apache.flink.table.api.DataTypes.MAP; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.SMALLINT; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.apache.flink.table.api.DataTypes.TIME; +import static org.apache.flink.table.api.DataTypes.TIMESTAMP; +import static org.apache.flink.table.api.DataTypes.TINYINT; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link JsonRowDataDeserializationSchema} and {@link JsonRowDataSerializationSchema}. + */ +public class JsonRowDataSerDeSchemaTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testSerDe() throws Exception { + byte tinyint = 'c'; + short smallint = 128; + int intValue = 45536; + float floatValue = 33.333F; + long bigint = 1238123899121L; + String name = "asdlkjasjkdla998y1122"; + byte[] bytes = new byte[1024]; + ThreadLocalRandom.current().nextBytes(bytes); + BigDecimal decimal = new BigDecimal("123.456789"); + Double[] doubles = new Double[]{1.1, 2.2, 3.3}; + LocalDate date = LocalDate.parse("1990-10-14"); + LocalTime time = LocalTime.parse("12:12:43"); + Timestamp timestamp3 = Timestamp.valueOf("1990-10-14 12:12:43.123"); + Timestamp timestamp9 = Timestamp.valueOf("1990-10-14 12:12:43.123456789"); + + Map map = new HashMap<>(); + map.put("flink", 123L); + + Map> nestedMap = new HashMap<>(); + Map innerMap = new HashMap<>(); + innerMap.put("key", 234); + nestedMap.put("inner_map", innerMap); + + ObjectMapper objectMapper = new ObjectMapper(); + ArrayNode doubleNode = objectMapper.createArrayNode().add(1.1D).add(2.2D).add(3.3D); + + // Root + ObjectNode root = objectMapper.createObjectNode(); + root.put("bool", true); + root.put("tinyint", tinyint); + root.put("smallint", smallint); + root.put("int", intValue); + root.put("bigint", bigint); + root.put("float", floatValue); + root.put("name", name); + root.put("bytes", bytes); + root.put("decimal", decimal); + root.set("doubles", doubleNode); + root.put("date", "1990-10-14"); + root.put("time", "12:12:43Z"); + root.put("timestamp3", "1990-10-14T12:12:43.123Z"); + root.put("timestamp9", "1990-10-14T12:12:43.123456789Z"); + root.putObject("map").put("flink", 123); + root.putObject("map2map").putObject("inner_map").put("key", 234); + + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + DataType dataType = ROW( + FIELD("bool", BOOLEAN()), + FIELD("tinyint", TINYINT()), + FIELD("smallint", SMALLINT()), + FIELD("int", INT()), + FIELD("bigint", BIGINT()), + FIELD("float", FLOAT()), + FIELD("name", STRING()), + FIELD("bytes", BYTES()), + FIELD("decimal", DECIMAL(9, 6)), + FIELD("doubles", ARRAY(DOUBLE())), + FIELD("date", DATE()), + FIELD("time", TIME(0)), + FIELD("timestamp3", TIMESTAMP(3)), + FIELD("timestamp9", TIMESTAMP(9)), + FIELD("map", MAP(STRING(), BIGINT())), + FIELD("map2map", MAP(STRING(), MAP(STRING(), INT())))); + RowType schema = (RowType) dataType.getLogicalType(); + RowDataTypeInfo resultTypeInfo = new RowDataTypeInfo(schema); + + JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( + schema, resultTypeInfo, false, false); + + Row expected = new Row(16); + expected.setField(0, true); + expected.setField(1, tinyint); + expected.setField(2, smallint); + expected.setField(3, intValue); + expected.setField(4, bigint); + expected.setField(5, floatValue); + expected.setField(6, name); + expected.setField(7, bytes); + expected.setField(8, decimal); + expected.setField(9, doubles); + expected.setField(10, date); + expected.setField(11, time); + expected.setField(12, timestamp3.toLocalDateTime()); + expected.setField(13, timestamp9.toLocalDateTime()); + expected.setField(14, map); + expected.setField(15, nestedMap); + + RowData rowData = deserializationSchema.deserialize(serializedJson); + Row actual = convertToExternal(rowData, dataType); + assertEquals(expected, actual); + + // test serialization + JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(schema); + + byte[] actualBytes = serializationSchema.serialize(rowData); + assertEquals(new String(serializedJson), new String(actualBytes)); + } + + /** + * Tests the deserialization slow path, + * e.g. convert into string and use {@link Double#parseDouble(String)}. + */ + @Test + public void testSlowDeserialization() throws Exception { + Random random = new Random(); + boolean bool = random.nextBoolean(); + int integer = random.nextInt(); + long bigint = random.nextLong(); + double doubleValue = random.nextDouble(); + float floatValue = random.nextFloat(); + + ObjectMapper objectMapper = new ObjectMapper(); + ObjectNode root = objectMapper.createObjectNode(); + root.put("bool", String.valueOf(bool)); + root.put("int", String.valueOf(integer)); + root.put("bigint", String.valueOf(bigint)); + root.put("double1", String.valueOf(doubleValue)); + root.put("double2", new BigDecimal(doubleValue)); + root.put("float1", String.valueOf(floatValue)); + root.put("float2", new BigDecimal(floatValue)); + + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + DataType dataType = ROW( + FIELD("bool", BOOLEAN()), + FIELD("int", INT()), + FIELD("bigint", BIGINT()), + FIELD("double1", DOUBLE()), + FIELD("double2", DOUBLE()), + FIELD("float1", FLOAT()), + FIELD("float2", FLOAT()) + ); + RowType rowType = (RowType) dataType.getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( + rowType, new RowDataTypeInfo(rowType), false, false); + + Row expected = new Row(7); + expected.setField(0, bool); + expected.setField(1, integer); + expected.setField(2, bigint); + expected.setField(3, doubleValue); + expected.setField(4, doubleValue); + expected.setField(5, floatValue); + expected.setField(6, floatValue); + + RowData rowData = deserializationSchema.deserialize(serializedJson); + Row actual = convertToExternal(rowData, dataType); + assertEquals(expected, actual); + } + + @Test + public void testSerDeMultiRows() throws Exception { + RowType rowType = (RowType) ROW( + FIELD("f1", INT()), + FIELD("f2", BOOLEAN()), + FIELD("f3", STRING()) + ).getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( + rowType, new RowDataTypeInfo(rowType), false, false); + JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType); + + ObjectMapper objectMapper = new ObjectMapper(); + + // the first row + { + ObjectNode root = objectMapper.createObjectNode(); + root.put("f1", 1); + root.put("f2", true); + root.put("f3", "str"); + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + RowData rowData = deserializationSchema.deserialize(serializedJson); + byte[] actual = serializationSchema.serialize(rowData); + assertEquals(new String(serializedJson), new String(actual)); + } + + // the second row + { + ObjectNode root = objectMapper.createObjectNode(); + root.put("f1", 10); + root.put("f2", false); + root.put("f3", "newStr"); + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + RowData rowData = deserializationSchema.deserialize(serializedJson); + byte[] actual = serializationSchema.serialize(rowData); + assertEquals(new String(serializedJson), new String(actual)); + } + } + + @Test + public void testSerDeMultiRowsWithNullValues() throws Exception { + String[] jsons = new String[] { + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{\"k1\":10.01,\"k2\":\"invalid\"}}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, " + + "\"ids\":[1, 2, 3]}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{}}", + }; + + String[] expected = new String[] { + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null,\"metrics\":{\"k1\":10.01,\"k2\":null}}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}," + + "\"ids\":[1,2,3],\"metrics\":null}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null,\"metrics\":{}}", + }; + + RowType rowType = (RowType) ROW( + FIELD("svt", STRING()), + FIELD("ops", ROW(FIELD("id", STRING()))), + FIELD("ids", ARRAY(INT())), + FIELD("metrics", MAP(STRING(), DOUBLE())) + ).getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( + rowType, new RowDataTypeInfo(rowType), false, true); + JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType); + + for (int i = 0; i < jsons.length; i++) { + String json = jsons[i]; + RowData row = deserializationSchema.deserialize(json.getBytes()); + String result = new String(serializationSchema.serialize(row)); + assertEquals(expected[i], result); + } + } + + @Test + public void testDeserializationMissingNode() throws Exception { + ObjectMapper objectMapper = new ObjectMapper(); + + // Root + ObjectNode root = objectMapper.createObjectNode(); + root.put("id", 123123123); + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + DataType dataType = ROW(FIELD("name", STRING())); + RowType schema = (RowType) dataType.getLogicalType(); + + // pass on missing field + JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( + schema, new RowDataTypeInfo(schema), false, false); + + Row expected = new Row(1); + Row actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); + assertEquals(expected, actual); + + // fail on missing field + deserializationSchema = deserializationSchema = new JsonRowDataDeserializationSchema( + schema, new RowDataTypeInfo(schema), true, false); + + thrown.expect(IOException.class); + thrown.expectMessage("Failed to deserialize JSON '{\"id\":123123123}'"); + deserializationSchema.deserialize(serializedJson); + + // ignore on parse error + deserializationSchema = new JsonRowDataDeserializationSchema( + schema, new RowDataTypeInfo(schema), false, true); + actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); + assertEquals(expected, actual); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled"); + // failOnMissingField and ignoreParseErrors both enabled + //noinspection ConstantConditions + new JsonRowDataDeserializationSchema( + schema, new RowDataTypeInfo(schema), true, true); + } + + @Test + public void testJsonParse() throws Exception { + for (TestSpec spec : testData) { + testIgnoreParseErrors(spec); + if (spec.errorMessage != null) { + testParseErrors(spec); + } + } + } + + private void testIgnoreParseErrors(TestSpec spec) throws Exception { + // the parsing field should be null and no exception is thrown + JsonRowDataDeserializationSchema ignoreErrorsSchema = new JsonRowDataDeserializationSchema( + spec.rowType, new RowDataTypeInfo(spec.rowType), false, true); + Row expected; + if (spec.expected != null) { + expected = spec.expected; + } else { + expected = new Row(1); + } + RowData rowData = ignoreErrorsSchema.deserialize(spec.json.getBytes()); + Row actual = convertToExternal(rowData, fromLogicalToDataType(spec.rowType)); + assertEquals("Test Ignore Parse Error: " + spec.json, + expected, + actual); + } + + private void testParseErrors(TestSpec spec) throws Exception { + // expect exception if parse error is not ignored + JsonRowDataDeserializationSchema failingSchema = new JsonRowDataDeserializationSchema( + spec.rowType, new RowDataTypeInfo(spec.rowType), false, false); + + thrown.expectMessage(spec.errorMessage); + failingSchema.deserialize(spec.json.getBytes()); + } + + private static List testData = Arrays.asList( + TestSpec + .json("{\"id\": \"trueA\"}") + .rowType(ROW(FIELD("id", BOOLEAN()))) + .expect(Row.of(false)), + + TestSpec + .json("{\"id\": true}") + .rowType(ROW(FIELD("id", BOOLEAN()))) + .expect(Row.of(true)), + + TestSpec + .json("{\"id\":\"abc\"}") + .rowType(ROW(FIELD("id", INT()))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), + + TestSpec + .json("{\"id\":112.013}") + .rowType(ROW(FIELD("id", BIGINT()))) + .expect(Row.of(112L)), + + TestSpec + .json("{\"id\":\"long\"}") + .rowType(ROW(FIELD("id", BIGINT()))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'"), + + TestSpec + .json("{\"id\":\"112.013.123\"}") + .rowType(ROW(FIELD("id", FLOAT()))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), + + TestSpec + .json("{\"id\":\"112.013.123\"}") + .rowType(ROW(FIELD("id", DOUBLE()))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), + + TestSpec + .json("{\"id\":\"18:00:243\"}") + .rowType(ROW(FIELD("id", TIME()))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"), + + TestSpec + .json("{\"id\":\"20191112\"}") + .rowType(ROW(FIELD("id", DATE()))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'"), + + TestSpec + .json("{\"id\":\"2019-11-12 18:00:12\"}") + .rowType(ROW(FIELD("id", TIMESTAMP(0)))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'"), + + TestSpec + .json("{\"id\":\"abc\"}") + .rowType(ROW(FIELD("id", DECIMAL(10, 3)))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), + + TestSpec + .json("{\"row\":{\"id\":\"abc\"}}") + .rowType(ROW(FIELD("row", ROW(FIELD("id", BOOLEAN()))))) + .expect(Row.of(new Row(1))) + .expectErrorMessage("Failed to deserialize JSON '{\"row\":{\"id\":\"abc\"}}'"), + + TestSpec + .json("{\"array\":[123, \"abc\"]}") + .rowType(ROW(FIELD("array", ARRAY(INT())))) + .expect(Row.of((Object) new Integer[]{123, null})) + .expectErrorMessage("Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'"), + + TestSpec + .json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") + .rowType(ROW(FIELD("map", MAP(STRING(), INT())))) + .expect(Row.of(createHashMap("key1", 123, "key2", null))) + .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'") + + + ); + + private static Map createHashMap(String k1, Integer v1, String k2, Integer v2) { + Map map = new HashMap<>(); + map.put(k1, v1); + map.put(k2, v2); + return map; + } + + @SuppressWarnings("unchecked") + private static Row convertToExternal(RowData rowData, DataType dataType) { + return (Row) DataFormatConverters.getConverterForDataType(dataType).toExternal(rowData); + } + + private static class TestSpec { + private final String json; + private RowType rowType; + private Row expected; + private String errorMessage; + + private TestSpec(String json) { + this.json = json; + } + + public static TestSpec json(String json) { + return new TestSpec(json); + } + + TestSpec expect(Row row) { + this.expected = row; + return this; + } + + TestSpec rowType(DataType rowType) { + this.rowType = (RowType) rowType.getLogicalType(); + return this; + } + + TestSpec expectErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + return this; + } + } +} From 05ab9e3baabff26404ae15be73faaad50f536d34 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Mon, 11 May 2020 15:35:31 +0800 Subject: [PATCH 067/322] [FLINK-17286][table][json] Integrate json to file system connector This closes #12010 --- .../flink-json-debezium/pom.xml | 17 ++ .../json/JsonFileSystemFormatFactory.java | 270 ++++++++++++++++++ ....apache.flink.table.factories.TableFactory | 1 + .../json/JsonBatchFileSystemITCase.java | 62 ++++ .../formats/json/JsonFsStreamSinkITCase.java | 39 +++ 5 files changed, 389 insertions(+) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index bff7fc1a0..19d5045ce 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -77,6 +77,23 @@ under the License. test + + + org.apache.flink + flink-table-planner-blink_${scala.binary.version} + ${project.version} + test + test-jar + + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + org.scala-lang diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java new file mode 100644 index 000000000..9a2be3140 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.api.common.io.DelimitedInputFormat; +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.factories.FileSystemFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.PartitionPathUtils; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_IGNORE_PARSE_ERRORS; + +/** + * Factory to build reader/writer to read/write json format file. + */ +public class JsonFileSystemFormatFactory implements FileSystemFormatFactory { + + @Override + public Map requiredContext() { + Map context = new HashMap<>(); + context.put(FORMAT, "json"); + return context; + } + + @Override + public List supportedProperties() { + ArrayList properties = new ArrayList<>(); + properties.add(FORMAT_FAIL_ON_MISSING_FIELD); + properties.add(FORMAT_IGNORE_PARSE_ERRORS); + return properties; + } + + @Override + public InputFormat createReader(ReaderContext context) { + DescriptorProperties properties = getValidatedProperties(context.getFormatProperties()); + boolean failOnMissingField = properties.getOptionalBoolean(FORMAT_FAIL_ON_MISSING_FIELD).orElse(false); + boolean ignoreParseErrors = properties.getOptionalBoolean(FORMAT_IGNORE_PARSE_ERRORS).orElse(false); + + RowType formatRowType = context.getFormatRowType(); + JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( + formatRowType, + new GenericTypeInfo(GenericRowData.class), + failOnMissingField, + ignoreParseErrors); + + String[] fieldNames = context.getSchema().getFieldNames(); + List projectFields = Arrays.stream(context.getProjectFields()) + .mapToObj(idx -> fieldNames[idx]) + .collect(Collectors.toList()); + List jsonFields = Arrays.stream(fieldNames) + .filter(field -> !context.getPartitionKeys().contains(field)) + .collect(Collectors.toList()); + + int[] jsonSelectFieldToProjectFieldMapping = context.getFormatProjectFields().stream() + .mapToInt(projectFields::indexOf) + .toArray(); + int[] jsonSelectFieldToJsonFieldMapping = context.getFormatProjectFields().stream() + .mapToInt(jsonFields::indexOf) + .toArray(); + + return new JsonInputFormat( + context.getPaths(), + context.getSchema().getFieldDataTypes(), + context.getSchema().getFieldNames(), + context.getProjectFields(), + context.getPartitionKeys(), + context.getDefaultPartName(), + context.getPushedDownLimit(), + jsonSelectFieldToProjectFieldMapping, + jsonSelectFieldToJsonFieldMapping, + deserializationSchema); + } + + @Override + public Optional> createEncoder(WriterContext context) { + return Optional.of(new JsonRowDataEncoder(new JsonRowDataSerializationSchema(context.getFormatRowType()))); + } + + @Override + public Optional> createBulkWriterFactory(WriterContext context) { + return Optional.empty(); + } + + @Override + public boolean supportsSchemaDerivation() { + return true; + } + + private static DescriptorProperties getValidatedProperties(Map propertiesMap) { + final DescriptorProperties properties = new DescriptorProperties(true); + properties.putProperties(propertiesMap); + properties.validateBoolean(FORMAT_FAIL_ON_MISSING_FIELD, true); + properties.validateBoolean(FORMAT_IGNORE_PARSE_ERRORS, true); + return properties; + } + + /** + * A {@link JsonInputFormat} is responsible to read {@link RowData} records + * from json format files. + */ + public static class JsonInputFormat extends DelimitedInputFormat { + /** + * Code of \r, used to remove \r from a line when the line ends with \r\n. + */ + private static final byte CARRIAGE_RETURN = (byte) '\r'; + + /** + * Code of \n, used to identify if \n is used as delimiter. + */ + private static final byte NEW_LINE = (byte) '\n'; + + private final DataType[] fieldTypes; + private final String[] fieldNames; + private final int[] selectFields; + private final List partitionKeys; + private final String defaultPartValue; + private final long limit; + private final int[] jsonSelectFieldToProjectFieldMapping; + private final int[] jsonSelectFieldToJsonFieldMapping; + private final JsonRowDataDeserializationSchema deserializationSchema; + + private transient boolean end; + private transient long emitted; + // reuse object for per record + private transient GenericRowData rowData; + + public JsonInputFormat( + Path[] filePaths, + DataType[] fieldTypes, + String[] fieldNames, + int[] selectFields, + List partitionKeys, + String defaultPartValue, + long limit, + int[] jsonSelectFieldToProjectFieldMapping, + int[] jsonSelectFieldToJsonFieldMapping, + JsonRowDataDeserializationSchema deserializationSchema) { + super.setFilePaths(filePaths); + this.fieldTypes = fieldTypes; + this.fieldNames = fieldNames; + this.selectFields = selectFields; + this.partitionKeys = partitionKeys; + this.defaultPartValue = defaultPartValue; + this.limit = limit; + this.jsonSelectFieldToProjectFieldMapping = jsonSelectFieldToProjectFieldMapping; + this.jsonSelectFieldToJsonFieldMapping = jsonSelectFieldToJsonFieldMapping; + this.deserializationSchema = deserializationSchema; + } + + @Override + public boolean supportsMultiPaths() { + return true; + } + + @Override + public void open(FileInputSplit split) throws IOException { + super.open(split); + this.end = false; + this.emitted = 0L; + this.rowData = PartitionPathUtils.fillPartitionValueForRecord(fieldNames, fieldTypes, selectFields, + partitionKeys, currentSplit.getPath(), defaultPartValue); + } + + @Override + public boolean reachedEnd() { + return emitted >= limit || end; + } + + @Override + public RowData readRecord(RowData reuse, byte[] bytes, int offset, int numBytes) throws IOException { + // remove \r from a line when the line ends with \r\n + if (this.getDelimiter() != null && this.getDelimiter().length == 1 + && this.getDelimiter()[0] == NEW_LINE && offset + numBytes >= 1 + && bytes[offset + numBytes - 1] == CARRIAGE_RETURN) { + numBytes -= 1; + } + byte[] trimBytes = Arrays.copyOfRange(bytes, offset, offset + numBytes); + GenericRowData jsonRow = (GenericRowData) deserializationSchema.deserialize(trimBytes); + + if (jsonRow == null) { + return null; + } + + GenericRowData returnRecord = rowData; + for (int i = 0; i < jsonSelectFieldToJsonFieldMapping.length; i++) { + returnRecord.setField(jsonSelectFieldToProjectFieldMapping[i], + jsonRow.getField(jsonSelectFieldToJsonFieldMapping[i])); + } + + emitted++; + return returnRecord; + } + + @Override + public RowData nextRecord(RowData record) throws IOException { + while (true) { + if (readLine()) { + RowData row = readRecord(record, this.currBuffer, this.currOffset, this.currLen); + if (row == null) { + continue; + } else { + return row; + } + } else { + this.end = true; + return null; + } + } + } + } + + /** + * A {@link JsonRowDataEncoder} is responsible to encode a {@link RowData} to {@link java.io.OutputStream} + * with json format. + */ + public static class JsonRowDataEncoder implements Encoder { + + private static final long serialVersionUID = 1L; + private static final String DEFAULT_LINE_DELIMITER = "\n"; + private final JsonRowDataSerializationSchema serializationSchema; + + public JsonRowDataEncoder(JsonRowDataSerializationSchema serializationSchema) { + this.serializationSchema = serializationSchema; + } + + @Override + public void encode(RowData element, OutputStream stream) throws IOException { + stream.write(serializationSchema.serialize(element)); + stream.write(DEFAULT_LINE_DELIMITER.getBytes(StandardCharsets.UTF_8)); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index aec584688..bc622f2bc 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -14,3 +14,4 @@ # limitations under the License. org.apache.flink.formats.json.JsonRowFormatFactory +org.apache.flink.formats.json.JsonFileSystemFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java new file mode 100644 index 000000000..2c01d29e1 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.table.planner.runtime.batch.sql.BatchFileSystemITCaseBase; +import org.apache.flink.types.Row; +import org.apache.flink.util.FileUtils; + +import org.junit.Test; + +import java.io.File; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * ITCase to test json format for {@link JsonFileSystemFormatFactory}. + */ +public class JsonBatchFileSystemITCase extends BatchFileSystemITCaseBase { + + @Override + public String[] formatProperties() { + List ret = new ArrayList<>(); + ret.add("'format'='json'"); + ret.add("'format.ignore-parse-errors'='true'"); + return ret.toArray(new String[0]); + } + + @Test + public void testParseError() throws Exception { + String path = new URI(resultPath()).getPath(); + new File(path).mkdirs(); + File file = new File(path, "temp_file"); + file.createNewFile(); + FileUtils.writeFileUtf8(file, + "{\"x\":\"x5\",\"y\":5,\"a\":1,\"b\":1}\n" + + "{I am a wrong json.}\n" + + "{\"x\":\"x5\",\"y\":5,\"a\":1,\"b\":1}"); + + check("select * from nonPartitionedTable", + Arrays.asList( + Row.of("x5,5,1,1"), + Row.of("x5,5,1,1"))); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java new file mode 100644 index 000000000..7690299cd --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.table.planner.runtime.stream.FsStreamingSinkITCaseBase; + +import java.util.ArrayList; +import java.util.List; + +/** + * Test checkpoint for file system table factory with json format. + */ +public class JsonFsStreamSinkITCase extends FsStreamingSinkITCaseBase { + + @Override + public String[] additionalProperties() { + List ret = new ArrayList<>(); + ret.add("'format'='json'"); + // for test purpose + ret.add("'sink.rolling-policy.file-size'='1'"); + return ret.toArray(new String[0]); + } +} From d4b482767dc04999a8fe103556de8912257974aa Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Fri, 15 May 2020 09:17:23 +0800 Subject: [PATCH 068/322] [FLINK-17629][json] Implement format factory for JSON serialization and deserialization schema This closes #12140 --- .../flink-json-debezium/pom.xml | 9 + .../flink/formats/json/JsonFormatFactory.java | 159 ++++++++++++++++ .../JsonRowDataDeserializationSchema.java | 20 ++ .../json/JsonRowDataSerializationSchema.java | 22 +++ .../org.apache.flink.table.factories.Factory | 16 ++ .../formats/json/JsonFormatFactoryTest.java | 175 ++++++++++++++++++ 6 files changed, 401 insertions(+) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 19d5045ce..0d3152505 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -100,6 +100,15 @@ under the License. scala-compiler test + + + + org.apache.flink + flink-core + ${project.version} + test + test-jar + diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java new file mode 100644 index 000000000..07e6d2d65 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.ScanFormat; +import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * Table format factory for providing configured instances of JSON to RowData + * {@link SerializationSchema} and {@link DeserializationSchema}. + */ +public class JsonFormatFactory implements + DeserializationFormatFactory, + SerializationFormatFactory { + + public static final String IDENTIFIER = "json"; + + // ------------------------------------------------------------------------ + // Options + // ------------------------------------------------------------------------ + + private static final ConfigOption FAIL_ON_MISSING_FIELD = ConfigOptions + .key("fail-on-missing-field") + .booleanType() + .defaultValue(false) + .withDescription("Optional flag to specify whether to fail if a field is missing or not, false by default"); + + private static final ConfigOption IGNORE_PARSE_ERRORS = ConfigOptions + .key("ignore-parse-errors") + .booleanType() + .defaultValue(false) + .withDescription("Optional flag to skip fields and rows with parse errors instead of failing;\n" + + "fields are set to null in case of errors, false by default"); + + @SuppressWarnings("unchecked") + @Override + public ScanFormat> createScanFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateFormatOptions(formatOptions); + + final boolean failOnMissingField = formatOptions.get(FAIL_ON_MISSING_FIELD); + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + + return new ScanFormat>() { + @Override + public DeserializationSchema createScanFormat( + ScanTableSource.Context scanContext, + DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + (TypeInformation) scanContext.createTypeInformation(producedDataType); + return new JsonRowDataDeserializationSchema( + rowType, + rowDataTypeInfo, + failOnMissingField, + ignoreParseErrors); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + }; + } + + @Override + public SinkFormat> createSinkFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + + return new SinkFormat>() { + @Override + public SerializationSchema createSinkFormat( + DynamicTableSink.Context context, + DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new JsonRowDataSerializationSchema(rowType); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(FAIL_ON_MISSING_FIELD); + options.add(IGNORE_PARSE_ERRORS); + return options; + } + + // ------------------------------------------------------------------------ + // Validation + // ------------------------------------------------------------------------ + + private void validateFormatOptions(ReadableConfig tableOptions) { + boolean failOnMissingField = tableOptions.get(FAIL_ON_MISSING_FIELD); + boolean ignoreParseErrors = tableOptions.get(IGNORE_PARSE_ERRORS); + if (ignoreParseErrors && failOnMissingField) { + throw new ValidationException(FAIL_ON_MISSING_FIELD.key() + + " and " + + IGNORE_PARSE_ERRORS.key() + + " shouldn't both be true."); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index 974b8dde3..87dee7f22 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -56,6 +56,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.Objects; import static java.lang.String.format; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; @@ -130,6 +131,25 @@ public TypeInformation getProducedType() { return resultTypeInfo; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JsonRowDataDeserializationSchema that = (JsonRowDataDeserializationSchema) o; + return failOnMissingField == that.failOnMissingField && + ignoreParseErrors == that.ignoreParseErrors && + resultTypeInfo.equals(that.resultTypeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(failOnMissingField, ignoreParseErrors, resultTypeInfo); + } + // ------------------------------------------------------------------------------------- // Runtime Converters // ------------------------------------------------------------------------------------- diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index 7c177386a..263e28255 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -42,6 +42,7 @@ import java.time.LocalDate; import java.time.LocalTime; import java.util.Arrays; +import java.util.Objects; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; @@ -59,6 +60,9 @@ public class JsonRowDataSerializationSchema implements SerializationSchema { private static final long serialVersionUID = 1L; + /** RowType to generate the runtime converter. */ + private final RowType rowType; + /** The converter that converts internal data formats to JsonNode. */ private final SerializationRuntimeConverter runtimeConverter; @@ -69,6 +73,7 @@ public class JsonRowDataSerializationSchema implements SerializationSchema tableOptions = getAllOptions(); + + testSchemaSerializationSchema(tableOptions); + + testSchemaDeserializationSchema(tableOptions); + } + + @Test + public void testFailOnMissingField() { + final Map tableOptions = getModifyOptions( + options -> options.put("json.fail-on-missing-field", "true")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new ValidationException("fail-on-missing-field and ignore-parse-errors shouldn't both be true."))); + testSchemaDeserializationSchema(tableOptions); + } + + @Test + public void testInvalidOptionForIgnoreParseErrors() { + final Map tableOptions = getModifyOptions( + options -> options.put("json.ignore-parse-errors", "abc")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new IllegalArgumentException("Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); + testSchemaDeserializationSchema(tableOptions); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + private void testSchemaDeserializationSchema(Map options) { + final JsonRowDataDeserializationSchema expectedDeser = + new JsonRowDataDeserializationSchema( + ROW_TYPE, + new RowDataTypeInfo(ROW_TYPE), + false, + true); + + final DynamicTableSource actualSource = createTableSource(options); + assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = scanSourceMock.sourceValueFormat + .createScanFormat( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toRowDataType()); + + assertEquals(expectedDeser, actualDeser); + } + + private void testSchemaSerializationSchema(Map options) { + final JsonRowDataSerializationSchema expectedSer = new JsonRowDataSerializationSchema(ROW_TYPE); + + final DynamicTableSink actualSink = createTableSink(options); + assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = sinkMock.sinkValueFormat + .createSinkFormat( + new SinkRuntimeProviderContext(false), + SCHEMA.toRowDataType()); + + assertEquals(expectedSer, actualSer); + } + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifyOptions(Consumer> optionModifier) { + Map options = getAllOptions(); + optionModifier.accept(options); + return options; + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", JsonFormatFactory.IDENTIFIER); + options.put("json.fail-on-missing-field", "false"); + options.put("json.ignore-parse-errors", "true"); + return options; + } + + private static DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "Mock scan table"), + new Configuration(), + JsonFormatFactoryTest.class.getClassLoader()); + } + + private static DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "Mock sink table"), + new Configuration(), + JsonFormatFactoryTest.class.getClassLoader()); + } +} From 415b3a03895885bec9b7bfb316cd6a394a56c06b Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Fri, 15 May 2020 23:07:03 +0800 Subject: [PATCH 069/322] [FLINK-17149][json][debezium] Introduce Debezium format to support reading debezium changelogs --- .../DebeziumJsonDeserializationSchema.java | 186 ++++++++++++++++++ .../debezium/DebeziumJsonFormatFactory.java | 126 ++++++++++++ .../org.apache.flink.table.factories.Factory | 1 + ...DebeziumJsonDeserializationSchemaTest.java | 162 +++++++++++++++ .../DebeziumJsonFormatFactoryTest.java | 145 ++++++++++++++ .../debezium-data-schema-exclude.txt | 16 ++ .../debezium-data-schema-include.txt | 16 ++ 7 files changed, 652 insertions(+) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-exclude.txt create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-include.txt diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java new file mode 100644 index 000000000..f0ae9d99d --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.debezium; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.Objects; + +import static java.lang.String.format; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Deserialization schema from Debezium JSON to Flink Table/SQL internal data structure {@link RowData}. + * The deserialization schema knows Debezium's schema definition and can extract the database data + * and convert into {@link RowData} with {@link RowKind}. + * + *

Deserializes a byte[] message as a JSON object and reads + * the specified fields. + * + *

Failures during deserialization are forwarded as wrapped IOExceptions. + * + * @see Debezium + */ +@Internal +public final class DebeziumJsonDeserializationSchema implements DeserializationSchema { + private static final long serialVersionUID = 1L; + + private static final String OP_READ = "r"; // snapshot read + private static final String OP_CREATE = "c"; // insert + private static final String OP_UPDATE = "u"; // update + private static final String OP_DELETE = "d"; // delete + + /** The deserializer to deserialize Debezium JSON data. */ + private final JsonRowDataDeserializationSchema jsonDeserializer; + + /** TypeInformation of the produced {@link RowData}. **/ + private final TypeInformation resultTypeInfo; + + /** + * Flag indicating whether the Debezium JSON data contains schema part or not. + * When Debezium Kafka Connect enables "value.converter.schemas.enable", the JSON + * will contain "schema" information, but we just ignore "schema" and extract data + * from "payload". + */ + private final boolean schemaInclude; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + public DebeziumJsonDeserializationSchema( + RowType rowType, + TypeInformation resultTypeInfo, + boolean schemaInclude, + boolean ignoreParseErrors) { + this.resultTypeInfo = resultTypeInfo; + this.schemaInclude = schemaInclude; + this.ignoreParseErrors = ignoreParseErrors; + this.jsonDeserializer = new JsonRowDataDeserializationSchema( + createJsonRowType(fromLogicalToDataType(rowType), schemaInclude), + // the result type is never used, so it's fine to pass in Debezium's result type + resultTypeInfo, + false, // ignoreParseErrors already contains the functionality of failOnMissingField + ignoreParseErrors); + } + + @Override + public RowData deserialize(byte[] message) throws IOException { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + + @Override + public void deserialize(byte[] message, Collector out) throws IOException { + try { + GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); + GenericRowData payload; + if (schemaInclude) { + payload = (GenericRowData) row.getField(0); + } else { + payload = row; + } + + GenericRowData before = (GenericRowData) payload.getField(0); + GenericRowData after = (GenericRowData) payload.getField(1); + String op = payload.getField(2).toString(); + if (OP_CREATE.equals(op) || OP_READ.equals(op)) { + after.setRowKind(RowKind.INSERT); + out.collect(after); + } else if (OP_UPDATE.equals(op)) { + before.setRowKind(RowKind.UPDATE_BEFORE); + after.setRowKind(RowKind.UPDATE_AFTER); + out.collect(before); + out.collect(after); + } else if (OP_DELETE.equals(op)) { + before.setRowKind(RowKind.DELETE); + out.collect(before); + } else { + if (!ignoreParseErrors) { + throw new IOException(format( + "Unknown \"op\" value \"%s\". The Debezium JSON message is '%s'", op, new String(message))); + } + } + } catch (Throwable t) { + // a big try catch to protect the processing. + if (!ignoreParseErrors) { + throw new IOException(format( + "Corrupt Debezium JSON message '%s'.", new String(message)), t); + } + } + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return resultTypeInfo; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DebeziumJsonDeserializationSchema that = (DebeziumJsonDeserializationSchema) o; + return schemaInclude == that.schemaInclude && + ignoreParseErrors == that.ignoreParseErrors && + Objects.equals(jsonDeserializer, that.jsonDeserializer) && + Objects.equals(resultTypeInfo, that.resultTypeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(jsonDeserializer, resultTypeInfo, schemaInclude, ignoreParseErrors); + } + + private static RowType createJsonRowType(DataType databaseSchema, boolean schemaInclude) { + DataType payload = DataTypes.ROW( + DataTypes.FIELD("before", databaseSchema), + DataTypes.FIELD("after", databaseSchema), + DataTypes.FIELD("op", DataTypes.STRING())); + if (schemaInclude) { + // when Debezium Kafka connect enables "value.converter.schemas.enable", + // the JSON will contain "schema" information, but we just ignore "schema" + // and extract data from "payload". + return (RowType) DataTypes.ROW( + DataTypes.FIELD("payload", payload)).getLogicalType(); + } else { + // payload contains some other information, e.g. "source", "ts_ms" + // but we don't need them. + return (RowType) payload.getLogicalType(); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java new file mode 100644 index 000000000..e3bb0a46d --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.debezium; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.ScanFormat; +import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * Format factory for providing configured instances of Debezium JSON to RowData {@link DeserializationSchema}. + */ +public class DebeziumJsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "debezium-json"; + + public static final ConfigOption SCHEMA_INCLUDE = ConfigOptions + .key("schema-include") + .booleanType() + .defaultValue(false) + .withDescription("When setting up a Debezium Kafka Connect, users can enable " + + "a Kafka configuration 'value.converter.schemas.enable' to include schema in the message. " + + "This option indicates the Debezium JSON data include the schema in the message or not. " + + "Default is false."); + + public static final ConfigOption IGNORE_PARSE_ERRORS = ConfigOptions + .key("ignore-parse-errors") + .booleanType() + .defaultValue(false) + .withDescription("Optional flag to skip fields and rows with parse errors instead of failing, " + + "fields are set to null in case of errors. Default is false."); + + @SuppressWarnings("unchecked") + @Override + public ScanFormat> createScanFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + final boolean schemaInclude = formatOptions.get(SCHEMA_INCLUDE); + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + + return new ScanFormat>() { + @Override + public DeserializationSchema createScanFormat(ScanTableSource.Context context, DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + (TypeInformation) context.createTypeInformation(producedDataType); + return new DebeziumJsonDeserializationSchema( + rowType, + rowDataTypeInfo, + schemaInclude, + ignoreParseErrors); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + }; + } + + @Override + public SinkFormat> createSinkFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + throw new UnsupportedOperationException("Debezium format doesn't support as a sink format yet."); + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(SCHEMA_INCLUDE); + options.add(IGNORE_PARSE_ERRORS); + return options; + } + +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 3a243d092..2b9ef3c9a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -14,3 +14,4 @@ # limitations under the License. org.apache.flink.formats.json.JsonFormatFactory +org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java new file mode 100644 index 000000000..ff7a3435a --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.debezium; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.FLOAT; +import static org.apache.flink.table.api.DataTypes.INT; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link DebeziumJsonDeserializationSchema}. + */ +public class DebeziumJsonDeserializationSchemaTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private static final RowType SCHEMA = (RowType) ROW( + FIELD("id", INT().notNull()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", FLOAT()) + ).getLogicalType(); + + @Test + public void testSchemaIncludeDeserialization() throws Exception { + testDeserialization("debezium-data-schema-include.txt", true); + } + + @Test + public void testSchemaExcludeDeserialization() throws Exception { + testDeserialization("debezium-data-schema-exclude.txt", false); + } + + private void testDeserialization(String resourceFile, boolean schemaInclude) throws Exception { + List lines = readLines(resourceFile); + DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( + SCHEMA, + new RowDataTypeInfo(SCHEMA), + schemaInclude, + false); + + SimpleCollector collector = new SimpleCollector(); + for (String line : lines) { + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + } + + // Debezium captures change data (`debezium-data-schema-include.txt`) on the `product` table: + // + // CREATE TABLE product ( + // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + // name VARCHAR(255), + // description VARCHAR(512), + // weight FLOAT + // ); + // ALTER TABLE product AUTO_INCREMENT = 101; + // + // INSERT INTO product + // VALUES (default,"scooter","Small 2-wheel scooter",3.14), + // (default,"car battery","12V car battery",8.1), + // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), + // (default,"hammer","12oz carpenter's hammer",0.75), + // (default,"hammer","14oz carpenter's hammer",0.875), + // (default,"hammer","16oz carpenter's hammer",1.0), + // (default,"rocks","box of assorted rocks",5.3), + // (default,"jacket","water resistent black wind breaker",0.1), + // (default,"spare tire","24 inch spare tire",22.2); + // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; + // UPDATE product SET weight='5.1' WHERE id=107; + // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); + // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); + // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110; + // UPDATE product SET weight='5.17' WHERE id=111; + // DELETE FROM product WHERE id=111; + List expected = Arrays.asList( + "+I(101,scooter,Small 2-wheel scooter,3.14)", + "+I(102,car battery,12V car battery,8.1)", + "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", + "+I(104,hammer,12oz carpenter's hammer,0.75)", + "+I(105,hammer,14oz carpenter's hammer,0.875)", + "+I(106,hammer,16oz carpenter's hammer,1.0)", + "+I(107,rocks,box of assorted rocks,5.3)", + "+I(108,jacket,water resistent black wind breaker,0.1)", + "+I(109,spare tire,24 inch spare tire,22.2)", + "-U(106,hammer,16oz carpenter's hammer,1.0)", + "+U(106,hammer,18oz carpenter hammer,1.0)", + "-U(107,rocks,box of assorted rocks,5.3)", + "+U(107,rocks,box of assorted rocks,5.1)", + "+I(110,jacket,water resistent white wind breaker,0.2)", + "+I(111,scooter,Big 2-wheel scooter ,5.18)", + "-U(110,jacket,water resistent white wind breaker,0.2)", + "+U(110,jacket,new water resistent white wind breaker,0.5)", + "-U(111,scooter,Big 2-wheel scooter ,5.18)", + "+U(111,scooter,Big 2-wheel scooter ,5.17)", + "-D(111,scooter,Big 2-wheel scooter ,5.17)" + ); + assertEquals(expected, collector.list); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private static List readLines(String resource) throws IOException { + final URL url = DebeziumJsonDeserializationSchemaTest.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllLines(path); + } + + private static class SimpleCollector implements Collector { + + private List list = new ArrayList<>(); + + @Override + public void collect(RowData record) { + list.add(record.toString()); + } + + @Override + public void close() { + // do nothing + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java new file mode 100644 index 000000000..9942bf708 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.debezium; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.Consumer; + +import static org.apache.flink.util.CoreMatchers.containsCause; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link DebeziumJsonFormatFactory}. + */ +public class DebeziumJsonFormatFactoryTest extends TestLogger { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + @Test + public void testSeDeSchema() { + final DebeziumJsonDeserializationSchema expectedDeser = new DebeziumJsonDeserializationSchema( + ROW_TYPE, + new RowDataTypeInfo(ROW_TYPE), + true, + true); + + final Map options = getAllOptions(); + + final DynamicTableSource actualSource = createTableSource(options); + assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = scanSourceMock.sourceValueFormat + .createScanFormat( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toRowDataType()); + + assertEquals(expectedDeser, actualDeser); + + thrown.expect(containsCause(new UnsupportedOperationException( + "Debezium format doesn't support as a sink format yet."))); + createTableSink(options); + } + + @Test + public void testInvalidIgnoreParseError() { + thrown.expect(containsCause(new IllegalArgumentException( + "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); + + final Map options = + getModifiedOptions(opts -> opts.put("debezium-json.ignore-parse-errors", "abc")); + + createTableSource(options); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifiedOptions(Consumer> optionModifier) { + Map options = getAllOptions(); + optionModifier.accept(options); + return options; + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", "debezium-json"); + options.put("debezium-json.ignore-parse-errors", "true"); + options.put("debezium-json.schema-include", "true"); + return options; + } + + private static DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + DebeziumJsonFormatFactoryTest.class.getClassLoader()); + } + + private static DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + DebeziumJsonFormatFactoryTest.class.getClassLoader()); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-exclude.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-exclude.txt new file mode 100644 index 000000000..3763369e4 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-exclude.txt @@ -0,0 +1,16 @@ +{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606100,"transaction":null} +{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589361987000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":362,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589361987936,"transaction":null} +{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362099000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362099505,"transaction":null} +{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362210230,"transaction":null} +{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362243000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1394,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362243428,"transaction":null} +{"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362293000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1707,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362293539,"transaction":null} +{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362330000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2090,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362330904,"transaction":null} +{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362344000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"d","ts_ms":1589362344455,"transaction":null} \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-include.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-include.txt new file mode 100644 index 000000000..b3e0f7d5b --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-include.txt @@ -0,0 +1,16 @@ +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606100,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589361987000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":362,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589361987936,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362099000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362099505,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362210230,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362243000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1394,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362243428,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362293000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1707,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362293539,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362330000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2090,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362330904,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362344000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"d","ts_ms":1589362344455,"transaction":null}} \ No newline at end of file From f17dcb38210278742aa46b1b4413f16997be21e5 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Fri, 15 May 2020 23:07:58 +0800 Subject: [PATCH 070/322] [FLINK-17150][json][canal] Introduce Canal format to support reading canal changelogs This closes #12152 --- .../canal/CanalJsonDeserializationSchema.java | 188 ++++++++++++++++++ .../json/canal/CanalJsonFormatFactory.java | 114 +++++++++++ .../org.apache.flink.table.factories.Factory | 1 + .../CanalJsonDeserializationSchemaTest.java | 160 +++++++++++++++ .../canal/CanalJsonFormatFactoryTest.java | 143 +++++++++++++ .../src/test/resources/canal-data.txt | 10 + 6 files changed, 616 insertions(+) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java new file mode 100644 index 000000000..4c03d041e --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.canal; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.Objects; + +import static java.lang.String.format; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Deserialization schema from Canal JSON to Flink Table/SQL internal data structure {@link RowData}. + * The deserialization schema knows Debezium's schema definition and can extract the database data + * and convert into {@link RowData} with {@link RowKind}. + * + *

Deserializes a byte[] message as a JSON object and reads + * the specified fields. + * + *

Failures during deserialization are forwarded as wrapped IOExceptions. + * + * @see Alibaba Canal + */ +public final class CanalJsonDeserializationSchema implements DeserializationSchema { + private static final long serialVersionUID = 1L; + + private static final String OP_INSERT = "INSERT"; + private static final String OP_UPDATE = "UPDATE"; + private static final String OP_DELETE = "DELETE"; + + /** The deserializer to deserialize Debezium JSON data. */ + private final JsonRowDataDeserializationSchema jsonDeserializer; + + /** TypeInformation of the produced {@link RowData}. **/ + private final TypeInformation resultTypeInfo; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + /** Number of fields. */ + private final int fieldCount; + + public CanalJsonDeserializationSchema( + RowType rowType, + TypeInformation resultTypeInfo, + boolean ignoreParseErrors) { + this.resultTypeInfo = resultTypeInfo; + this.ignoreParseErrors = ignoreParseErrors; + this.fieldCount = rowType.getFieldCount(); + this.jsonDeserializer = new JsonRowDataDeserializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + // the result type is never used, so it's fine to pass in Canal's result type + resultTypeInfo, + false, // ignoreParseErrors already contains the functionality of failOnMissingField + ignoreParseErrors); + + } + + @Override + public RowData deserialize(byte[] message) throws IOException { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + + @Override + public void deserialize(byte[] message, Collector out) throws IOException { + try { + RowData row = jsonDeserializer.deserialize(message); + String type = row.getString(2).toString(); // "type" field + if (OP_INSERT.equals(type)) { + // "data" field is an array of row, contains inserted rows + ArrayData data = row.getArray(0); + for (int i = 0; i < data.size(); i++) { + RowData insert = data.getRow(i, fieldCount); + insert.setRowKind(RowKind.INSERT); + out.collect(insert); + } + } else if (OP_UPDATE.equals(type)) { + // "data" field is an array of row, contains new rows + ArrayData data = row.getArray(0); + // "old" field is an array of row, contains old values + ArrayData old = row.getArray(1); + for (int i = 0; i < data.size(); i++) { + // the underlying JSON deserialization schema always produce GenericRowData. + GenericRowData after = (GenericRowData) data.getRow(i, fieldCount); + GenericRowData before = (GenericRowData) old.getRow(i, fieldCount); + for (int f = 0; f < fieldCount; f++) { + if (before.isNullAt(f)) { + // not null fields in "old" (before) means the fields are changed + // null/empty fields in "old" (before) means the fields are not changed + // so we just copy the not changed fields into before + before.setField(f, after.getField(f)); + } + } + before.setRowKind(RowKind.UPDATE_BEFORE); + after.setRowKind(RowKind.UPDATE_AFTER); + out.collect(before); + out.collect(after); + } + } else if (OP_DELETE.equals(type)) { + // "data" field is an array of row, contains deleted rows + ArrayData data = row.getArray(0); + for (int i = 0; i < data.size(); i++) { + RowData insert = data.getRow(i, fieldCount); + insert.setRowKind(RowKind.DELETE); + out.collect(insert); + } + } else { + if (!ignoreParseErrors) { + throw new IOException(format( + "Unknown \"type\" value \"%s\". The Canal JSON message is '%s'", type, new String(message))); + } + } + } catch (Throwable t) { + // a big try catch to protect the processing. + if (!ignoreParseErrors) { + throw new IOException(format( + "Corrupt Canal JSON message '%s'.", new String(message)), t); + } + } + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return resultTypeInfo; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CanalJsonDeserializationSchema that = (CanalJsonDeserializationSchema) o; + return ignoreParseErrors == that.ignoreParseErrors && + fieldCount == that.fieldCount && + Objects.equals(jsonDeserializer, that.jsonDeserializer) && + Objects.equals(resultTypeInfo, that.resultTypeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors, fieldCount); + } + + private static RowType createJsonRowType(DataType databaseSchema) { + // Canal JSON contains other information, e.g. "database", "ts" + // but we don't need them + return (RowType) DataTypes.ROW( + DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)), + DataTypes.FIELD("old", DataTypes.ARRAY(databaseSchema)), + DataTypes.FIELD("type", DataTypes.STRING())).getLogicalType(); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java new file mode 100644 index 000000000..1170953e3 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.canal; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.ScanFormat; +import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * Format factory for providing configured instances of Canal JSON to RowData {@link DeserializationSchema}. + */ +public class CanalJsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "canal-json"; + + public static final ConfigOption IGNORE_PARSE_ERRORS = ConfigOptions + .key("ignore-parse-errors") + .booleanType() + .defaultValue(false) + .withDescription("Optional flag to skip fields and rows with parse errors instead of failing, " + + "fields are set to null in case of errors. Default is false."); + + @SuppressWarnings("unchecked") + @Override + public ScanFormat> createScanFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + + return new ScanFormat>() { + @Override + public DeserializationSchema createScanFormat(ScanTableSource.Context context, DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + (TypeInformation) context.createTypeInformation(producedDataType); + return new CanalJsonDeserializationSchema( + rowType, + rowDataTypeInfo, + ignoreParseErrors); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + }; + } + + @Override + public SinkFormat> createSinkFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + throw new UnsupportedOperationException("Canal format doesn't support as a sink format yet."); + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(IGNORE_PARSE_ERRORS); + return options; + } + +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 2b9ef3c9a..5349ab10c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -15,3 +15,4 @@ org.apache.flink.formats.json.JsonFormatFactory org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory +org.apache.flink.formats.json.canal.CanalJsonFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java new file mode 100644 index 000000000..02f055ea1 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.canal; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.FLOAT; +import static org.apache.flink.table.api.DataTypes.INT; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link CanalJsonDeserializationSchema}. + */ +public class CanalJsonDeserializationSchemaTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private static final RowType SCHEMA = (RowType) ROW( + FIELD("id", INT().notNull()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", FLOAT()) + ).getLogicalType(); + + @Test + public void testDeserialization() throws Exception { + List lines = readLines("canal-data.txt"); + CanalJsonDeserializationSchema deserializationSchema = new CanalJsonDeserializationSchema( + SCHEMA, + new RowDataTypeInfo(SCHEMA), + false); + + SimpleCollector collector = new SimpleCollector(); + for (String line : lines) { + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + } + + // Canal captures change data (`canal-data.txt`) on the `product` table: + // + // CREATE TABLE product ( + // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + // name VARCHAR(255), + // description VARCHAR(512), + // weight FLOAT + // ); + // ALTER TABLE product AUTO_INCREMENT = 101; + // + // INSERT INTO product + // VALUES (default,"scooter","Small 2-wheel scooter",3.14), + // (default,"car battery","12V car battery",8.1), + // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), + // (default,"hammer","12oz carpenter's hammer",0.75), + // (default,"hammer","14oz carpenter's hammer",0.875), + // (default,"hammer","16oz carpenter's hammer",1.0), + // (default,"rocks","box of assorted rocks",5.3), + // (default,"jacket","water resistent black wind breaker",0.1), + // (default,"spare tire","24 inch spare tire",22.2); + // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; + // UPDATE product SET weight='5.1' WHERE id=107; + // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); + // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); + // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110; + // UPDATE product SET weight='5.17' WHERE id=111; + // DELETE FROM product WHERE id=111; + // UPDATE products SET weight='5.17' WHERE id=102 or id = 101; + // DELETE FROM products WHERE id=102 or id = 103; + List expected = Arrays.asList( + "+I(101,scooter,Small 2-wheel scooter,3.14)", + "+I(102,car battery,12V car battery,8.1)", + "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", + "+I(104,hammer,12oz carpenter's hammer,0.75)", + "+I(105,hammer,14oz carpenter's hammer,0.875)", + "+I(106,hammer,16oz carpenter's hammer,1.0)", + "+I(107,rocks,box of assorted rocks,5.3)", + "+I(108,jacket,water resistent black wind breaker,0.1)", + "+I(109,spare tire,24 inch spare tire,22.2)", + "-U(106,hammer,16oz carpenter's hammer,1.0)", + "+U(106,hammer,18oz carpenter hammer,1.0)", + "-U(107,rocks,box of assorted rocks,5.3)", + "+U(107,rocks,box of assorted rocks,5.1)", + "+I(110,jacket,water resistent white wind breaker,0.2)", + "+I(111,scooter,Big 2-wheel scooter ,5.18)", + "-U(110,jacket,water resistent white wind breaker,0.2)", + "+U(110,jacket,new water resistent white wind breaker,0.5)", + "-U(111,scooter,Big 2-wheel scooter ,5.18)", + "+U(111,scooter,Big 2-wheel scooter ,5.17)", + "-D(111,scooter,Big 2-wheel scooter ,5.17)", + "-U(101,scooter,Small 2-wheel scooter,3.14)", + "+U(101,scooter,Small 2-wheel scooter,5.17)", + "-U(102,car battery,12V car battery,8.1)", + "+U(102,car battery,12V car battery,5.17)", + "-D(102,car battery,12V car battery,5.17)", + "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)" + ); + assertEquals(expected, collector.list); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private static List readLines(String resource) throws IOException { + final URL url = CanalJsonDeserializationSchemaTest.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllLines(path); + } + + private static class SimpleCollector implements Collector { + + private List list = new ArrayList<>(); + + @Override + public void collect(RowData record) { + list.add(record.toString()); + } + + @Override + public void close() { + // do nothing + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java new file mode 100644 index 000000000..b5ebfc0ff --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.canal; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.Consumer; + +import static org.apache.flink.util.CoreMatchers.containsCause; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link CanalJsonFormatFactory}. + */ +public class CanalJsonFormatFactoryTest extends TestLogger { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + @Test + public void testSeDeSchema() { + final CanalJsonDeserializationSchema expectedDeser = new CanalJsonDeserializationSchema( + ROW_TYPE, + new RowDataTypeInfo(ROW_TYPE), + true); + + final Map options = getAllOptions(); + + final DynamicTableSource actualSource = createTableSource(options); + assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = scanSourceMock.sourceValueFormat + .createScanFormat( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toRowDataType()); + + assertEquals(expectedDeser, actualDeser); + + thrown.expect(containsCause(new UnsupportedOperationException( + "Canal format doesn't support as a sink format yet."))); + createTableSink(options); + } + + @Test + public void testInvalidIgnoreParseError() { + thrown.expect(containsCause(new IllegalArgumentException( + "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); + + final Map options = + getModifiedOptions(opts -> opts.put("canal-json.ignore-parse-errors", "abc")); + + createTableSource(options); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifiedOptions(Consumer> optionModifier) { + Map options = getAllOptions(); + optionModifier.accept(options); + return options; + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", "canal-json"); + options.put("canal-json.ignore-parse-errors", "true"); + return options; + } + + private static DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + CanalJsonFormatFactoryTest.class.getClassLoader()); + } + + private static DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + CanalJsonFormatFactoryTest.class.getClassLoader()); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt new file mode 100644 index 000000000..e81fad25c --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt @@ -0,0 +1,10 @@ +{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"106","name":"hammer","description":"16oz carpenter's hammer","weight":"1.0"},{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"inventory","es":1589373515000,"id":3,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373515477,"type":"INSERT"} +{"data":[{"id":"106","name":"hammer","description":"18oz carpenter hammer","weight":"1.0"}],"database":"inventory","es":1589373546000,"id":4,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"description":"16oz carpenter's hammer"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373546301,"type":"UPDATE"} +{"data":[{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.1"}],"database":"inventory","es":1589373549000,"id":5,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373549489,"type":"UPDATE"} +{"data":[{"id":"110","name":"jacket","description":"water resistent white wind breaker","weight":"0.2"}],"database":"inventory","es":1589373552000,"id":6,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373552882,"type":"INSERT"} +{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.18"}],"database":"inventory","es":1589373555000,"id":7,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373555457,"type":"INSERT"} +{"data":[{"id":"110","name":"jacket","description":"new water resistent white wind breaker","weight":"0.5"}],"database":"inventory","es":1589373558000,"id":8,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"description":"water resistent white wind breaker","weight":"0.2"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373558230,"type":"UPDATE"} +{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"inventory","es":1589373560000,"id":9,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"5.18"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373560798,"type":"UPDATE"} +{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"inventory","es":1589373563000,"id":10,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373563798,"type":"DELETE"} +{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"5.17"},{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"}],"database":"inventory","es":1589373753000,"id":11,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"3.14"},{"weight":"8.1"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373753939,"type":"UPDATE"} +{"data":[{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"}],"database":"inventory","es":1589374013000,"id":12,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589374013680,"type":"DELETE"} \ No newline at end of file From 2956192e8f95048a051ec92d6aec1ca7f6d33560 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 18 May 2020 13:36:46 +0200 Subject: [PATCH 071/322] Update version to 1.12-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 6e96bff0c..a143bc5b6 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.11-SNAPSHOT + 1.12-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 289d10003..8e39402d0 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.11-SNAPSHOT + 1.12-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 0d3152505..b19b701ea 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. From 541477a4ee396c8f266094589b4029d6914c7fb8 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 19 May 2020 18:23:17 +0800 Subject: [PATCH 072/322] [FLINK-17626][fs-connector] Fs connector should use FLIP-122 format options style This closes #12212 --- .../json/JsonFileSystemFormatFactory.java | 57 ++++++++----------- .../flink/formats/json/JsonFormatFactory.java | 23 ++------ .../flink/formats/json/JsonOptions.java | 41 +++++++++++++ .../org.apache.flink.table.factories.Factory | 1 + ....apache.flink.table.factories.TableFactory | 1 - .../json/JsonBatchFileSystemITCase.java | 2 +- 6 files changed, 72 insertions(+), 53 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java index 9a2be3140..d464634e4 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java @@ -23,11 +23,12 @@ import org.apache.flink.api.common.serialization.BulkWriter; import org.apache.flink.api.common.serialization.Encoder; import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.fs.FileInputSplit; import org.apache.flink.core.fs.Path; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.factories.FileSystemFormatFactory; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; @@ -36,43 +37,48 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.charset.StandardCharsets; -import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; +import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; -import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; -import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD; -import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.json.JsonFormatFactory.validateFormatOptions; +import static org.apache.flink.formats.json.JsonOptions.FAIL_ON_MISSING_FIELD; +import static org.apache.flink.formats.json.JsonOptions.IGNORE_PARSE_ERRORS; /** * Factory to build reader/writer to read/write json format file. */ public class JsonFileSystemFormatFactory implements FileSystemFormatFactory { + public static final String IDENTIFIER = "json"; + @Override - public Map requiredContext() { - Map context = new HashMap<>(); - context.put(FORMAT, "json"); - return context; + public String factoryIdentifier() { + return IDENTIFIER; } @Override - public List supportedProperties() { - ArrayList properties = new ArrayList<>(); - properties.add(FORMAT_FAIL_ON_MISSING_FIELD); - properties.add(FORMAT_IGNORE_PARSE_ERRORS); - return properties; + public Set> requiredOptions() { + return new HashSet<>(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(FAIL_ON_MISSING_FIELD); + options.add(IGNORE_PARSE_ERRORS); + return options; } @Override public InputFormat createReader(ReaderContext context) { - DescriptorProperties properties = getValidatedProperties(context.getFormatProperties()); - boolean failOnMissingField = properties.getOptionalBoolean(FORMAT_FAIL_ON_MISSING_FIELD).orElse(false); - boolean ignoreParseErrors = properties.getOptionalBoolean(FORMAT_IGNORE_PARSE_ERRORS).orElse(false); + ReadableConfig options = context.getFormatOptions(); + validateFormatOptions(options); + boolean failOnMissingField = options.get(FAIL_ON_MISSING_FIELD); + boolean ignoreParseErrors = options.get(IGNORE_PARSE_ERRORS); RowType formatRowType = context.getFormatRowType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( @@ -119,19 +125,6 @@ public Optional> createBulkWriterFactory(WriterConte return Optional.empty(); } - @Override - public boolean supportsSchemaDerivation() { - return true; - } - - private static DescriptorProperties getValidatedProperties(Map propertiesMap) { - final DescriptorProperties properties = new DescriptorProperties(true); - properties.putProperties(propertiesMap); - properties.validateBoolean(FORMAT_FAIL_ON_MISSING_FIELD, true); - properties.validateBoolean(FORMAT_IGNORE_PARSE_ERRORS, true); - return properties; - } - /** * A {@link JsonInputFormat} is responsible to read {@link RowData} records * from json format files. diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index 07e6d2d65..ca8015911 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.ChangelogMode; @@ -42,6 +41,9 @@ import java.util.HashSet; import java.util.Set; +import static org.apache.flink.formats.json.JsonOptions.FAIL_ON_MISSING_FIELD; +import static org.apache.flink.formats.json.JsonOptions.IGNORE_PARSE_ERRORS; + /** * Table format factory for providing configured instances of JSON to RowData * {@link SerializationSchema} and {@link DeserializationSchema}. @@ -52,23 +54,6 @@ public class JsonFormatFactory implements public static final String IDENTIFIER = "json"; - // ------------------------------------------------------------------------ - // Options - // ------------------------------------------------------------------------ - - private static final ConfigOption FAIL_ON_MISSING_FIELD = ConfigOptions - .key("fail-on-missing-field") - .booleanType() - .defaultValue(false) - .withDescription("Optional flag to specify whether to fail if a field is missing or not, false by default"); - - private static final ConfigOption IGNORE_PARSE_ERRORS = ConfigOptions - .key("ignore-parse-errors") - .booleanType() - .defaultValue(false) - .withDescription("Optional flag to skip fields and rows with parse errors instead of failing;\n" - + "fields are set to null in case of errors, false by default"); - @SuppressWarnings("unchecked") @Override public ScanFormat> createScanFormat( @@ -146,7 +131,7 @@ public Set> optionalOptions() { // Validation // ------------------------------------------------------------------------ - private void validateFormatOptions(ReadableConfig tableOptions) { + static void validateFormatOptions(ReadableConfig tableOptions) { boolean failOnMissingField = tableOptions.get(FAIL_ON_MISSING_FIELD); boolean ignoreParseErrors = tableOptions.get(IGNORE_PARSE_ERRORS); if (ignoreParseErrors && failOnMissingField) { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java new file mode 100644 index 000000000..dca8c16c5 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** + * This class holds configuration constants used by json format. + */ +public class JsonOptions { + + public static final ConfigOption FAIL_ON_MISSING_FIELD = ConfigOptions + .key("fail-on-missing-field") + .booleanType() + .defaultValue(false) + .withDescription("Optional flag to specify whether to fail if a field is missing or not, false by default"); + + public static final ConfigOption IGNORE_PARSE_ERRORS = ConfigOptions + .key("ignore-parse-errors") + .booleanType() + .defaultValue(false) + .withDescription("Optional flag to skip fields and rows with parse errors instead of failing;\n" + + "fields are set to null in case of errors, false by default"); +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 5349ab10c..781a963f7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,6 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. +org.apache.flink.formats.json.JsonFileSystemFormatFactory org.apache.flink.formats.json.JsonFormatFactory org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory org.apache.flink.formats.json.canal.CanalJsonFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index bc622f2bc..aec584688 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -14,4 +14,3 @@ # limitations under the License. org.apache.flink.formats.json.JsonRowFormatFactory -org.apache.flink.formats.json.JsonFileSystemFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java index 2c01d29e1..f1ee52b33 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java @@ -39,7 +39,7 @@ public class JsonBatchFileSystemITCase extends BatchFileSystemITCaseBase { public String[] formatProperties() { List ret = new ArrayList<>(); ret.add("'format'='json'"); - ret.add("'format.ignore-parse-errors'='true'"); + ret.add("'json.ignore-parse-errors'='true'"); return ret.toArray(new String[0]); } From 20223deef1e6bbeeb3adffb0fa8ed6a2aab1adf1 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 27 May 2020 19:19:56 +0800 Subject: [PATCH 073/322] [FLINK-17925][fs-connector] Fix Filesystem options to default values and types This closes #12323 --- .../org/apache/flink/formats/json/JsonFsStreamSinkITCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java index 7690299cd..f2cd6d4d0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java @@ -33,7 +33,7 @@ public String[] additionalProperties() { List ret = new ArrayList<>(); ret.add("'format'='json'"); // for test purpose - ret.add("'sink.rolling-policy.file-size'='1'"); + ret.add("'sink.rolling-policy.file-size'='1b'"); return ret.toArray(new String[0]); } } From b77f97c2f4e0d49d046826f81789fdc33f04b655 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 29 May 2020 10:04:39 +0200 Subject: [PATCH 074/322] [hotfix][tests] Move containsCause to FlinkMatchers --- flink-formats-kafka/flink-json-debezium/pom.xml | 9 --------- .../apache/flink/formats/json/JsonFormatFactoryTest.java | 2 +- .../formats/json/canal/CanalJsonFormatFactoryTest.java | 2 +- .../json/debezium/DebeziumJsonFormatFactoryTest.java | 2 +- 4 files changed, 3 insertions(+), 12 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index b19b701ea..67e2a8058 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -100,15 +100,6 @@ under the License. scala-compiler test - - - - org.apache.flink - flink-core - ${project.version} - test - test-jar - diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index 7638378bc..d54daa687 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -45,7 +45,7 @@ import java.util.Map; import java.util.function.Consumer; -import static org.apache.flink.util.CoreMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.junit.Assert.assertEquals; /** diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index b5ebfc0ff..e6758739c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -42,7 +42,7 @@ import java.util.Map; import java.util.function.Consumer; -import static org.apache.flink.util.CoreMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.junit.Assert.assertEquals; /** diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 9942bf708..8eb6d9b54 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -42,7 +42,7 @@ import java.util.Map; import java.util.function.Consumer; -import static org.apache.flink.util.CoreMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.junit.Assert.assertEquals; /** From 3a9bc1375f7800dd1f70c5e63769a90605aa8b88 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Mon, 1 Jun 2020 10:32:40 +0800 Subject: [PATCH 075/322] [FLINK-17887][table][connector] Improve interface of ScanFormatFactory and SinkFormatFactory We improved the interfaces with the following changes: 1. Have a common interface DynamicTableSource.Context, and make Context of ScanTableSource and LookupTableSource extend it, and rename them to LookupContext and ScanContext 2. Change parameter of ScanFormat.createScanFormat from ScanTableSource.Context to DynamicTableSource.Context 3. Rename ScanFormat.createScanFormat to DecodingFormat#createRuntimeDecoder() 4. Rename SinkFormat.createSinkFormat to EncodingFormat#createRuntimeEncoder() 5. Rename ScanFormatFactory to DecodingFormatFactory 6. Rename SinkFormatFactory to EncodingFormatFactory This closes #12320 --- .../flink/formats/json/JsonFormatFactory.java | 22 +++++++++---------- .../json/canal/CanalJsonFormatFactory.java | 15 +++++++------ .../debezium/DebeziumJsonFormatFactory.java | 15 +++++++------ .../formats/json/JsonFormatFactoryTest.java | 8 +++---- .../canal/CanalJsonFormatFactoryTest.java | 4 ++-- .../DebeziumJsonFormatFactoryTest.java | 4 ++-- 6 files changed, 35 insertions(+), 33 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index ca8015911..ba9886701 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -25,10 +25,10 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.ScanFormat; -import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DeserializationFormatFactory; import org.apache.flink.table.factories.DynamicTableFactory; @@ -56,7 +56,7 @@ public class JsonFormatFactory implements @SuppressWarnings("unchecked") @Override - public ScanFormat> createScanFormat( + public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); @@ -65,14 +65,14 @@ public ScanFormat> createScanFormat( final boolean failOnMissingField = formatOptions.get(FAIL_ON_MISSING_FIELD); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - return new ScanFormat>() { + return new DecodingFormat>() { @Override - public DeserializationSchema createScanFormat( - ScanTableSource.Context scanContext, + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = - (TypeInformation) scanContext.createTypeInformation(producedDataType); + (TypeInformation) context.createTypeInformation(producedDataType); return new JsonRowDataDeserializationSchema( rowType, rowDataTypeInfo, @@ -88,14 +88,14 @@ public ChangelogMode getChangelogMode() { } @Override - public SinkFormat> createSinkFormat( + public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - return new SinkFormat>() { + return new EncodingFormat>() { @Override - public SerializationSchema createSinkFormat( + public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java index 1170953e3..fba98bf48 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java @@ -25,9 +25,9 @@ import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.ScanFormat; -import org.apache.flink.table.connector.format.SinkFormat; -import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DeserializationFormatFactory; import org.apache.flink.table.factories.DynamicTableFactory; @@ -57,15 +57,16 @@ public class CanalJsonFormatFactory implements DeserializationFormatFactory, Ser @SuppressWarnings("unchecked") @Override - public ScanFormat> createScanFormat( + public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - return new ScanFormat>() { + return new DecodingFormat>() { @Override - public DeserializationSchema createScanFormat(ScanTableSource.Context context, DataType producedDataType) { + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = (TypeInformation) context.createTypeInformation(producedDataType); @@ -88,7 +89,7 @@ public ChangelogMode getChangelogMode() { } @Override - public SinkFormat> createSinkFormat( + public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { throw new UnsupportedOperationException("Canal format doesn't support as a sink format yet."); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java index e3bb0a46d..3458014ec 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -25,9 +25,9 @@ import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.ScanFormat; -import org.apache.flink.table.connector.format.SinkFormat; -import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DeserializationFormatFactory; import org.apache.flink.table.factories.DynamicTableFactory; @@ -66,16 +66,17 @@ public class DebeziumJsonFormatFactory implements DeserializationFormatFactory, @SuppressWarnings("unchecked") @Override - public ScanFormat> createScanFormat( + public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); final boolean schemaInclude = formatOptions.get(SCHEMA_INCLUDE); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - return new ScanFormat>() { + return new DecodingFormat>() { @Override - public DeserializationSchema createScanFormat(ScanTableSource.Context context, DataType producedDataType) { + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = (TypeInformation) context.createTypeInformation(producedDataType); @@ -99,7 +100,7 @@ public ChangelogMode getChangelogMode() { } @Override - public SinkFormat> createSinkFormat( + public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { throw new UnsupportedOperationException("Debezium format doesn't support as a sink format yet."); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index d54daa687..8ddf03ea7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -108,8 +108,8 @@ private void testSchemaDeserializationSchema(Map options) { TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - DeserializationSchema actualDeser = scanSourceMock.sourceValueFormat - .createScanFormat( + DeserializationSchema actualDeser = scanSourceMock.valueFormat + .createRuntimeDecoder( ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); @@ -124,8 +124,8 @@ private void testSchemaSerializationSchema(Map options) { TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - SerializationSchema actualSer = sinkMock.sinkValueFormat - .createSinkFormat( + SerializationSchema actualSer = sinkMock.valueFormat + .createRuntimeEncoder( new SinkRuntimeProviderContext(false), SCHEMA.toRowDataType()); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index e6758739c..83afa113d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -74,8 +74,8 @@ public void testSeDeSchema() { TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - DeserializationSchema actualDeser = scanSourceMock.sourceValueFormat - .createScanFormat( + DeserializationSchema actualDeser = scanSourceMock.valueFormat + .createRuntimeDecoder( ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 8eb6d9b54..869ff2812 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -75,8 +75,8 @@ public void testSeDeSchema() { TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - DeserializationSchema actualDeser = scanSourceMock.sourceValueFormat - .createScanFormat( + DeserializationSchema actualDeser = scanSourceMock.valueFormat + .createRuntimeDecoder( ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); From bd6467aef638ecdb615747bf8bccb182bd1b6aaa Mon Sep 17 00:00:00 2001 From: Shengkai <33114724+fsk119@users.noreply.github.com> Date: Wed, 17 Jun 2020 21:32:10 +0800 Subject: [PATCH 076/322] [FLINK-18299][json] Fix the non SQL standard timestamp format in JSON format The current timestamp format in JSON format is not SQL standard which uses RFC-3339. This commit changes the default behavior to parse/generate timestamp using SQL standard. Besides, it introduces an option "json.timestamp-format.standard" to have the ability to fallback to ISO standard. This closes #12661 --- .../json/JsonFileSystemFormatFactory.java | 9 +- .../flink/formats/json/JsonFormatFactory.java | 17 +++- .../flink/formats/json/JsonOptions.java | 43 +++++++++ .../JsonRowDataDeserializationSchema.java | 54 +++++------ .../json/JsonRowDataSerializationSchema.java | 40 ++++++--- .../flink/formats/json/TimeFormats.java | 18 +++- .../flink/formats/json/TimestampFormat.java | 35 ++++++++ .../canal/CanalJsonDeserializationSchema.java | 7 +- .../json/canal/CanalJsonFormatFactory.java | 17 ++-- .../DebeziumJsonDeserializationSchema.java | 7 +- .../debezium/DebeziumJsonFormatFactory.java | 16 ++-- .../formats/json/JsonFormatFactoryTest.java | 26 +++++- .../json/JsonRowDataSerDeSchemaTest.java | 89 +++++++++++++++---- .../CanalJsonDeserializationSchemaTest.java | 4 +- .../canal/CanalJsonFormatFactoryTest.java | 5 +- ...DebeziumJsonDeserializationSchemaTest.java | 4 +- .../DebeziumJsonFormatFactoryTest.java | 5 +- 17 files changed, 308 insertions(+), 88 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java index d464634e4..1ecdc735d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java @@ -47,6 +47,7 @@ import static org.apache.flink.formats.json.JsonFormatFactory.validateFormatOptions; import static org.apache.flink.formats.json.JsonOptions.FAIL_ON_MISSING_FIELD; import static org.apache.flink.formats.json.JsonOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.json.JsonOptions.TIMESTAMP_FORMAT; /** * Factory to build reader/writer to read/write json format file. @@ -70,6 +71,7 @@ public Set> optionalOptions() { Set> options = new HashSet<>(); options.add(FAIL_ON_MISSING_FIELD); options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); return options; } @@ -79,13 +81,15 @@ public Set> optionalOptions() { validateFormatOptions(options); boolean failOnMissingField = options.get(FAIL_ON_MISSING_FIELD); boolean ignoreParseErrors = options.get(IGNORE_PARSE_ERRORS); + TimestampFormat timestampOption = JsonOptions.getTimestampFormat(options); RowType formatRowType = context.getFormatRowType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( formatRowType, new GenericTypeInfo(GenericRowData.class), failOnMissingField, - ignoreParseErrors); + ignoreParseErrors, + timestampOption); String[] fieldNames = context.getSchema().getFieldNames(); List projectFields = Arrays.stream(context.getProjectFields()) @@ -117,7 +121,8 @@ public Set> optionalOptions() { @Override public Optional> createEncoder(WriterContext context) { - return Optional.of(new JsonRowDataEncoder(new JsonRowDataSerializationSchema(context.getFormatRowType()))); + return Optional.of(new JsonRowDataEncoder(new JsonRowDataSerializationSchema(context.getFormatRowType(), + JsonOptions.getTimestampFormat(context.getFormatOptions())))); } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index ba9886701..57952b920 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -43,6 +43,8 @@ import static org.apache.flink.formats.json.JsonOptions.FAIL_ON_MISSING_FIELD; import static org.apache.flink.formats.json.JsonOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.json.JsonOptions.TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.JsonOptions.TIMESTAMP_FORMAT_ENUM; /** * Table format factory for providing configured instances of JSON to RowData @@ -64,6 +66,7 @@ public DecodingFormat> createDecodingFormat( final boolean failOnMissingField = formatOptions.get(FAIL_ON_MISSING_FIELD); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + TimestampFormat timestampOption = JsonOptions.getTimestampFormat(formatOptions); return new DecodingFormat>() { @Override @@ -77,7 +80,9 @@ public DeserializationSchema createRuntimeDecoder( rowType, rowDataTypeInfo, failOnMissingField, - ignoreParseErrors); + ignoreParseErrors, + timestampOption + ); } @Override @@ -93,13 +98,15 @@ public EncodingFormat> createEncodingFormat( ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); + TimestampFormat timestampOption = JsonOptions.getTimestampFormat(formatOptions); + return new EncodingFormat>() { @Override public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new JsonRowDataSerializationSchema(rowType); + return new JsonRowDataSerializationSchema(rowType, timestampOption); } @Override @@ -124,6 +131,7 @@ public Set> optionalOptions() { Set> options = new HashSet<>(); options.add(FAIL_ON_MISSING_FIELD); options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); return options; } @@ -134,11 +142,16 @@ public Set> optionalOptions() { static void validateFormatOptions(ReadableConfig tableOptions) { boolean failOnMissingField = tableOptions.get(FAIL_ON_MISSING_FIELD); boolean ignoreParseErrors = tableOptions.get(IGNORE_PARSE_ERRORS); + String timestampFormat = tableOptions.get(TIMESTAMP_FORMAT); if (ignoreParseErrors && failOnMissingField) { throw new ValidationException(FAIL_ON_MISSING_FIELD.key() + " and " + IGNORE_PARSE_ERRORS.key() + " shouldn't both be true."); } + if (!TIMESTAMP_FORMAT_ENUM.contains(timestampFormat)){ + throw new ValidationException(String.format("Unsupported value '%s' for %s. Supported values are [SQL, ISO-8601].", + timestampFormat, TIMESTAMP_FORMAT.key())); + } } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java index dca8c16c5..6fc726baf 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java @@ -20,6 +20,12 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableException; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; /** * This class holds configuration constants used by json format. @@ -38,4 +44,41 @@ public class JsonOptions { .defaultValue(false) .withDescription("Optional flag to skip fields and rows with parse errors instead of failing;\n" + "fields are set to null in case of errors, false by default"); + + public static final ConfigOption TIMESTAMP_FORMAT = ConfigOptions + .key("timestamp-format.standard") + .stringType() + .defaultValue("SQL") + .withDescription("Optional flag to specify timestamp format, SQL by default." + + " Option ISO-8601 will parse input timestamp in \"yyyy-MM-ddTHH:mm:ss.s{precision}\" format and output timestamp in the same format." + + " Option SQL will parse input timestamp in \"yyyy-MM-dd HH:mm:ss.s{precision}\" format and output timestamp in the same format."); + + // -------------------------------------------------------------------------------------------- + // Option enumerations + // -------------------------------------------------------------------------------------------- + + public static final String SQL = "SQL"; + public static final String ISO_8601 = "ISO-8601"; + + public static final Set TIMESTAMP_FORMAT_ENUM = new HashSet<>(Arrays.asList( + SQL, + ISO_8601 + )); + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + public static TimestampFormat getTimestampFormat(ReadableConfig config){ + String timestampFormat = config.get(TIMESTAMP_FORMAT); + switch (timestampFormat){ + case SQL: + return TimestampFormat.SQL; + case ISO_8601: + return TimestampFormat.ISO_8601; + default: + throw new TableException( + String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index 87dee7f22..d66ecce22 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericMapData; @@ -50,7 +51,6 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; -import java.time.ZoneOffset; import java.time.temporal.TemporalAccessor; import java.time.temporal.TemporalQueries; import java.util.HashMap; @@ -60,8 +60,9 @@ import static java.lang.String.format; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -93,11 +94,15 @@ public class JsonRowDataDeserializationSchema implements DeserializationSchema resultTypeInfo, boolean failOnMissingField, - boolean ignoreParseErrors) { + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { if (ignoreParseErrors && failOnMissingField) { throw new IllegalArgumentException( "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."); @@ -106,6 +111,7 @@ public JsonRowDataDeserializationSchema( this.failOnMissingField = failOnMissingField; this.ignoreParseErrors = ignoreParseErrors; this.runtimeConverter = createRowConverter(checkNotNull(rowType)); + this.timestampFormat = timestampFormat; } @Override @@ -142,12 +148,13 @@ public boolean equals(Object o) { JsonRowDataDeserializationSchema that = (JsonRowDataDeserializationSchema) o; return failOnMissingField == that.failOnMissingField && ignoreParseErrors == that.ignoreParseErrors && - resultTypeInfo.equals(that.resultTypeInfo); + resultTypeInfo.equals(that.resultTypeInfo) && + timestampFormat.equals(that.timestampFormat); } @Override public int hashCode() { - return Objects.hash(failOnMissingField, ignoreParseErrors, resultTypeInfo); + return Objects.hash(failOnMissingField, ignoreParseErrors, resultTypeInfo, timestampFormat); } // ------------------------------------------------------------------------------------- @@ -193,7 +200,6 @@ private DeserializationRuntimeConverter createNotNullConverter(LogicalType type) return this::convertToDate; case TIME_WITHOUT_TIME_ZONE: return this::convertToTime; - case TIMESTAMP_WITH_TIME_ZONE: case TIMESTAMP_WITHOUT_TIME_ZONE: return this::convertToTimestamp; case FLOAT: @@ -272,37 +278,25 @@ private int convertToDate(JsonNode jsonNode) { } private int convertToTime(JsonNode jsonNode) { - // according to RFC 3339 every full-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText()); - - ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset()); + TemporalAccessor parsedTime = SQL_TIME_FORMAT.parse(jsonNode.asText()); LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); - if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0 || localTime.getNano() != 0) { - throw new JsonParseException( - "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet."); - } - // get number of milliseconds of the day return localTime.toSecondOfDay() * 1000; } private TimestampData convertToTimestamp(JsonNode jsonNode) { - // according to RFC 3339 every date-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - TemporalAccessor parsedTimestamp = RFC3339_TIMESTAMP_FORMAT.parse(jsonNode.asText()); - - ZoneOffset zoneOffset = parsedTimestamp.query(TemporalQueries.offset()); - - if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0) { - throw new JsonParseException( - "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + - "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + TemporalAccessor parsedTimestamp; + switch (timestampFormat){ + case SQL: + parsedTimestamp = SQL_TIMESTAMP_FORMAT.parse(jsonNode.asText()); + break; + case ISO_8601: + parsedTimestamp = ISO8601_TIMESTAMP_FORMAT.parse(jsonNode.asText()); + break; + default: + throw new TableException(String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); } - LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index 263e28255..89b3b87af 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.MapData; @@ -45,8 +46,9 @@ import java.util.Objects; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; /** * Serialization schema that serializes an object of Flink internal data structure into a JSON bytes. @@ -72,8 +74,12 @@ public class JsonRowDataSerializationSchema implements SerializationSchema { int millisecond = (int) value; LocalTime time = LocalTime.ofSecondOfDay(millisecond / 1000L); - return mapper.getNodeFactory().textNode(RFC3339_TIME_FORMAT.format(time)); + return mapper.getNodeFactory().textNode(SQL_TIME_FORMAT.format(time)); }; } private SerializationRuntimeConverter createTimestampConverter() { - return (mapper, reuse, value) -> { - TimestampData timestamp = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(RFC3339_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); - }; + switch (timestampFormat){ + case ISO_8601: + return (mapper, reuse, value) -> { + TimestampData timestamp = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(ISO8601_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); + }; + case SQL: + return (mapper, reuse, value) -> { + TimestampData timestamp = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(SQL_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); + }; + default: + throw new TableException("Unsupported timestamp format. Validator should have checked that."); + } } private SerializationRuntimeConverter createArrayConverter(ArrayType type) { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java index c946c5db7..693781ba3 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java @@ -23,7 +23,7 @@ import java.time.temporal.ChronoField; /** - * Time formats respecting the RFC3339 specification. + * Time formats and timestamp formats respecting the RFC3339 specification, ISO-8601 specification and SQL specification. */ class TimeFormats { @@ -41,6 +41,22 @@ class TimeFormats { .append(RFC3339_TIME_FORMAT) .toFormatter(); + /** Formatter for ISO8601 string representation of a timestamp value (without UTC timezone). */ + static final DateTimeFormatter ISO8601_TIMESTAMP_FORMAT = DateTimeFormatter.ISO_LOCAL_DATE_TIME; + + /** Formatter for SQL string representation of a time value. */ + static final DateTimeFormatter SQL_TIME_FORMAT = new DateTimeFormatterBuilder() + .appendPattern("HH:mm:ss") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) + .toFormatter(); + + /** Formatter for SQL string representation of a timestamp value (without UTC timezone). */ + static final DateTimeFormatter SQL_TIMESTAMP_FORMAT = new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(SQL_TIME_FORMAT) + .toFormatter(); + private TimeFormats() { } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java new file mode 100644 index 000000000..0c31fd421 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.Internal; + +/** + * Timestamp format Enums. + */ +@Internal +public enum TimestampFormat { + /** Options to specify timestamp format. It will parse timestamp in "yyyy-MM-dd HH:mm:ss.s{precision}" format + * and output timestamp in the same format*/ + SQL, + + /** Options to specify timestamp format. It will parse timestamp in "yyyy-MM-ddTHH:mm:ss.s{precision}" format + * and output timestamp in the same format*/ + ISO_8601 +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index 4c03d041e..761a0f4db 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; +import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericRowData; @@ -70,7 +71,8 @@ public final class CanalJsonDeserializationSchema implements DeserializationSche public CanalJsonDeserializationSchema( RowType rowType, TypeInformation resultTypeInfo, - boolean ignoreParseErrors) { + boolean ignoreParseErrors, + TimestampFormat timestampFormatOption) { this.resultTypeInfo = resultTypeInfo; this.ignoreParseErrors = ignoreParseErrors; this.fieldCount = rowType.getFieldCount(); @@ -79,7 +81,8 @@ public CanalJsonDeserializationSchema( // the result type is never used, so it's fine to pass in Canal's result type resultTypeInfo, false, // ignoreParseErrors already contains the functionality of failOnMissingField - ignoreParseErrors); + ignoreParseErrors, + timestampFormatOption); } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java index fba98bf48..cedff8081 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java @@ -22,8 +22,9 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; @@ -48,12 +49,9 @@ public class CanalJsonFormatFactory implements DeserializationFormatFactory, Ser public static final String IDENTIFIER = "canal-json"; - public static final ConfigOption IGNORE_PARSE_ERRORS = ConfigOptions - .key("ignore-parse-errors") - .booleanType() - .defaultValue(false) - .withDescription("Optional flag to skip fields and rows with parse errors instead of failing, " + - "fields are set to null in case of errors. Default is false."); + public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + + public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; @SuppressWarnings("unchecked") @Override @@ -62,6 +60,7 @@ public DecodingFormat> createDecodingFormat( ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); return new DecodingFormat>() { @Override @@ -73,7 +72,8 @@ public DeserializationSchema createRuntimeDecoder( return new CanalJsonDeserializationSchema( rowType, rowDataTypeInfo, - ignoreParseErrors); + ignoreParseErrors, + timestampFormatOption); } @Override @@ -109,6 +109,7 @@ public Set> requiredOptions() { public Set> optionalOptions() { Set> options = new HashSet<>(); options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java index f0ae9d99d..d1f1d96ac 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; +import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; @@ -78,7 +79,8 @@ public DebeziumJsonDeserializationSchema( RowType rowType, TypeInformation resultTypeInfo, boolean schemaInclude, - boolean ignoreParseErrors) { + boolean ignoreParseErrors, + TimestampFormat timestampFormatOption) { this.resultTypeInfo = resultTypeInfo; this.schemaInclude = schemaInclude; this.ignoreParseErrors = ignoreParseErrors; @@ -87,7 +89,8 @@ public DebeziumJsonDeserializationSchema( // the result type is never used, so it's fine to pass in Debezium's result type resultTypeInfo, false, // ignoreParseErrors already contains the functionality of failOnMissingField - ignoreParseErrors); + ignoreParseErrors, + timestampFormatOption); } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java index 3458014ec..17c6ab2e8 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -24,6 +24,8 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; @@ -57,12 +59,9 @@ public class DebeziumJsonFormatFactory implements DeserializationFormatFactory, "This option indicates the Debezium JSON data include the schema in the message or not. " + "Default is false."); - public static final ConfigOption IGNORE_PARSE_ERRORS = ConfigOptions - .key("ignore-parse-errors") - .booleanType() - .defaultValue(false) - .withDescription("Optional flag to skip fields and rows with parse errors instead of failing, " + - "fields are set to null in case of errors. Default is false."); + public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + + public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; @SuppressWarnings("unchecked") @Override @@ -72,6 +71,7 @@ public DecodingFormat> createDecodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); final boolean schemaInclude = formatOptions.get(SCHEMA_INCLUDE); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); return new DecodingFormat>() { @Override @@ -84,7 +84,8 @@ public DeserializationSchema createRuntimeDecoder( rowType, rowDataTypeInfo, schemaInclude, - ignoreParseErrors); + ignoreParseErrors, + timestampFormatOption); } @Override @@ -121,6 +122,7 @@ public Set> optionalOptions() { Set> options = new HashSet<>(); options.add(SCHEMA_INCLUDE); options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index 8ddf03ea7..dfd75e963 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -91,6 +91,25 @@ public void testInvalidOptionForIgnoreParseErrors() { testSchemaDeserializationSchema(tableOptions); } + @Test + public void testInvalidOptionForTimestampFormat() { + final Map tableOptions = getModifyOptions( + options -> options.put("json.timestamp-format.standard", "test")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new ValidationException("Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + testSchemaDeserializationSchema(tableOptions); + } + + @Test + public void testLowerCaseOptionForTimestampFormat() { + final Map tableOptions = getModifyOptions( + options -> options.put("json.timestamp-format.standard", "iso-8601")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new ValidationException("Unsupported value 'iso-8601' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + testSchemaDeserializationSchema(tableOptions); + } // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -101,7 +120,8 @@ private void testSchemaDeserializationSchema(Map options) { ROW_TYPE, new RowDataTypeInfo(ROW_TYPE), false, - true); + true, + TimestampFormat.ISO_8601); final DynamicTableSource actualSource = createTableSource(options); assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; @@ -117,7 +137,8 @@ private void testSchemaDeserializationSchema(Map options) { } private void testSchemaSerializationSchema(Map options) { - final JsonRowDataSerializationSchema expectedSer = new JsonRowDataSerializationSchema(ROW_TYPE); + final JsonRowDataSerializationSchema expectedSer = new JsonRowDataSerializationSchema(ROW_TYPE, + TimestampFormat.ISO_8601); final DynamicTableSink actualSink = createTableSink(options); assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; @@ -152,6 +173,7 @@ private Map getAllOptions() { options.put("format", JsonFormatFactory.IDENTIFIER); options.put("json.fail-on-missing-field", "false"); options.put("json.ignore-parse-errors", "true"); + options.put("json.timestamp-format.standard", "ISO-8601"); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index dcfae9c26..fedfbeab4 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -114,9 +114,9 @@ public void testSerDe() throws Exception { root.put("decimal", decimal); root.set("doubles", doubleNode); root.put("date", "1990-10-14"); - root.put("time", "12:12:43Z"); - root.put("timestamp3", "1990-10-14T12:12:43.123Z"); - root.put("timestamp9", "1990-10-14T12:12:43.123456789Z"); + root.put("time", "12:12:43"); + root.put("timestamp3", "1990-10-14T12:12:43.123"); + root.put("timestamp9", "1990-10-14T12:12:43.123456789"); root.putObject("map").put("flink", 123); root.putObject("map2map").putObject("inner_map").put("key", 234); @@ -143,7 +143,7 @@ public void testSerDe() throws Exception { RowDataTypeInfo resultTypeInfo = new RowDataTypeInfo(schema); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - schema, resultTypeInfo, false, false); + schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); Row expected = new Row(16); expected.setField(0, true); @@ -168,7 +168,7 @@ public void testSerDe() throws Exception { assertEquals(expected, actual); // test serialization - JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(schema); + JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(schema, TimestampFormat.ISO_8601); byte[] actualBytes = serializationSchema.serialize(rowData); assertEquals(new String(serializedJson), new String(actualBytes)); @@ -211,7 +211,7 @@ public void testSlowDeserialization() throws Exception { RowType rowType = (RowType) dataType.getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, new RowDataTypeInfo(rowType), false, false); + rowType, new RowDataTypeInfo(rowType), false, false, TimestampFormat.ISO_8601); Row expected = new Row(7); expected.setField(0, bool); @@ -236,8 +236,8 @@ public void testSerDeMultiRows() throws Exception { ).getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, new RowDataTypeInfo(rowType), false, false); - JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType); + rowType, new RowDataTypeInfo(rowType), false, false, TimestampFormat.ISO_8601); + JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.ISO_8601); ObjectMapper objectMapper = new ObjectMapper(); @@ -290,8 +290,8 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { ).getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, new RowDataTypeInfo(rowType), false, true); - JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType); + rowType, new RowDataTypeInfo(rowType), false, true, TimestampFormat.ISO_8601); + JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.ISO_8601); for (int i = 0; i < jsons.length; i++) { String json = jsons[i]; @@ -315,7 +315,7 @@ public void testDeserializationMissingNode() throws Exception { // pass on missing field JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - schema, new RowDataTypeInfo(schema), false, false); + schema, new RowDataTypeInfo(schema), false, false, TimestampFormat.ISO_8601); Row expected = new Row(1); Row actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); @@ -323,7 +323,7 @@ public void testDeserializationMissingNode() throws Exception { // fail on missing field deserializationSchema = deserializationSchema = new JsonRowDataDeserializationSchema( - schema, new RowDataTypeInfo(schema), true, false); + schema, new RowDataTypeInfo(schema), true, false, TimestampFormat.ISO_8601); thrown.expect(IOException.class); thrown.expectMessage("Failed to deserialize JSON '{\"id\":123123123}'"); @@ -331,7 +331,7 @@ public void testDeserializationMissingNode() throws Exception { // ignore on parse error deserializationSchema = new JsonRowDataDeserializationSchema( - schema, new RowDataTypeInfo(schema), false, true); + schema, new RowDataTypeInfo(schema), false, true, TimestampFormat.ISO_8601); actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); assertEquals(expected, actual); @@ -340,7 +340,29 @@ public void testDeserializationMissingNode() throws Exception { // failOnMissingField and ignoreParseErrors both enabled //noinspection ConstantConditions new JsonRowDataDeserializationSchema( - schema, new RowDataTypeInfo(schema), true, true); + schema, new RowDataTypeInfo(schema), true, true, TimestampFormat.ISO_8601); + } + + @Test + public void testSerDeSQLTimestampFormat() throws Exception{ + RowType rowType = (RowType) ROW( + FIELD("timestamp3", TIMESTAMP(3)), + FIELD("timestamp9", TIMESTAMP(9)) + ).getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( + rowType, new RowDataTypeInfo(rowType), false, false, TimestampFormat.SQL); + JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.SQL); + + ObjectMapper objectMapper = new ObjectMapper(); + + ObjectNode root = objectMapper.createObjectNode(); + root.put("timestamp3", "1990-10-14 12:12:43.123"); + root.put("timestamp9", "1990-10-14 12:12:43.123456789"); + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + RowData rowData = deserializationSchema.deserialize(serializedJson); + byte[] actual = serializationSchema.serialize(rowData); + assertEquals(new String(serializedJson), new String(actual)); } @Test @@ -356,7 +378,8 @@ public void testJsonParse() throws Exception { private void testIgnoreParseErrors(TestSpec spec) throws Exception { // the parsing field should be null and no exception is thrown JsonRowDataDeserializationSchema ignoreErrorsSchema = new JsonRowDataDeserializationSchema( - spec.rowType, new RowDataTypeInfo(spec.rowType), false, true); + spec.rowType, new RowDataTypeInfo(spec.rowType), false, true, + TimestampFormat.ISO_8601); Row expected; if (spec.expected != null) { expected = spec.expected; @@ -373,7 +396,8 @@ private void testIgnoreParseErrors(TestSpec spec) throws Exception { private void testParseErrors(TestSpec spec) throws Exception { // expect exception if parse error is not ignored JsonRowDataDeserializationSchema failingSchema = new JsonRowDataDeserializationSchema( - spec.rowType, new RowDataTypeInfo(spec.rowType), false, false); + spec.rowType, new RowDataTypeInfo(spec.rowType), false, false, + spec.timestampFormat); thrown.expectMessage(spec.errorMessage); failingSchema.deserialize(spec.json.getBytes()); @@ -420,6 +444,16 @@ private void testParseErrors(TestSpec spec) throws Exception { .rowType(ROW(FIELD("id", TIME()))) .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"), + TestSpec + .json("{\"id\":\"18:00:243\"}") + .rowType(ROW(FIELD("id", TIME()))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"), + + TestSpec + .json("{\"id\":\"20191112\"}") + .rowType(ROW(FIELD("id", DATE()))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'"), + TestSpec .json("{\"id\":\"20191112\"}") .rowType(ROW(FIELD("id", DATE()))) @@ -428,8 +462,25 @@ private void testParseErrors(TestSpec spec) throws Exception { TestSpec .json("{\"id\":\"2019-11-12 18:00:12\"}") .rowType(ROW(FIELD("id", TIMESTAMP(0)))) + .timestampFormat(TimestampFormat.ISO_8601) .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'"), + TestSpec + .json("{\"id\":\"2019-11-12T18:00:12\"}") + .rowType(ROW(FIELD("id", TIMESTAMP(0)))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'"), + + TestSpec + .json("{\"id\":\"2019-11-12T18:00:12Z\"}") + .rowType(ROW(FIELD("id", TIMESTAMP(0)))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'"), + + TestSpec + .json("{\"id\":\"2019-11-12T18:00:12Z\"}") + .rowType(ROW(FIELD("id", TIMESTAMP(0)))) + .timestampFormat(TimestampFormat.ISO_8601) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'"), + TestSpec .json("{\"id\":\"abc\"}") .rowType(ROW(FIELD("id", DECIMAL(10, 3)))) @@ -471,6 +522,7 @@ private static Row convertToExternal(RowData rowData, DataType dataType) { private static class TestSpec { private final String json; private RowType rowType; + private TimestampFormat timestampFormat = TimestampFormat.SQL; private Row expected; private String errorMessage; @@ -496,5 +548,10 @@ TestSpec expectErrorMessage(String errorMessage) { this.errorMessage = errorMessage; return this; } + + TestSpec timestampFormat(TimestampFormat timestampFormat){ + this.timestampFormat = timestampFormat; + return this; + } } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java index 02f055ea1..36099effb 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json.canal; +import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; import org.apache.flink.table.types.logical.RowType; @@ -65,7 +66,8 @@ public void testDeserialization() throws Exception { CanalJsonDeserializationSchema deserializationSchema = new CanalJsonDeserializationSchema( SCHEMA, new RowDataTypeInfo(SCHEMA), - false); + false, + TimestampFormat.ISO_8601); SimpleCollector collector = new SimpleCollector(); for (String line : lines) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 83afa113d..4486a498b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.CatalogTableImpl; @@ -65,7 +66,8 @@ public void testSeDeSchema() { final CanalJsonDeserializationSchema expectedDeser = new CanalJsonDeserializationSchema( ROW_TYPE, new RowDataTypeInfo(ROW_TYPE), - true); + true, + TimestampFormat.ISO_8601); final Map options = getAllOptions(); @@ -120,6 +122,7 @@ private Map getAllOptions() { options.put("format", "canal-json"); options.put("canal-json.ignore-parse-errors", "true"); + options.put("canal-json.timestamp-format.standard", "ISO-8601"); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java index ff7a3435a..d8114a8fc 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json.debezium; +import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; import org.apache.flink.table.types.logical.RowType; @@ -75,7 +76,8 @@ private void testDeserialization(String resourceFile, boolean schemaInclude) thr SCHEMA, new RowDataTypeInfo(SCHEMA), schemaInclude, - false); + false, + TimestampFormat.ISO_8601); SimpleCollector collector = new SimpleCollector(); for (String line : lines) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 869ff2812..2f157c9fd 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.CatalogTableImpl; @@ -66,7 +67,8 @@ public void testSeDeSchema() { ROW_TYPE, new RowDataTypeInfo(ROW_TYPE), true, - true); + true, + TimestampFormat.ISO_8601); final Map options = getAllOptions(); @@ -122,6 +124,7 @@ private Map getAllOptions() { options.put("format", "debezium-json"); options.put("debezium-json.ignore-parse-errors", "true"); options.put("debezium-json.schema-include", "true"); + options.put("debezium-json.timestamp-format.standard", "ISO-8601"); return options; } From bc8d5a7d945889cb6bf08305828c2265d6e1f027 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 23 Jun 2020 12:07:17 +0200 Subject: [PATCH 077/322] [FLINK-17000][table] Ensure that every logical type can be represented as TypeInformation Introduces a WrapperTypeInfo that can replace most (if not all) TypeInformation classes in the Blink planner. It is backed by logical types and uses internal serializers. This closes #12852. --- .../formats/json/JsonFormatFactoryTest.java | 4 +-- .../json/JsonRowDataSerDeSchemaTest.java | 25 ++++++++++--------- .../CanalJsonDeserializationSchemaTest.java | 4 +-- .../canal/CanalJsonFormatFactoryTest.java | 4 +-- ...DebeziumJsonDeserializationSchemaTest.java | 4 +-- .../DebeziumJsonFormatFactoryTest.java | 4 +-- 6 files changed, 23 insertions(+), 22 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index dfd75e963..d1c26be52 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -33,7 +33,7 @@ import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; @@ -118,7 +118,7 @@ private void testSchemaDeserializationSchema(Map options) { final JsonRowDataDeserializationSchema expectedDeser = new JsonRowDataDeserializationSchema( ROW_TYPE, - new RowDataTypeInfo(ROW_TYPE), + WrapperTypeInfo.of(ROW_TYPE), false, true, TimestampFormat.ISO_8601); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index fedfbeab4..cab427f11 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -18,9 +18,10 @@ package org.apache.flink.formats.json; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -140,7 +141,7 @@ public void testSerDe() throws Exception { FIELD("map", MAP(STRING(), BIGINT())), FIELD("map2map", MAP(STRING(), MAP(STRING(), INT())))); RowType schema = (RowType) dataType.getLogicalType(); - RowDataTypeInfo resultTypeInfo = new RowDataTypeInfo(schema); + TypeInformation resultTypeInfo = WrapperTypeInfo.of(schema); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); @@ -211,7 +212,7 @@ public void testSlowDeserialization() throws Exception { RowType rowType = (RowType) dataType.getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, new RowDataTypeInfo(rowType), false, false, TimestampFormat.ISO_8601); + rowType, WrapperTypeInfo.of(rowType), false, false, TimestampFormat.ISO_8601); Row expected = new Row(7); expected.setField(0, bool); @@ -236,7 +237,7 @@ public void testSerDeMultiRows() throws Exception { ).getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, new RowDataTypeInfo(rowType), false, false, TimestampFormat.ISO_8601); + rowType, WrapperTypeInfo.of(rowType), false, false, TimestampFormat.ISO_8601); JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.ISO_8601); ObjectMapper objectMapper = new ObjectMapper(); @@ -290,7 +291,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { ).getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, new RowDataTypeInfo(rowType), false, true, TimestampFormat.ISO_8601); + rowType, WrapperTypeInfo.of(rowType), false, true, TimestampFormat.ISO_8601); JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.ISO_8601); for (int i = 0; i < jsons.length; i++) { @@ -315,7 +316,7 @@ public void testDeserializationMissingNode() throws Exception { // pass on missing field JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - schema, new RowDataTypeInfo(schema), false, false, TimestampFormat.ISO_8601); + schema, WrapperTypeInfo.of(schema), false, false, TimestampFormat.ISO_8601); Row expected = new Row(1); Row actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); @@ -323,7 +324,7 @@ public void testDeserializationMissingNode() throws Exception { // fail on missing field deserializationSchema = deserializationSchema = new JsonRowDataDeserializationSchema( - schema, new RowDataTypeInfo(schema), true, false, TimestampFormat.ISO_8601); + schema, WrapperTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); thrown.expect(IOException.class); thrown.expectMessage("Failed to deserialize JSON '{\"id\":123123123}'"); @@ -331,7 +332,7 @@ public void testDeserializationMissingNode() throws Exception { // ignore on parse error deserializationSchema = new JsonRowDataDeserializationSchema( - schema, new RowDataTypeInfo(schema), false, true, TimestampFormat.ISO_8601); + schema, WrapperTypeInfo.of(schema), false, true, TimestampFormat.ISO_8601); actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); assertEquals(expected, actual); @@ -340,7 +341,7 @@ public void testDeserializationMissingNode() throws Exception { // failOnMissingField and ignoreParseErrors both enabled //noinspection ConstantConditions new JsonRowDataDeserializationSchema( - schema, new RowDataTypeInfo(schema), true, true, TimestampFormat.ISO_8601); + schema, WrapperTypeInfo.of(schema), true, true, TimestampFormat.ISO_8601); } @Test @@ -351,7 +352,7 @@ public void testSerDeSQLTimestampFormat() throws Exception{ ).getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, new RowDataTypeInfo(rowType), false, false, TimestampFormat.SQL); + rowType, WrapperTypeInfo.of(rowType), false, false, TimestampFormat.SQL); JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.SQL); ObjectMapper objectMapper = new ObjectMapper(); @@ -378,7 +379,7 @@ public void testJsonParse() throws Exception { private void testIgnoreParseErrors(TestSpec spec) throws Exception { // the parsing field should be null and no exception is thrown JsonRowDataDeserializationSchema ignoreErrorsSchema = new JsonRowDataDeserializationSchema( - spec.rowType, new RowDataTypeInfo(spec.rowType), false, true, + spec.rowType, WrapperTypeInfo.of(spec.rowType), false, true, TimestampFormat.ISO_8601); Row expected; if (spec.expected != null) { @@ -396,7 +397,7 @@ spec.rowType, new RowDataTypeInfo(spec.rowType), false, true, private void testParseErrors(TestSpec spec) throws Exception { // expect exception if parse error is not ignored JsonRowDataDeserializationSchema failingSchema = new JsonRowDataDeserializationSchema( - spec.rowType, new RowDataTypeInfo(spec.rowType), false, false, + spec.rowType, WrapperTypeInfo.of(spec.rowType), false, false, spec.timestampFormat); thrown.expectMessage(spec.errorMessage); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java index 36099effb..1a4d22259 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java @@ -20,7 +20,7 @@ import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; @@ -65,7 +65,7 @@ public void testDeserialization() throws Exception { List lines = readLines("canal-data.txt"); CanalJsonDeserializationSchema deserializationSchema = new CanalJsonDeserializationSchema( SCHEMA, - new RowDataTypeInfo(SCHEMA), + WrapperTypeInfo.of(SCHEMA), false, TimestampFormat.ISO_8601); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 4486a498b..3b97114f7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -31,7 +31,7 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; @@ -65,7 +65,7 @@ public class CanalJsonFormatFactoryTest extends TestLogger { public void testSeDeSchema() { final CanalJsonDeserializationSchema expectedDeser = new CanalJsonDeserializationSchema( ROW_TYPE, - new RowDataTypeInfo(ROW_TYPE), + WrapperTypeInfo.of(ROW_TYPE), true, TimestampFormat.ISO_8601); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java index d8114a8fc..3bcb16c9d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java @@ -20,7 +20,7 @@ import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; @@ -74,7 +74,7 @@ private void testDeserialization(String resourceFile, boolean schemaInclude) thr List lines = readLines(resourceFile); DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( SCHEMA, - new RowDataTypeInfo(SCHEMA), + WrapperTypeInfo.of(SCHEMA), schemaInclude, false, TimestampFormat.ISO_8601); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 2f157c9fd..134b7fb13 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -31,7 +31,7 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; @@ -65,7 +65,7 @@ public class DebeziumJsonFormatFactoryTest extends TestLogger { public void testSeDeSchema() { final DebeziumJsonDeserializationSchema expectedDeser = new DebeziumJsonDeserializationSchema( ROW_TYPE, - new RowDataTypeInfo(ROW_TYPE), + WrapperTypeInfo.of(ROW_TYPE), true, true, TimestampFormat.ISO_8601); From 1acd1227a43932a1cfee94dc3ab081cbc5f14d05 Mon Sep 17 00:00:00 2001 From: libenchao Date: Mon, 1 Jun 2020 11:58:11 +0800 Subject: [PATCH 078/322] [FLINK-18002][json] Correct the behavior for ContainerNode as varchar type This closes #12421 --- .../JsonRowDataDeserializationSchema.java | 6 +- .../json/JsonRowDeserializationSchema.java | 10 +- .../json/JsonRowDataSerDeSchemaTest.java | 107 ++++++++++++------ .../JsonRowDeserializationSchemaTest.java | 30 +++++ 4 files changed, 114 insertions(+), 39 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index d66ecce22..956ca5baa 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -304,7 +304,11 @@ private TimestampData convertToTimestamp(JsonNode jsonNode) { } private StringData convertToString(JsonNode jsonNode) { - return StringData.fromString(jsonNode.asText()); + if (jsonNode.isContainerNode()) { + return StringData.fromString(jsonNode.toString()); + } else { + return StringData.fromString(jsonNode.asText()); + } } private byte[] convertToBytes(JsonNode jsonNode) { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index cc9b55d58..f9a6395df 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -347,7 +347,7 @@ private Optional createConverterForSimpleType(T } else if (simpleTypeInfo == Types.BOOLEAN) { return Optional.of(this::convertToBoolean); } else if (simpleTypeInfo == Types.STRING) { - return Optional.of((mapper, jsonNode) -> jsonNode.asText()); + return Optional.of(this::convertToString); } else if (simpleTypeInfo == Types.INT) { return Optional.of(this::convertToInt); } else if (simpleTypeInfo == Types.LONG) { @@ -381,6 +381,14 @@ private Optional createConverterForSimpleType(T } } + private String convertToString(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isContainerNode()) { + return jsonNode.toString(); + } else { + return jsonNode.asText(); + } + } + private boolean convertToBoolean(ObjectMapper mapper, JsonNode jsonNode) { if (jsonNode.isBoolean()) { // avoid redundant toString and parseBoolean, for better performance diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index cab427f11..7b561aaf0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -30,11 +30,9 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; -import org.junit.Rule; +import org.junit.Assert; import org.junit.Test; -import org.junit.rules.ExpectedException; -import java.io.IOException; import java.math.BigDecimal; import java.sql.Timestamp; import java.time.LocalDate; @@ -71,9 +69,6 @@ */ public class JsonRowDataSerDeSchemaTest { - @Rule - public ExpectedException thrown = ExpectedException.none(); - @Test public void testSerDe() throws Exception { byte tinyint = 'c'; @@ -326,9 +321,13 @@ public void testDeserializationMissingNode() throws Exception { deserializationSchema = deserializationSchema = new JsonRowDataDeserializationSchema( schema, WrapperTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); - thrown.expect(IOException.class); - thrown.expectMessage("Failed to deserialize JSON '{\"id\":123123123}'"); - deserializationSchema.deserialize(serializedJson); + String errorMessage = "Failed to deserialize JSON '{\"id\":123123123}'."; + try { + deserializationSchema.deserialize(serializedJson); + Assert.fail("expecting exception message: " + errorMessage); + } catch (Throwable t) { + assertEquals(errorMessage, t.getMessage()); + } // ignore on parse error deserializationSchema = new JsonRowDataDeserializationSchema( @@ -336,12 +335,15 @@ public void testDeserializationMissingNode() throws Exception { actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); assertEquals(expected, actual); - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled"); - // failOnMissingField and ignoreParseErrors both enabled - //noinspection ConstantConditions - new JsonRowDataDeserializationSchema( - schema, WrapperTypeInfo.of(schema), true, true, TimestampFormat.ISO_8601); + errorMessage = "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."; + try { + // failOnMissingField and ignoreParseErrors both enabled + new JsonRowDataDeserializationSchema( + schema, WrapperTypeInfo.of(schema), true, true, TimestampFormat.ISO_8601); + Assert.fail("expecting exception message: " + errorMessage); + } catch (Throwable t) { + assertEquals(errorMessage, t.getMessage()); + } } @Test @@ -380,7 +382,7 @@ private void testIgnoreParseErrors(TestSpec spec) throws Exception { // the parsing field should be null and no exception is thrown JsonRowDataDeserializationSchema ignoreErrorsSchema = new JsonRowDataDeserializationSchema( spec.rowType, WrapperTypeInfo.of(spec.rowType), false, true, - TimestampFormat.ISO_8601); + spec.timestampFormat); Row expected; if (spec.expected != null) { expected = spec.expected; @@ -400,8 +402,12 @@ private void testParseErrors(TestSpec spec) throws Exception { spec.rowType, WrapperTypeInfo.of(spec.rowType), false, false, spec.timestampFormat); - thrown.expectMessage(spec.errorMessage); - failingSchema.deserialize(spec.json.getBytes()); + try { + failingSchema.deserialize(spec.json.getBytes()); + Assert.fail("expecting exception " + spec.errorMessage); + } catch (Throwable t) { + assertEquals(t.getMessage(), spec.errorMessage); + } } private static List testData = Arrays.asList( @@ -418,7 +424,7 @@ private void testParseErrors(TestSpec spec) throws Exception { TestSpec .json("{\"id\":\"abc\"}") .rowType(ROW(FIELD("id", INT()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'."), TestSpec .json("{\"id\":112.013}") @@ -428,84 +434,111 @@ private void testParseErrors(TestSpec spec) throws Exception { TestSpec .json("{\"id\":\"long\"}") .rowType(ROW(FIELD("id", BIGINT()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'."), TestSpec .json("{\"id\":\"112.013.123\"}") .rowType(ROW(FIELD("id", FLOAT()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'."), TestSpec .json("{\"id\":\"112.013.123\"}") .rowType(ROW(FIELD("id", DOUBLE()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'."), TestSpec .json("{\"id\":\"18:00:243\"}") .rowType(ROW(FIELD("id", TIME()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'."), TestSpec .json("{\"id\":\"18:00:243\"}") .rowType(ROW(FIELD("id", TIME()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'."), TestSpec .json("{\"id\":\"20191112\"}") .rowType(ROW(FIELD("id", DATE()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'."), TestSpec .json("{\"id\":\"20191112\"}") .rowType(ROW(FIELD("id", DATE()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'."), + + TestSpec + .json("{\"id\":true}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("true")), + + TestSpec + .json("{\"id\":123.234}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("123.234")), + + TestSpec + .json("{\"id\":1234567}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("1234567")), + + TestSpec + .json("{\"id\":\"string field\"}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("string field")), + + TestSpec + .json("{\"id\":[\"array data1\",\"array data2\",123,234.345]}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("[\"array data1\",\"array data2\",123,234.345]")), + + TestSpec + .json("{\"id\":{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}")), TestSpec .json("{\"id\":\"2019-11-12 18:00:12\"}") .rowType(ROW(FIELD("id", TIMESTAMP(0)))) .timestampFormat(TimestampFormat.ISO_8601) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'."), TestSpec .json("{\"id\":\"2019-11-12T18:00:12\"}") .rowType(ROW(FIELD("id", TIMESTAMP(0)))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."), TestSpec .json("{\"id\":\"2019-11-12T18:00:12Z\"}") .rowType(ROW(FIELD("id", TIMESTAMP(0)))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'."), TestSpec .json("{\"id\":\"2019-11-12T18:00:12Z\"}") .rowType(ROW(FIELD("id", TIMESTAMP(0)))) .timestampFormat(TimestampFormat.ISO_8601) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'."), TestSpec .json("{\"id\":\"abc\"}") .rowType(ROW(FIELD("id", DECIMAL(10, 3)))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'."), TestSpec .json("{\"row\":{\"id\":\"abc\"}}") .rowType(ROW(FIELD("row", ROW(FIELD("id", BOOLEAN()))))) - .expect(Row.of(new Row(1))) - .expectErrorMessage("Failed to deserialize JSON '{\"row\":{\"id\":\"abc\"}}'"), + .expect(Row.of(Row.of(false))), TestSpec .json("{\"array\":[123, \"abc\"]}") .rowType(ROW(FIELD("array", ARRAY(INT())))) .expect(Row.of((Object) new Integer[]{123, null})) - .expectErrorMessage("Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'"), + .expectErrorMessage("Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'."), TestSpec .json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") .rowType(ROW(FIELD("map", MAP(STRING(), INT())))) .expect(Row.of(createHashMap("key1", 123, "key2", null))) - .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'") - - + .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'.") ); private static Map createHashMap(String k1, Integer v1, String k2, Integer v2) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index cba7fcefd..438a18c20 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -313,6 +313,36 @@ private void testParseErrors(TestSpec spec) { .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.LONG)) .expect(Row.of(112L)), + TestSpec + .json("{\"id\":true}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) + .expect(Row.of("true")), + + TestSpec + .json("{\"id\":123.234}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) + .expect(Row.of("123.234")), + + TestSpec + .json("{\"id\":1234567}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) + .expect(Row.of("1234567")), + + TestSpec + .json("{\"id\":\"string field\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) + .expect(Row.of("string field")), + + TestSpec + .json("{\"id\":[\"array data1\",\"array data2\",123,234.345]}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) + .expect(Row.of("[\"array data1\",\"array data2\",123,234.345]")), + + TestSpec + .json("{\"id\":{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) + .expect(Row.of("{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}")), + TestSpec .json("{\"id\":\"long\"}") .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.LONG)) From fbc291f3414e67b628490ee72578adc09d3b322b Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 10 Jul 2020 09:40:59 +0200 Subject: [PATCH 079/322] [FLINK-18537][table] Replace RowDataTypeInfo with InternalTypeInfo TypeInformation is a legacy class for the sole purpose of creating a TypeSerializer. Instances of TypeInformation are not required in the table ecosystem but sometimes enforced by interfaces of other modules (such as org.apache.flink.api.dag.Transformation). Therefore, we introduce InternalTypeInfo which acts as an adapter whenever type information is required. Instances of InternalTypeInfo should only be created for passing it to interfaces that require type information. The class should not be used as a replacement for a LogicalType. Information such as the arity of a row type, field types, field names, etc. should be derived from the LogicalType directly. This closes #12900. --- .../formats/json/JsonFormatFactoryTest.java | 4 ++-- .../json/JsonRowDataSerDeSchemaTest.java | 24 +++++++++---------- .../CanalJsonDeserializationSchemaTest.java | 4 ++-- .../canal/CanalJsonFormatFactoryTest.java | 4 ++-- ...DebeziumJsonDeserializationSchemaTest.java | 4 ++-- .../DebeziumJsonFormatFactoryTest.java | 4 ++-- 6 files changed, 22 insertions(+), 22 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index d1c26be52..5c0747933 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -33,7 +33,7 @@ import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; @@ -118,7 +118,7 @@ private void testSchemaDeserializationSchema(Map options) { final JsonRowDataDeserializationSchema expectedDeser = new JsonRowDataDeserializationSchema( ROW_TYPE, - WrapperTypeInfo.of(ROW_TYPE), + InternalTypeInfo.of(ROW_TYPE), false, true, TimestampFormat.ISO_8601); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 7b561aaf0..58dc120b0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -136,7 +136,7 @@ public void testSerDe() throws Exception { FIELD("map", MAP(STRING(), BIGINT())), FIELD("map2map", MAP(STRING(), MAP(STRING(), INT())))); RowType schema = (RowType) dataType.getLogicalType(); - TypeInformation resultTypeInfo = WrapperTypeInfo.of(schema); + TypeInformation resultTypeInfo = InternalTypeInfo.of(schema); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); @@ -207,7 +207,7 @@ public void testSlowDeserialization() throws Exception { RowType rowType = (RowType) dataType.getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, WrapperTypeInfo.of(rowType), false, false, TimestampFormat.ISO_8601); + rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.ISO_8601); Row expected = new Row(7); expected.setField(0, bool); @@ -232,7 +232,7 @@ public void testSerDeMultiRows() throws Exception { ).getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, WrapperTypeInfo.of(rowType), false, false, TimestampFormat.ISO_8601); + rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.ISO_8601); JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.ISO_8601); ObjectMapper objectMapper = new ObjectMapper(); @@ -286,7 +286,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { ).getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, WrapperTypeInfo.of(rowType), false, true, TimestampFormat.ISO_8601); + rowType, InternalTypeInfo.of(rowType), false, true, TimestampFormat.ISO_8601); JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.ISO_8601); for (int i = 0; i < jsons.length; i++) { @@ -311,7 +311,7 @@ public void testDeserializationMissingNode() throws Exception { // pass on missing field JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - schema, WrapperTypeInfo.of(schema), false, false, TimestampFormat.ISO_8601); + schema, InternalTypeInfo.of(schema), false, false, TimestampFormat.ISO_8601); Row expected = new Row(1); Row actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); @@ -319,7 +319,7 @@ public void testDeserializationMissingNode() throws Exception { // fail on missing field deserializationSchema = deserializationSchema = new JsonRowDataDeserializationSchema( - schema, WrapperTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); + schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); String errorMessage = "Failed to deserialize JSON '{\"id\":123123123}'."; try { @@ -331,7 +331,7 @@ public void testDeserializationMissingNode() throws Exception { // ignore on parse error deserializationSchema = new JsonRowDataDeserializationSchema( - schema, WrapperTypeInfo.of(schema), false, true, TimestampFormat.ISO_8601); + schema, InternalTypeInfo.of(schema), false, true, TimestampFormat.ISO_8601); actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); assertEquals(expected, actual); @@ -339,7 +339,7 @@ public void testDeserializationMissingNode() throws Exception { try { // failOnMissingField and ignoreParseErrors both enabled new JsonRowDataDeserializationSchema( - schema, WrapperTypeInfo.of(schema), true, true, TimestampFormat.ISO_8601); + schema, InternalTypeInfo.of(schema), true, true, TimestampFormat.ISO_8601); Assert.fail("expecting exception message: " + errorMessage); } catch (Throwable t) { assertEquals(errorMessage, t.getMessage()); @@ -354,7 +354,7 @@ public void testSerDeSQLTimestampFormat() throws Exception{ ).getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, WrapperTypeInfo.of(rowType), false, false, TimestampFormat.SQL); + rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.SQL); ObjectMapper objectMapper = new ObjectMapper(); @@ -381,7 +381,7 @@ public void testJsonParse() throws Exception { private void testIgnoreParseErrors(TestSpec spec) throws Exception { // the parsing field should be null and no exception is thrown JsonRowDataDeserializationSchema ignoreErrorsSchema = new JsonRowDataDeserializationSchema( - spec.rowType, WrapperTypeInfo.of(spec.rowType), false, true, + spec.rowType, InternalTypeInfo.of(spec.rowType), false, true, spec.timestampFormat); Row expected; if (spec.expected != null) { @@ -399,7 +399,7 @@ private void testIgnoreParseErrors(TestSpec spec) throws Exception { private void testParseErrors(TestSpec spec) throws Exception { // expect exception if parse error is not ignored JsonRowDataDeserializationSchema failingSchema = new JsonRowDataDeserializationSchema( - spec.rowType, WrapperTypeInfo.of(spec.rowType), false, false, + spec.rowType, InternalTypeInfo.of(spec.rowType), false, false, spec.timestampFormat); try { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java index 1a4d22259..b7135097c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java @@ -20,7 +20,7 @@ import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; @@ -65,7 +65,7 @@ public void testDeserialization() throws Exception { List lines = readLines("canal-data.txt"); CanalJsonDeserializationSchema deserializationSchema = new CanalJsonDeserializationSchema( SCHEMA, - WrapperTypeInfo.of(SCHEMA), + InternalTypeInfo.of(SCHEMA), false, TimestampFormat.ISO_8601); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 3b97114f7..2d4761600 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -31,7 +31,7 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; @@ -65,7 +65,7 @@ public class CanalJsonFormatFactoryTest extends TestLogger { public void testSeDeSchema() { final CanalJsonDeserializationSchema expectedDeser = new CanalJsonDeserializationSchema( ROW_TYPE, - WrapperTypeInfo.of(ROW_TYPE), + InternalTypeInfo.of(ROW_TYPE), true, TimestampFormat.ISO_8601); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java index 3bcb16c9d..fa448888b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java @@ -20,7 +20,7 @@ import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; @@ -74,7 +74,7 @@ private void testDeserialization(String resourceFile, boolean schemaInclude) thr List lines = readLines(resourceFile); DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( SCHEMA, - WrapperTypeInfo.of(SCHEMA), + InternalTypeInfo.of(SCHEMA), schemaInclude, false, TimestampFormat.ISO_8601); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 134b7fb13..71cb13d67 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -31,7 +31,7 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.WrapperTypeInfo; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; @@ -65,7 +65,7 @@ public class DebeziumJsonFormatFactoryTest extends TestLogger { public void testSeDeSchema() { final DebeziumJsonDeserializationSchema expectedDeser = new DebeziumJsonDeserializationSchema( ROW_TYPE, - WrapperTypeInfo.of(ROW_TYPE), + InternalTypeInfo.of(ROW_TYPE), true, true, TimestampFormat.ISO_8601); From ed9f5af2db5456aca834abae6d94e8fc0ca6c0e6 Mon Sep 17 00:00:00 2001 From: Shengkai <1059623455@qq.com> Date: Tue, 23 Jun 2020 20:06:33 +0800 Subject: [PATCH 080/322] [FLINK-18296][json] Add support for TIMESTAMP_WITH_LOCAL_ZONE type for Json format This closes #12756 --- .../JsonRowDataDeserializationSchema.java | 23 +++++++++++ .../json/JsonRowDataSerializationSchema.java | 24 ++++++++++++ .../flink/formats/json/TimeFormats.java | 16 ++++++++ .../flink/formats/json/TimestampFormat.java | 8 ++-- .../json/JsonRowDataSerDeSchemaTest.java | 39 +++++++++++++++---- 5 files changed, 99 insertions(+), 11 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index 956ca5baa..0804ae3a9 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -51,6 +51,7 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.ZoneOffset; import java.time.temporal.TemporalAccessor; import java.time.temporal.TemporalQueries; import java.util.HashMap; @@ -61,7 +62,9 @@ import static java.lang.String.format; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -202,6 +205,8 @@ private DeserializationRuntimeConverter createNotNullConverter(LogicalType type) return this::convertToTime; case TIMESTAMP_WITHOUT_TIME_ZONE: return this::convertToTimestamp; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return this::convertToTimestampWithLocalZone; case FLOAT: return this::convertToFloat; case DOUBLE: @@ -303,6 +308,24 @@ private TimestampData convertToTimestamp(JsonNode jsonNode) { return TimestampData.fromLocalDateTime(LocalDateTime.of(localDate, localTime)); } + private TimestampData convertToTimestampWithLocalZone(JsonNode jsonNode){ + TemporalAccessor parsedTimestampWithLocalZone; + switch (timestampFormat){ + case SQL: + parsedTimestampWithLocalZone = SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); + break; + case ISO_8601: + parsedTimestampWithLocalZone = ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); + break; + default: + throw new TableException(String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); + } + LocalTime localTime = parsedTimestampWithLocalZone.query(TemporalQueries.localTime()); + LocalDate localDate = parsedTimestampWithLocalZone.query(TemporalQueries.localDate()); + + return TimestampData.fromInstant(LocalDateTime.of(localDate, localTime).toInstant(ZoneOffset.UTC)); + } + private StringData convertToString(JsonNode jsonNode) { if (jsonNode.isContainerNode()) { return StringData.fromString(jsonNode.toString()); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index 89b3b87af..ba35e7a8a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -42,12 +42,15 @@ import java.math.BigDecimal; import java.time.LocalDate; import java.time.LocalTime; +import java.time.ZoneOffset; import java.util.Arrays; import java.util.Objects; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; /** @@ -170,6 +173,8 @@ private SerializationRuntimeConverter createNotNullConverter(LogicalType type) { return createTimeConverter(); case TIMESTAMP_WITHOUT_TIME_ZONE: return createTimestampConverter(); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return createTimestampWithLocalZone(); case DECIMAL: return createDecimalConverter(); case ARRAY: @@ -227,6 +232,25 @@ private SerializationRuntimeConverter createTimestampConverter() { } } + private SerializationRuntimeConverter createTimestampWithLocalZone() { + switch (timestampFormat){ + case ISO_8601: + return (mapper, reuse, value) -> { + TimestampData timestampWithLocalZone = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format(timestampWithLocalZone.toInstant().atOffset(ZoneOffset.UTC))); + }; + case SQL: + return (mapper, reuse, value) -> { + TimestampData timestampWithLocalZone = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format(timestampWithLocalZone.toInstant().atOffset(ZoneOffset.UTC))); + }; + default: + throw new TableException("Unsupported timestamp format. Validator should have checked that."); + } + } + private SerializationRuntimeConverter createArrayConverter(ArrayType type) { final LogicalType elementType = type.getElementType(); final SerializationRuntimeConverter elementConverter = createConverter(elementType); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java index 693781ba3..f0b4b3adf 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java @@ -44,6 +44,14 @@ class TimeFormats { /** Formatter for ISO8601 string representation of a timestamp value (without UTC timezone). */ static final DateTimeFormatter ISO8601_TIMESTAMP_FORMAT = DateTimeFormatter.ISO_LOCAL_DATE_TIME; + /** Formatter for ISO8601 string representation of a timestamp value (with UTC timezone). */ + static final DateTimeFormatter ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT = new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral('T') + .append(DateTimeFormatter.ISO_LOCAL_TIME) + .appendPattern("'Z'") + .toFormatter(); + /** Formatter for SQL string representation of a time value. */ static final DateTimeFormatter SQL_TIME_FORMAT = new DateTimeFormatterBuilder() .appendPattern("HH:mm:ss") @@ -57,6 +65,14 @@ class TimeFormats { .append(SQL_TIME_FORMAT) .toFormatter(); + /** Formatter for SQL string representation of a timestamp value (with UTC timezone). */ + static final DateTimeFormatter SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT = new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(SQL_TIME_FORMAT) + .appendPattern("'Z'") + .toFormatter(); + private TimeFormats() { } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java index 0c31fd421..e9db8de2a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java @@ -25,11 +25,11 @@ */ @Internal public enum TimestampFormat { - /** Options to specify timestamp format. It will parse timestamp in "yyyy-MM-dd HH:mm:ss.s{precision}" format - * and output timestamp in the same format*/ + /** Options to specify TIMESTAMP/TIMESTAMP_WITH_LOCAL_ZONE format. It will parse TIMESTAMP in "yyyy-MM-dd HH:mm:ss.s{precision}" format, + * TIMESTAMP_WITH_LOCAL_TIMEZONE in "yyyy-MM-dd HH:mm:ss.s{precision}'Z'" and output in the same format.*/ SQL, - /** Options to specify timestamp format. It will parse timestamp in "yyyy-MM-ddTHH:mm:ss.s{precision}" format - * and output timestamp in the same format*/ + /** Options to specify TIMESTAMP/TIMESTAMP_WITH_LOCAL_ZONE format. It will pase TIMESTAMP in "yyyy-MM-ddTHH:mm:ss.s{precision}" format, + * TIMESTAMP_WITH_LOCAL_TIMEZONE in "yyyy-MM-ddTHH:mm:ss.s{precision}'Z'" and output in the same format.*/ ISO_8601 } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 58dc120b0..7a150c355 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -35,8 +35,11 @@ import java.math.BigDecimal; import java.sql.Timestamp; +import java.time.Instant; import java.time.LocalDate; +import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.ZoneOffset; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -60,9 +63,11 @@ import static org.apache.flink.table.api.DataTypes.STRING; import static org.apache.flink.table.api.DataTypes.TIME; import static org.apache.flink.table.api.DataTypes.TIMESTAMP; +import static org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE; import static org.apache.flink.table.api.DataTypes.TINYINT; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; /** * Tests for {@link JsonRowDataDeserializationSchema} and {@link JsonRowDataSerializationSchema}. @@ -85,6 +90,9 @@ public void testSerDe() throws Exception { LocalTime time = LocalTime.parse("12:12:43"); Timestamp timestamp3 = Timestamp.valueOf("1990-10-14 12:12:43.123"); Timestamp timestamp9 = Timestamp.valueOf("1990-10-14 12:12:43.123456789"); + Instant timestampWithLocalZone = + LocalDateTime.of(1990, 10, 14, 12, 12, 43, 123456789). + atOffset(ZoneOffset.of("Z")).toInstant(); Map map = new HashMap<>(); map.put("flink", 123L); @@ -113,6 +121,7 @@ public void testSerDe() throws Exception { root.put("time", "12:12:43"); root.put("timestamp3", "1990-10-14T12:12:43.123"); root.put("timestamp9", "1990-10-14T12:12:43.123456789"); + root.put("timestampWithLocalZone", "1990-10-14T12:12:43.123456789Z"); root.putObject("map").put("flink", 123); root.putObject("map2map").putObject("inner_map").put("key", 234); @@ -133,6 +142,7 @@ public void testSerDe() throws Exception { FIELD("time", TIME(0)), FIELD("timestamp3", TIMESTAMP(3)), FIELD("timestamp9", TIMESTAMP(9)), + FIELD("timestampWithLocalZone", TIMESTAMP_WITH_LOCAL_TIME_ZONE(9)), FIELD("map", MAP(STRING(), BIGINT())), FIELD("map2map", MAP(STRING(), MAP(STRING(), INT())))); RowType schema = (RowType) dataType.getLogicalType(); @@ -141,7 +151,7 @@ public void testSerDe() throws Exception { JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); - Row expected = new Row(16); + Row expected = new Row(17); expected.setField(0, true); expected.setField(1, tinyint); expected.setField(2, smallint); @@ -156,8 +166,9 @@ public void testSerDe() throws Exception { expected.setField(11, time); expected.setField(12, timestamp3.toLocalDateTime()); expected.setField(13, timestamp9.toLocalDateTime()); - expected.setField(14, map); - expected.setField(15, nestedMap); + expected.setField(14, timestampWithLocalZone); + expected.setField(15, map); + expected.setField(16, nestedMap); RowData rowData = deserializationSchema.deserialize(serializedJson); Row actual = convertToExternal(rowData, dataType); @@ -324,7 +335,7 @@ public void testDeserializationMissingNode() throws Exception { String errorMessage = "Failed to deserialize JSON '{\"id\":123123123}'."; try { deserializationSchema.deserialize(serializedJson); - Assert.fail("expecting exception message: " + errorMessage); + fail("expecting exception message: " + errorMessage); } catch (Throwable t) { assertEquals(errorMessage, t.getMessage()); } @@ -350,7 +361,9 @@ public void testDeserializationMissingNode() throws Exception { public void testSerDeSQLTimestampFormat() throws Exception{ RowType rowType = (RowType) ROW( FIELD("timestamp3", TIMESTAMP(3)), - FIELD("timestamp9", TIMESTAMP(9)) + FIELD("timestamp9", TIMESTAMP(9)), + FIELD("timestamp_with_local_timezone3", TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), + FIELD("timestamp_with_local_timezone9", TIMESTAMP_WITH_LOCAL_TIME_ZONE(9)) ).getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( @@ -362,6 +375,8 @@ public void testSerDeSQLTimestampFormat() throws Exception{ ObjectNode root = objectMapper.createObjectNode(); root.put("timestamp3", "1990-10-14 12:12:43.123"); root.put("timestamp9", "1990-10-14 12:12:43.123456789"); + root.put("timestamp_with_local_timezone3", "1990-10-14 12:12:43.123Z"); + root.put("timestamp_with_local_timezone9", "1990-10-14 12:12:43.123456789Z"); byte[] serializedJson = objectMapper.writeValueAsBytes(root); RowData rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); @@ -404,7 +419,7 @@ private void testParseErrors(TestSpec spec) throws Exception { try { failingSchema.deserialize(spec.json.getBytes()); - Assert.fail("expecting exception " + spec.errorMessage); + fail("expecting exception " + spec.errorMessage); } catch (Throwable t) { assertEquals(t.getMessage(), spec.errorMessage); } @@ -538,7 +553,17 @@ private void testParseErrors(TestSpec spec) throws Exception { .json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") .rowType(ROW(FIELD("map", MAP(STRING(), INT())))) .expect(Row.of(createHashMap("key1", 123, "key2", null))) - .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'.") + .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'."), + + TestSpec + .json("{\"id\":\"2019-11-12T18:00:12\"}") + .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0)))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."), + + TestSpec + .json("{\"id\":\"2019-11-12T18:00:12+0800\"}") + .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0)))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12+0800\"}'.") ); private static Map createHashMap(String k1, Integer v1, String k2, Integer v2) { From 7006696e990ad32941d087dd110fbfe98ecf8094 Mon Sep 17 00:00:00 2001 From: Niels Basjes Date: Wed, 22 Jul 2020 11:26:03 +0200 Subject: [PATCH 081/322] [FLINK-18607][build] Give the maven module a human readable name This closes #12907 --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 ++ flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 ++ flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index a143bc5b6..3ffec7851 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -27,6 +27,8 @@ under the License. 4.0.0 flink-confluent-schema-registry + Flink : E2E Tests : Confluent schema registry + UTF-8 4.1.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 8e39402d0..89438c27d 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -29,6 +29,8 @@ under the License. flink-avro-confluent-registry + Flink : Formats : Avro confluent registry + 4.1.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 67e2a8058..81424eb57 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -30,7 +30,7 @@ under the License. flink-json - flink-json + Flink : Formats : Json jar From 1380f0ed8d617006a9b86d9dec369939c999c8d6 Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Thu, 30 Jul 2020 21:09:26 +0800 Subject: [PATCH 082/322] [FLINK-16048][avro] Support read/write confluent schema registry avro data from Kafka --- .../flink-avro-confluent-registry/pom.xml | 56 ++++- .../confluent/CachedSchemaCoderProvider.java | 76 +++++++ ...uentRegistryAvroDeserializationSchema.java | 20 -- ...fluentRegistryAvroSerializationSchema.java | 22 -- .../confluent/RegistryAvroFormatFactory.java | 145 +++++++++++++ .../confluent/RegistryAvroOptions.java | 39 ++++ .../src/main/resources/META-INF/NOTICE | 4 +- .../org.apache.flink.table.factories.Factory | 16 ++ .../RegistryAvroFormatFactoryTest.java | 177 ++++++++++++++++ .../RegistryAvroRowDataSeDeSchemaTest.java | 199 ++++++++++++++++++ 10 files changed, 707 insertions(+), 47 deletions(-) create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 89438c27d..e60f98126 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -32,7 +32,8 @@ under the License. Flink : Formats : Avro confluent registry - 4.1.0 + 2.4.1 + 5.4.2 @@ -46,7 +47,7 @@ under the License. io.confluent kafka-schema-registry-client - ${confluent.schema.registry.version} + ${confluent.version} org.apache.avro @@ -56,6 +57,14 @@ under the License. org.slf4j slf4j-log4j12 + + org.lz4 + lz4-java + + + io.swagger + swagger-core + @@ -69,7 +78,48 @@ under the License. flink-avro ${project.version} - + + org.apache.flink + flink-table-api-java + ${project.version} + provided + + + org.apache.flink + flink-table-common + ${project.version} + provided + + + + + org.apache.flink + flink-table-api-java + ${project.version} + test + test-jar + + + org.apache.flink + flink-table-runtime-blink_2.11 + ${project.version} + test + + + org.apache.flink + flink-table-common + ${project.version} + test + test-jar + + + org.apache.flink + flink-avro + ${project.version} + test + test-jar + + diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java new file mode 100644 index 000000000..a739ae17a --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.formats.avro.SchemaCoder; + +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; + +import javax.annotation.Nullable; + +import java.util.Objects; + +/** A {@link SchemaCoder.SchemaCoderProvider} that uses a cached schema registry + * client underlying. **/ +@Internal +class CachedSchemaCoderProvider implements SchemaCoder.SchemaCoderProvider { + + private static final long serialVersionUID = 8610401613495438381L; + private final String subject; + private final String url; + private final int identityMapCapacity; + + CachedSchemaCoderProvider(String url, int identityMapCapacity) { + this(null, url, identityMapCapacity); + } + + CachedSchemaCoderProvider(@Nullable String subject, String url, int identityMapCapacity) { + this.subject = subject; + this.url = Objects.requireNonNull(url); + this.identityMapCapacity = identityMapCapacity; + } + + @Override + public SchemaCoder get() { + return new ConfluentSchemaRegistryCoder(this.subject, + new CachedSchemaRegistryClient( + url, + identityMapCapacity)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CachedSchemaCoderProvider that = (CachedSchemaCoderProvider) o; + return identityMapCapacity == that.identityMapCapacity && + Objects.equals(subject, that.subject) && + url.equals(that.url); + } + + @Override + public int hashCode() { + return Objects.hash(subject, url, identityMapCapacity); + } +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java index 1135bb998..c5c421e80 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java @@ -22,7 +22,6 @@ import org.apache.flink.formats.avro.RegistryAvroDeserializationSchema; import org.apache.flink.formats.avro.SchemaCoder; -import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.specific.SpecificRecord; @@ -114,23 +113,4 @@ public static ConfluentRegistryAvroDeserializationSch new CachedSchemaCoderProvider(url, identityMapCapacity) ); } - - private static class CachedSchemaCoderProvider implements SchemaCoder.SchemaCoderProvider { - - private static final long serialVersionUID = 4023134423033312666L; - private final String url; - private final int identityMapCapacity; - - CachedSchemaCoderProvider(String url, int identityMapCapacity) { - this.url = url; - this.identityMapCapacity = identityMapCapacity; - } - - @Override - public SchemaCoder get() { - return new ConfluentSchemaRegistryCoder(new CachedSchemaRegistryClient( - url, - identityMapCapacity)); - } - } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java index b5680b0d8..0afbd885d 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java @@ -22,7 +22,6 @@ import org.apache.flink.formats.avro.RegistryAvroSerializationSchema; import org.apache.flink.formats.avro.SchemaCoder; -import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.specific.SpecificRecord; @@ -92,25 +91,4 @@ public static ConfluentRegistryAvroSerializationSchema forGeneric new CachedSchemaCoderProvider(subject, schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY) ); } - - private static class CachedSchemaCoderProvider implements SchemaCoder.SchemaCoderProvider { - - private static final long serialVersionUID = 4023134423033312666L; - private final String subject; - private final String url; - private final int identityMapCapacity; - - CachedSchemaCoderProvider(String subject, String url, int identityMapCapacity) { - this.subject = subject; - this.url = url; - this.identityMapCapacity = identityMapCapacity; - } - - @Override - public SchemaCoder get() { - return new ConfluentSchemaRegistryCoder(subject, new CachedSchemaRegistryClient( - url, - identityMapCapacity)); - } - } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java new file mode 100644 index 000000000..9a134254e --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; +import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; +import org.apache.flink.formats.avro.AvroToRowDataConverters; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; + +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_SUBJECT; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_URL; + +/** + * Table format factory for providing configured instances of Schema Registry Avro to RowData + * {@link SerializationSchema} and {@link DeserializationSchema}. + */ +public class RegistryAvroFormatFactory implements + DeserializationFormatFactory, + SerializationFormatFactory { + + public static final String IDENTIFIER = "avro-confluent"; + + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + + String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); + return new DecodingFormat>() { + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, + DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + (TypeInformation) context.createTypeInformation(producedDataType); + return new AvroRowDataDeserializationSchema( + ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(rowType), + schemaRegistryURL), + AvroToRowDataConverters.createRowConverter(rowType), + rowDataTypeInfo); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + }; + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + + String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); + Optional subject = formatOptions.getOptional(SCHEMA_REGISTRY_SUBJECT); + if (!subject.isPresent()) { + throw new ValidationException(String.format("Option %s.%s is required for serialization", + IDENTIFIER, SCHEMA_REGISTRY_SUBJECT.key())); + } + + return new EncodingFormat>() { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, + DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new AvroRowDataSerializationSchema( + rowType, + ConfluentRegistryAvroSerializationSchema.forGeneric( + subject.get(), + AvroSchemaConverter.convertToSchema(rowType), + schemaRegistryURL), + RowDataToAvroConverters.createRowConverter(rowType)); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + Set> options = new HashSet<>(); + options.add(SCHEMA_REGISTRY_URL); + return options; + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(SCHEMA_REGISTRY_SUBJECT); + return options; + } +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java new file mode 100644 index 000000000..a3739c144 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** Options for Schema Registry Avro format. */ +public class RegistryAvroOptions { + private RegistryAvroOptions () {} + + public static final ConfigOption SCHEMA_REGISTRY_URL = ConfigOptions + .key("schema-registry.url") + .stringType() + .noDefaultValue() + .withDescription("Required URL to connect to schema registry service"); + + public static final ConfigOption SCHEMA_REGISTRY_SUBJECT = ConfigOptions + .key("schema-registry.subject") + .stringType() + .noDefaultValue() + .withDescription("Subject name to write to the Schema Registry service, required for sink"); +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 82acd60fc..a5f4b4f3c 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -10,6 +10,6 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-databind:2.10.1 - com.fasterxml.jackson.core:jackson-annotations:2.10.1 - com.fasterxml.jackson.core:jackson-core:2.10.1 -- io.confluent:common-utils:4.1.0 -- io.confluent:kafka-schema-registry-client:4.1.0 +- io.confluent:common-utils:5.4.2 +- io.confluent:kafka-schema-registry-client:5.4.2 - org.apache.zookeeper:zookeeper:3.4.10 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 000000000..57dd1ba79 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.formats.avro.registry.confluent.RegistryAvroFormatFactory diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java new file mode 100644 index 000000000..dbfcc794c --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; +import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; +import org.apache.flink.formats.avro.AvroToRowDataConverters; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.Consumer; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +/** + * Tests for the {@link RegistryAvroFormatFactory}. + */ +public class RegistryAvroFormatFactoryTest { + private static final TableSchema SCHEMA = TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + private static final String SUBJECT = "test-subject"; + private static final String REGISTRY_URL = "http://localhost:8081"; + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testDeserializationSchema() { + final AvroRowDataDeserializationSchema expectedDeser = + new AvroRowDataDeserializationSchema( + ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(ROW_TYPE), + REGISTRY_URL), + AvroToRowDataConverters.createRowConverter(ROW_TYPE), + InternalTypeInfo.of(ROW_TYPE)); + + final DynamicTableSource actualSource = createTableSource(getDefaultOptions()); + assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = scanSourceMock.valueFormat + .createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toRowDataType()); + + assertEquals(expectedDeser, actualDeser); + } + + @Test + public void testSerializationSchema() { + final AvroRowDataSerializationSchema expectedSer = + new AvroRowDataSerializationSchema( + ROW_TYPE, + ConfluentRegistryAvroSerializationSchema.forGeneric( + SUBJECT, + AvroSchemaConverter.convertToSchema(ROW_TYPE), + REGISTRY_URL), + RowDataToAvroConverters.createRowConverter(ROW_TYPE)); + + final DynamicTableSink actualSink = createTableSink(getDefaultOptions()); + assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = sinkMock.valueFormat + .createRuntimeEncoder( + null, + SCHEMA.toRowDataType()); + + assertEquals(expectedSer, actualSer); + } + + @Test + public void testMissingSubjectForSink() { + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException("Option avro-confluent.schema-registry.subject " + + "is required for serialization"))); + + final Map options = + getModifiedOptions(opts -> opts.remove("avro-confluent.schema-registry.subject")); + + createTableSink(options); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifiedOptions(Consumer> optionModifier) { + Map options = getDefaultOptions(); + optionModifier.accept(options); + return options; + } + + private Map getDefaultOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", RegistryAvroFormatFactory.IDENTIFIER); + options.put("avro-confluent.schema-registry.subject", SUBJECT); + options.put("avro-confluent.schema-registry.url", REGISTRY_URL); + return options; + } + + private DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + RegistryAvroFormatFactoryTest.class.getClassLoader()); + } + + private DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + RegistryAvroFormatFactoryTest.class.getClassLoader()); + } +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java new file mode 100644 index 000000000..1cf348a9f --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent; + +import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; +import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; +import org.apache.flink.formats.avro.AvroToRowDataConverters; +import org.apache.flink.formats.avro.RegistryAvroDeserializationSchema; +import org.apache.flink.formats.avro.RegistryAvroSerializationSchema; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.generated.Address; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.avro.utils.TestDataGenerator; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryStringData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.IOException; +import java.util.Random; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.formats.avro.utils.AvroTestUtils.writeRecord; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link AvroRowDataDeserializationSchema} and + * {@link AvroRowDataSerializationSchema} for schema registry avro. + */ +public class RegistryAvroRowDataSeDeSchemaTest { + private static final Schema ADDRESS_SCHEMA = Address.getClassSchema(); + + private static final Schema ADDRESS_SCHEMA_COMPATIBLE = new Schema.Parser().parse( + "" + + "{\"namespace\": \"org.apache.flink.formats.avro.generated\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"Address\",\n" + + " \"fields\": [\n" + + " {\"name\": \"num\", \"type\": \"int\"},\n" + + " {\"name\": \"street\", \"type\": \"string\"}\n" + + " ]\n" + + "}"); + + private static final String SUBJECT = "address-value"; + + private static SchemaRegistryClient client; + + private Address address; + + @Rule + public ExpectedException expectedEx = ExpectedException.none(); + + @BeforeClass + public static void beforeClass() { + client = new MockSchemaRegistryClient(); + } + + @Before + public void before() { + this.address = TestDataGenerator.generateRandomAddress(new Random()); + } + + @After + public void after() throws IOException, RestClientException { + client.deleteSubject(SUBJECT); + } + + @Test + public void testRowDataWriteReadWithFullSchema() throws Exception { + testRowDataWriteReadWithSchema(ADDRESS_SCHEMA); + } + + @Test + public void testRowDataWriteReadWithCompatibleSchema() throws Exception { + testRowDataWriteReadWithSchema(ADDRESS_SCHEMA_COMPATIBLE); + // Validates new schema has been registered. + assertThat(client.getAllVersions(SUBJECT).size(), is(1)); + } + + @Test + public void testRowDataWriteReadWithPreRegisteredSchema() throws Exception { + client.register(SUBJECT, ADDRESS_SCHEMA); + testRowDataWriteReadWithSchema(ADDRESS_SCHEMA); + // Validates it does not produce new schema. + assertThat(client.getAllVersions(SUBJECT).size(), is(1)); + } + + @Test + public void testRowDataReadWithNonRegistryAvro() throws Exception { + DataType dataType = AvroSchemaConverter.convertToDataType(ADDRESS_SCHEMA.toString()); + RowType rowType = (RowType) dataType.getLogicalType(); + + AvroRowDataDeserializationSchema deserializer = getDeserializationSchema(rowType, ADDRESS_SCHEMA); + + deserializer.open(null); + + client.register(SUBJECT, ADDRESS_SCHEMA); + byte[] oriBytes = writeRecord(address, ADDRESS_SCHEMA); + expectedEx.expect(IOException.class); + expectedEx.expect(containsCause(new IOException("Unknown data format. Magic number does not match"))); + deserializer.deserialize(oriBytes); + } + + private void testRowDataWriteReadWithSchema(Schema schema) throws Exception { + DataType dataType = AvroSchemaConverter.convertToDataType(schema.toString()); + RowType rowType = (RowType) dataType.getLogicalType(); + + AvroRowDataSerializationSchema serializer = getSerializationSchema(rowType, schema); + AvroRowDataDeserializationSchema deserializer = getDeserializationSchema(rowType, schema); + + serializer.open(null); + deserializer.open(null); + + RowData oriData = address2RowData(address); + byte[] serialized = serializer.serialize(oriData); + RowData rowData = deserializer.deserialize(serialized); + assertThat(rowData.getArity(), equalTo(schema.getFields().size())); + assertEquals(address.getNum(), Integer.valueOf(rowData.getInt(0))); + assertEquals(address.getStreet(), rowData.getString(1).toString()); + if (schema != ADDRESS_SCHEMA_COMPATIBLE) { + assertEquals(address.getCity(), rowData.getString(2).toString()); + assertEquals(address.getState(), rowData.getString(3).toString()); + assertEquals(address.getZip(), rowData.getString(4).toString()); + } + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + private static AvroRowDataSerializationSchema getSerializationSchema( + RowType rowType, + Schema avroSchema) { + ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); + return new AvroRowDataSerializationSchema( + rowType, + new RegistryAvroSerializationSchema( + GenericRecord.class, + avroSchema, + () -> registryCoder), + RowDataToAvroConverters.createRowConverter(rowType)); + } + + private static AvroRowDataDeserializationSchema getDeserializationSchema( + RowType rowType, + Schema avroSchema) { + ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); + return new AvroRowDataDeserializationSchema( + new RegistryAvroDeserializationSchema( + GenericRecord.class, + avroSchema, + () -> registryCoder), + AvroToRowDataConverters.createRowConverter(rowType), + InternalTypeInfo.of(rowType)); + } + + private static RowData address2RowData(Address address) { + GenericRowData rowData = new GenericRowData(5); + rowData.setField(0, address.getNum()); + rowData.setField(1, new BinaryStringData(address.getStreet().toString())); + rowData.setField(2, new BinaryStringData(address.getCity().toString())); + rowData.setField(3, new BinaryStringData(address.getState().toString())); + rowData.setField(4, new BinaryStringData(address.getZip().toString())); + return rowData; + } +} From ae8ac9404bc9e276f60dacbccb8e751f7616beda Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 31 Jul 2020 10:38:32 +0800 Subject: [PATCH 083/322] [FLINK-18776][avro] Avoid hardcoded scala version This closes #13032. --- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index e60f98126..235e45190 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -101,7 +101,7 @@ under the License. org.apache.flink - flink-table-runtime-blink_2.11 + flink-table-runtime-blink_${scala.binary.version} ${project.version} test From 5561317e67ba51c537b71ab29ea385061b6117c1 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 29 Jul 2020 17:19:35 +0800 Subject: [PATCH 084/322] [FLINK-18700][debezium] Debezium-json format throws NPE when PG table's IDENTITY config is not FULL This commit add documentation for this case and throws a guide message in the exception. This closes #13019 --- .../DebeziumJsonDeserializationSchema.java | 10 ++++++++ ...DebeziumJsonDeserializationSchemaTest.java | 23 +++++++++++++++++++ ...ebezium-postgres-data-replica-identity.txt | 16 +++++++++++++ .../debezium-postgres-data-schema-exclude.txt | 16 +++++++++++++ .../debezium-postgres-data-schema-include.txt | 16 +++++++++++++ 5 files changed, 81 insertions(+) create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-replica-identity.txt create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-exclude.txt create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-include.txt diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java index d1f1d96ac..81c4af020 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java @@ -58,6 +58,10 @@ public final class DebeziumJsonDeserializationSchema implements DeserializationS private static final String OP_UPDATE = "u"; // update private static final String OP_DELETE = "d"; // delete + private static final String REPLICA_IDENTITY_EXCEPTION = "The \"before\" field of %s message is null, " + + "if you are using Debezium Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; + /** The deserializer to deserialize Debezium JSON data. */ private final JsonRowDataDeserializationSchema jsonDeserializer; @@ -117,11 +121,17 @@ public void deserialize(byte[] message, Collector out) throws IOExcepti after.setRowKind(RowKind.INSERT); out.collect(after); } else if (OP_UPDATE.equals(op)) { + if (before == null) { + throw new IllegalStateException(String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } before.setRowKind(RowKind.UPDATE_BEFORE); after.setRowKind(RowKind.UPDATE_AFTER); out.collect(before); out.collect(after); } else if (OP_DELETE.equals(op)) { + if (before == null) { + throw new IllegalStateException(String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); + } before.setRowKind(RowKind.DELETE); out.collect(before); } else { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java index fa448888b..0c98c8c34 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java @@ -23,6 +23,7 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; +import org.apache.flink.util.ExceptionUtils; import org.junit.Rule; import org.junit.Test; @@ -44,6 +45,7 @@ import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** * Tests for {@link DebeziumJsonDeserializationSchema}. @@ -70,6 +72,27 @@ public void testSchemaExcludeDeserialization() throws Exception { testDeserialization("debezium-data-schema-exclude.txt", false); } + @Test + public void testPostgresSchemaIncludeDeserialization() throws Exception { + testDeserialization("debezium-postgres-data-schema-include.txt", true); + } + + @Test + public void testPostgresSchemaExcludeDeserialization() throws Exception { + testDeserialization("debezium-postgres-data-schema-exclude.txt", false); + } + + @Test + public void testPostgresDefaultReplicaIdentify() throws Exception { + try { + testDeserialization("debezium-postgres-data-replica-identity.txt", false); + } catch (Exception e) { + assertTrue(ExceptionUtils.findThrowableWithMessage(e, + "The \"before\" field of UPDATE message is null, if you are using Debezium Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level.").isPresent()); + } + } + private void testDeserialization(String resourceFile, boolean schemaInclude) throws Exception { List lines = readLines(resourceFile); DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-replica-identity.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-replica-identity.txt new file mode 100644 index 000000000..d4fcb8898 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-replica-identity.txt @@ -0,0 +1,16 @@ +{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099434,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099434,"transaction":null} +{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null} +{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null} +{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} +{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} +{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} +{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null} +{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null} +{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099438,"transaction":null} +{"before":null,"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010889629,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":602,"lsn":34131104,"xmin":null},"op":"u","ts_ms":1596010890411,"transaction":null} +{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010930407,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":603,"lsn":34132200,"xmin":null},"op":"u","ts_ms":1596010930623,"transaction":null} +{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010946488,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":604,"lsn":34132560,"xmin":null},"op":"c","ts_ms":1596010946870,"transaction":null} +{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010976756,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":605,"lsn":34133072,"xmin":null},"op":"c","ts_ms":1596010976880,"transaction":null} +{"before":null,"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010982228,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":606,"lsn":34133344,"xmin":null},"op":"u","ts_ms":1596010982481,"transaction":null} +{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010985627,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":607,"lsn":34133584,"xmin":null},"op":"u","ts_ms":1596010986047,"transaction":null} +{"before":null,"after":null,"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010988168,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":608,"lsn":34133800,"xmin":null},"op":"d","ts_ms":1596010988596,"transaction":null} \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-exclude.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-exclude.txt new file mode 100644 index 000000000..993f5f489 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-exclude.txt @@ -0,0 +1,16 @@ +{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099434,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099434,"transaction":null} +{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null} +{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null} +{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} +{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} +{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} +{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null} +{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null} +{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099438,"transaction":null} +{"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010889629,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":602,"lsn":34131104,"xmin":null},"op":"u","ts_ms":1596010890411,"transaction":null} +{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010930407,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":603,"lsn":34132200,"xmin":null},"op":"u","ts_ms":1596010930623,"transaction":null} +{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010946488,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":604,"lsn":34132560,"xmin":null},"op":"c","ts_ms":1596010946870,"transaction":null} +{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010976756,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":605,"lsn":34133072,"xmin":null},"op":"c","ts_ms":1596010976880,"transaction":null} +{"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010982228,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":606,"lsn":34133344,"xmin":null},"op":"u","ts_ms":1596010982481,"transaction":null} +{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010985627,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":607,"lsn":34133584,"xmin":null},"op":"u","ts_ms":1596010986047,"transaction":null} +{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"after":null,"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010988168,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":608,"lsn":34133800,"xmin":null},"op":"d","ts_ms":1596010988596,"transaction":null} \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-include.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-include.txt new file mode 100644 index 000000000..8301935f3 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-include.txt @@ -0,0 +1,16 @@ +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099434,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099434,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099438,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010889629,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":602,"lsn":34131104,"xmin":null},"op":"u","ts_ms":1596010890411,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010930407,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":603,"lsn":34132200,"xmin":null},"op":"u","ts_ms":1596010930623,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010946488,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":604,"lsn":34132560,"xmin":null},"op":"c","ts_ms":1596010946870,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010976756,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":605,"lsn":34133072,"xmin":null},"op":"c","ts_ms":1596010976880,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010982228,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":606,"lsn":34133344,"xmin":null},"op":"u","ts_ms":1596010982481,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010985627,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":607,"lsn":34133584,"xmin":null},"op":"u","ts_ms":1596010986047,"transaction":null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"after":null,"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010988168,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":608,"lsn":34133800,"xmin":null},"op":"d","ts_ms":1596010988596,"transaction":null}} \ No newline at end of file From b76981ee90a63116759c217809bb25822cd651ca Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 29 Jul 2020 17:20:23 +0800 Subject: [PATCH 085/322] [FLINK-18705][debezium] Fix Debezium-JSON throws NPE when tombstone message is received Just skip the tombstone messages This closes #13019 --- .../DebeziumJsonDeserializationSchema.java | 4 ++++ .../DebeziumJsonDeserializationSchemaTest.java | 14 ++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java index 81c4af020..deb3653e6 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java @@ -105,6 +105,10 @@ public RowData deserialize(byte[] message) throws IOException { @Override public void deserialize(byte[] message, Collector out) throws IOException { + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } try { GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); GenericRowData payload; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java index 0c98c8c34..edaddbfcc 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java @@ -93,6 +93,20 @@ public void testPostgresDefaultReplicaIdentify() throws Exception { } } + @Test + public void testTombstoneMessages() throws Exception { + DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( + SCHEMA, + InternalTypeInfo.of(SCHEMA), + false, + false, + TimestampFormat.ISO_8601); + SimpleCollector collector = new SimpleCollector(); + deserializationSchema.deserialize(null, collector); + deserializationSchema.deserialize(new byte[]{}, collector); + assertTrue(collector.list.isEmpty()); + } + private void testDeserialization(String resourceFile, boolean schemaInclude) throws Exception { List lines = readLines(resourceFile); DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( From 526003391a7cf7a595917c3e4bdec21401e07a1b Mon Sep 17 00:00:00 2001 From: dijie Date: Wed, 12 Aug 2020 11:11:58 +0800 Subject: [PATCH 086/322] [FLINK-18844][json][maxwell] Support maxwell-json format to read Maxwell changelogs This closes #13090 --- .../MaxwellJsonDeserializationSchema.java | 179 ++++++++++++++++ .../maxwell/MaxwellJsonFormatFactory.java | 139 ++++++++++++ .../MaxwellJsonSerializationSchema.java | 111 ++++++++++ .../org.apache.flink.table.factories.Factory | 1 + .../maxwell/MaxwellJsonFormatFactoryTest.java | 159 ++++++++++++++ .../json/maxwell/MaxwellJsonSerDerTest.java | 198 ++++++++++++++++++ .../src/test/resources/maxwell-data.txt | 20 ++ 7 files changed, 807 insertions(+) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java new file mode 100644 index 000000000..c34ab5597 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.maxwell; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.Objects; + +import static java.lang.String.format; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Deserialization schema from Maxwell JSON to Flink Table/SQL internal data structure {@link RowData}. + * The deserialization schema knows Maxwell's schema definition and can extract the database data + * and convert into {@link RowData} with {@link RowKind}. + * + *

Deserializes a byte[] message as a JSON object and reads + * the specified fields. + * + *

Failures during deserialization are forwarded as wrapped IOExceptions. + * + * @see Maxwell + */ +public class MaxwellJsonDeserializationSchema implements DeserializationSchema { + private static final long serialVersionUID = 1L; + + private static final String OP_INSERT = "insert"; + private static final String OP_UPDATE = "update"; + private static final String OP_DELETE = "delete"; + + /** The deserializer to deserialize Maxwell JSON data. */ + private final JsonRowDataDeserializationSchema jsonDeserializer; + + /** TypeInformation of the produced {@link RowData}. **/ + private final TypeInformation resultTypeInfo; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + /** Number of fields. */ + private final int fieldCount; + + public MaxwellJsonDeserializationSchema(RowType rowType, + TypeInformation resultTypeInfo, + boolean ignoreParseErrors, + TimestampFormat timestampFormatOption) { + this.resultTypeInfo = resultTypeInfo; + this.ignoreParseErrors = ignoreParseErrors; + this.fieldCount = rowType.getFieldCount(); + this.jsonDeserializer = new JsonRowDataDeserializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + // the result type is never used, so it's fine to pass in Canal's result type + resultTypeInfo, + false, // ignoreParseErrors already contains the functionality of failOnMissingField + ignoreParseErrors, + timestampFormatOption); + } + + @Override + public RowData deserialize(byte[] message) throws IOException { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + + @Override + public void deserialize(byte[] message, Collector out) throws IOException { + try { + RowData row = jsonDeserializer.deserialize(message); + String type = row.getString(2).toString(); // "type" field + if (OP_INSERT.equals(type)) { + // "data" field is a row, contains inserted rows + RowData insert = row.getRow(0, fieldCount); + insert.setRowKind(RowKind.INSERT); + out.collect(insert); + } else if (OP_UPDATE.equals(type)) { + // "data" field is a row, contains new rows + // "old" field is an array of row, contains old values + // the underlying JSON deserialization schema always produce GenericRowData. + GenericRowData after = (GenericRowData) row.getRow(0, fieldCount); // "data" field + GenericRowData before = (GenericRowData) row.getRow(1, fieldCount); // "old" field + for (int f = 0; f < fieldCount; f++) { + if (before.isNullAt(f)) { + // not null fields in "old" (before) means the fields are changed + // null/empty fields in "old" (before) means the fields are not changed + // so we just copy the not changed fields into before + before.setField(f, after.getField(f)); + } + } + before.setRowKind(RowKind.UPDATE_BEFORE); + after.setRowKind(RowKind.UPDATE_AFTER); + out.collect(before); + out.collect(after); + } else if (OP_DELETE.equals(type)) { + // "data" field is a row, contains deleted rows + RowData delete = row.getRow(0, fieldCount); + delete.setRowKind(RowKind.DELETE); + out.collect(delete); + + } else { + if (!ignoreParseErrors) { + throw new IOException(format( + "Unknown \"type\" value \"%s\". The Maxwell JSON message is '%s'", type, new String(message))); + } + } + } catch (Throwable t) { + // a big try catch to protect the processing. + if (!ignoreParseErrors) { + throw new IOException(format( + "Corrupt Maxwell JSON message '%s'.", new String(message)), t); + } + } + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return resultTypeInfo; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MaxwellJsonDeserializationSchema that = (MaxwellJsonDeserializationSchema) o; + return ignoreParseErrors == that.ignoreParseErrors && + fieldCount == that.fieldCount && + Objects.equals(jsonDeserializer, that.jsonDeserializer) && + Objects.equals(resultTypeInfo, that.resultTypeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors, fieldCount); + } + + private RowType createJsonRowType(DataType databaseSchema) { + // Maxwell JSON contains other information, e.g. "database", "ts" + // but we don't need them + return (RowType) DataTypes.ROW( + DataTypes.FIELD("data", databaseSchema), + DataTypes.FIELD("old", databaseSchema), + DataTypes.FIELD("type", DataTypes.STRING())).getLogicalType(); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java new file mode 100644 index 000000000..e3df0434c --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.maxwell; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * Format factory for providing configured instances of Maxwell JSON to RowData {@link DeserializationSchema}. + */ +public class MaxwellJsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "maxwell-json"; + + public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + + public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + + @SuppressWarnings("unchecked") + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); + + return new DecodingFormat>() { + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + (TypeInformation) context.createTypeInformation(producedDataType); + return new MaxwellJsonDeserializationSchema( + rowType, + rowDataTypeInfo, + ignoreParseErrors, + timestampFormatOption); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + }; + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + + return new EncodingFormat>() { + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new MaxwellJsonSerializationSchema( + rowType, + timestampFormat); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); + return options; + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java new file mode 100644 index 000000000..be9df5edc --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.maxwell; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.json.JsonRowDataSerializationSchema; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Objects; + +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to maxwell-Json. + */ +public class MaxwellJsonSerializationSchema implements SerializationSchema { + private static final long serialVersionUID = 1L; + + private static final StringData OP_INSERT = StringData.fromString("insert"); + private static final StringData OP_DELETE = StringData.fromString("delete"); + + private final JsonRowDataSerializationSchema jsonSerializer; + + /** + * Timestamp format specification which is used to parse timestamp. + */ + private final TimestampFormat timestampFormat; + + private transient GenericRowData reuse; + + public MaxwellJsonSerializationSchema(RowType rowType, TimestampFormat timestampFormat) { + this.jsonSerializer = new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat); + this.timestampFormat = timestampFormat; + } + + @Override + public void open(InitializationContext context) throws Exception { + this.reuse = new GenericRowData(2); + } + + @Override + public byte[] serialize(RowData element) { + reuse.setField(0, element); + reuse.setField(1, rowKind2String(element.getRowKind())); + return jsonSerializer.serialize(reuse); + } + + private StringData rowKind2String(RowKind rowKind) { + switch (rowKind) { + case INSERT: + case UPDATE_AFTER: + return OP_INSERT; + case UPDATE_BEFORE: + case DELETE: + return OP_DELETE; + default: + throw new UnsupportedOperationException("Unsupported operation '" + rowKind + "' for row kind."); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MaxwellJsonSerializationSchema that = (MaxwellJsonSerializationSchema) o; + return Objects.equals(jsonSerializer, that.jsonSerializer) && + timestampFormat == that.timestampFormat; + } + + @Override + public int hashCode() { + return Objects.hash(jsonSerializer, timestampFormat); + } + + private RowType createJsonRowType(DataType databaseSchema) { + DataType payload = DataTypes.ROW( + DataTypes.FIELD("data", databaseSchema), + DataTypes.FIELD("type", DataTypes.STRING())); + return (RowType) payload.getLogicalType(); + } + +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 781a963f7..942b3dc43 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -17,3 +17,4 @@ org.apache.flink.formats.json.JsonFileSystemFormatFactory org.apache.flink.formats.json.JsonFormatFactory org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory org.apache.flink.formats.json.canal.CanalJsonFormatFactory +org.apache.flink.formats.json.maxwell.MaxwellJsonFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java new file mode 100644 index 000000000..d9dcc06d9 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.maxwell; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.Consumer; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link MaxwellJsonFormatFactory}. + */ +public class MaxwellJsonFormatFactoryTest extends TestLogger { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + @Test + public void testSeDeSchema() { + final MaxwellJsonDeserializationSchema expectedDeser = new MaxwellJsonDeserializationSchema( + ROW_TYPE, + InternalTypeInfo.of(ROW_TYPE), + true, + TimestampFormat.ISO_8601); + + final MaxwellJsonSerializationSchema expectedSer = new MaxwellJsonSerializationSchema( + ROW_TYPE, + TimestampFormat.ISO_8601); + + final Map options = getAllOptions(); + + final DynamicTableSource actualSource = createTableSource(options); + assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = scanSourceMock.valueFormat + .createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toRowDataType()); + + assertEquals(expectedDeser, actualDeser); + + final DynamicTableSink actualSink = createTableSink(options); + assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = sinkMock.valueFormat + .createRuntimeEncoder( + new SinkRuntimeProviderContext(false), + SCHEMA.toRowDataType()); + + assertEquals(expectedSer, actualSer); + } + + @Test + public void testInvalidIgnoreParseError() { + thrown.expect(containsCause(new IllegalArgumentException( + "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); + + final Map options = + getModifiedOptions(opts -> opts.put("maxwell-json.ignore-parse-errors", "abc")); + + createTableSource(options); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifiedOptions(Consumer> optionModifier) { + Map options = getAllOptions(); + optionModifier.accept(options); + return options; + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", "maxwell-json"); + options.put("maxwell-json.ignore-parse-errors", "true"); + options.put("maxwell-json.timestamp-format.standard", "ISO-8601"); + return options; + } + + private static DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + MaxwellJsonFormatFactoryTest.class.getClassLoader()); + } + + private static DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + MaxwellJsonFormatFactoryTest.class.getClassLoader()); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java new file mode 100644 index 000000000..cab2b61bd --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.maxwell; + +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; + +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.FLOAT; +import static org.apache.flink.table.api.DataTypes.INT; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link MaxwellJsonSerializationSchema} and {@link MaxwellJsonDeserializationSchema}. + */ +public class MaxwellJsonSerDerTest { + private static final RowType SCHEMA = (RowType) ROW( + FIELD("id", INT().notNull()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", FLOAT()) + ).getLogicalType(); + + @Test + public void testSerializationDeserialization() throws Exception { + List lines = readLines("maxwell-data.txt"); + MaxwellJsonDeserializationSchema deserializationSchema = new MaxwellJsonDeserializationSchema( + SCHEMA, + InternalTypeInfo.of(SCHEMA), + false, + TimestampFormat.ISO_8601); + + SimpleCollector collector = new SimpleCollector(); + for (String line : lines) { + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + } + + // Maxwell captures change data (`maxwell-data.txt`) on the `product` table: + // + // CREATE TABLE product ( + // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + // name VARCHAR(255), + // description VARCHAR(512), + // weight FLOAT + // ); + // ALTER TABLE product AUTO_INCREMENT = 101; + // + // INSERT INTO product + // VALUES (default,"scooter","Small 2-wheel scooter",3.14), + // (default,"car battery","12V car battery",8.1), + // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), + // (default,"hammer","12oz carpenter's hammer",0.75), + // (default,"hammer","14oz carpenter's hammer",0.875), + // (default,"hammer","16oz carpenter's hammer",1.0), + // (default,"rocks","box of assorted rocks",5.3), + // (default,"jacket","water resistent black wind breaker",0.1), + // (default,"spare tire","24 inch spare tire",22.2); + // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; + // UPDATE product SET weight='5.1' WHERE id=107; + // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); + // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); + // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110; + // UPDATE product SET weight='5.17' WHERE id=111; + // DELETE FROM product WHERE id=111; + // UPDATE product SET weight='5.17' WHERE id=102 or id = 101; + // DELETE FROM product WHERE id=102 or id = 103; + List expected = Arrays.asList( + "+I(101,scooter,Small 2-wheel scooter,3.14)", + "+I(102,car battery,12V car battery,8.1)", + "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", + "+I(104,hammer,12oz carpenter's hammer,0.75)", + "+I(105,hammer,14oz carpenter's hammer,0.875)", + "+I(106,hammer,16oz carpenter's hammer,1.0)", + "+I(107,rocks,box of assorted rocks,5.3)", + "+I(108,jacket,water resistent black wind breaker,0.1)", + "+I(109,spare tire,24 inch spare tire,22.2)", + "-U(106,hammer,16oz carpenter's hammer,1.0)", + "+U(106,hammer,18oz carpenter hammer,1.0)", + "-U(107,rocks,box of assorted rocks,5.3)", + "+U(107,rocks,box of assorted rocks,5.1)", + "+I(110,jacket,water resistent white wind breaker,0.2)", + "+I(111,scooter,Big 2-wheel scooter ,5.18)", + "-U(110,jacket,water resistent white wind breaker,0.2)", + "+U(110,jacket,new water resistent white wind breaker,0.5)", + "-U(111,scooter,Big 2-wheel scooter ,5.18)", + "+U(111,scooter,Big 2-wheel scooter ,5.17)", + "-D(111,scooter,Big 2-wheel scooter ,5.17)", + "-U(101,scooter,Small 2-wheel scooter,3.14)", + "+U(101,scooter,Small 2-wheel scooter,5.17)", + "-U(102,car battery,12V car battery,8.1)", + "+U(102,car battery,12V car battery,5.17)", + "-D(102,car battery,12V car battery,5.17)", + "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)" + ); + List actual = collector.list.stream() + .map(Object::toString) + .collect(Collectors.toList()); + assertEquals(expected, actual); + + MaxwellJsonSerializationSchema serializationSchema = new MaxwellJsonSerializationSchema( + SCHEMA, + TimestampFormat.SQL); + serializationSchema.open(null); + List result = new ArrayList<>(); + for (RowData rowData : collector.list) { + result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + List expectedResult = Arrays.asList( + "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"insert\"}", + "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"insert\"}", + "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"insert\"}", + "{\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"type\":\"insert\"}", + "{\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"type\":\"insert\"}", + "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"insert\"}", + "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"insert\"}", + "{\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"type\":\"insert\"}", + "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"type\":\"insert\"}", + "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"delete\"}", + "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"type\":\"insert\"}", + "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"delete\"}", + "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"type\":\"insert\"}", + "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"insert\"}", + "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"insert\"}", + "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"delete\"}", + "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"type\":\"insert\"}", + "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"delete\"}", + "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"insert\"}", + "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"delete\"}", + "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"delete\"}", + "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17},\"type\":\"insert\"}", + "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"delete\"}", + "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"insert\"}", + "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"delete\"}", + "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"delete\"}" + ); + assertEquals(expectedResult, result); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private static List readLines(String resource) throws IOException { + final URL url = MaxwellJsonSerDerTest.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllLines(path); + } + + private static class SimpleCollector implements Collector { + + private List list = new ArrayList<>(); + + @Override + public void collect(RowData record) { + list.add(record); + } + + @Override + public void close() { + // do nothing + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt new file mode 100644 index 000000000..ecba573aa --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt @@ -0,0 +1,20 @@ +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":0,"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14}} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":1,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1}} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":2,"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8}} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":3,"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75}} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":4,"data":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875}} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":5,"data":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0}} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":6,"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3}} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":7,"data":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1}} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"commit":true,"data":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2}} +{"database":"test","table":"product","type":"update","ts":1596684893,"xid":7152,"commit":true,"data":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"old":{"description":"16oz carpenter's hammer"}} +{"database":"test","table":"product","type":"update","ts":1596684897,"xid":7169,"commit":true,"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"old":{"weight":5.3}} +{"database":"test","table":"product","type":"insert","ts":1596684900,"xid":7186,"commit":true,"data":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2}} +{"database":"test","table":"product","type":"insert","ts":1596684904,"xid":7201,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18}} +{"database":"test","table":"product","type":"update","ts":1596684906,"xid":7216,"commit":true,"data":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"old":{"description":"water resistent white wind breaker","weight":0.2}} +{"database":"test","table":"product","type":"update","ts":1596684912,"xid":7235,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"old":{"weight":5.18}} +{"database":"test","table":"product","type":"delete","ts":1596684914,"xid":7250,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17}} +{"database":"test","table":"product","type":"update","ts":1596684928,"xid":7291,"xoffset":0,"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":5.17},"old":{"weight":3.14}} +{"database":"test","table":"product","type":"update","ts":1596684928,"xid":7291,"commit":true,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":5.17},"old":{"weight":8.1}} +{"database":"test","table":"product","type":"delete","ts":1596684938,"xid":7322,"xoffset":0,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":5.17}} +{"database":"test","table":"product","type":"delete","ts":1596684938,"xid":7322,"commit":true,"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8}} From bad1aebab3f47f8ae789ed80cbaace1b966cb81f Mon Sep 17 00:00:00 2001 From: Brandon Date: Mon, 17 Aug 2020 16:56:12 +0800 Subject: [PATCH 087/322] [hoxfix] Fix various typos --- .../java/org/apache/flink/formats/json/TimestampFormat.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java index e9db8de2a..f06bfada5 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java @@ -29,7 +29,7 @@ public enum TimestampFormat { * TIMESTAMP_WITH_LOCAL_TIMEZONE in "yyyy-MM-dd HH:mm:ss.s{precision}'Z'" and output in the same format.*/ SQL, - /** Options to specify TIMESTAMP/TIMESTAMP_WITH_LOCAL_ZONE format. It will pase TIMESTAMP in "yyyy-MM-ddTHH:mm:ss.s{precision}" format, + /** Options to specify TIMESTAMP/TIMESTAMP_WITH_LOCAL_ZONE format. It will parse TIMESTAMP in "yyyy-MM-ddTHH:mm:ss.s{precision}" format, * TIMESTAMP_WITH_LOCAL_TIMEZONE in "yyyy-MM-ddTHH:mm:ss.s{precision}'Z'" and output in the same format.*/ ISO_8601 } From 3776d1efe8852d0ca60d581aac70db39210ef98c Mon Sep 17 00:00:00 2001 From: caozhen Date: Tue, 1 Sep 2020 13:28:57 +0800 Subject: [PATCH 088/322] [FLINK-18824][json][table] Support serialization for canal-json format This closes #13122 --- .../json/canal/CanalJsonFormatFactory.java | 26 +++- .../canal/CanalJsonSerializationSchema.java | 120 ++++++++++++++++++ .../canal/CanalJsonFormatFactoryTest.java | 61 +++++---- ...est.java => CanalJsonSerDeSchemaTest.java} | 56 +++++++- 4 files changed, 234 insertions(+), 29 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java rename flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/{CanalJsonDeserializationSchemaTest.java => CanalJsonSerDeSchemaTest.java} (57%) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java index cedff8081..0112b8a67 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java @@ -28,6 +28,7 @@ import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DeserializationFormatFactory; @@ -92,7 +93,30 @@ public ChangelogMode getChangelogMode() { public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { - throw new UnsupportedOperationException("Canal format doesn't support as a sink format yet."); + + FactoryUtil.validateFactoryOptions(this, formatOptions); + TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + + return new EncodingFormat>() { + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + @Override + public SerializationSchema createRuntimeEncoder(DynamicTableSink.Context context, DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new CanalJsonSerializationSchema( + rowType, + timestampFormat); + } + }; + } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java new file mode 100644 index 000000000..ea5d4e141 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.canal; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.json.JsonRowDataSerializationSchema; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Objects; + +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Serialization schema that serializes an object of Flink Table/SQL internal data structure {@link RowData} into a Canal JSON bytes. + * + * @see Alibaba Canal + */ +public class CanalJsonSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = 1L; + + private static final StringData OP_INSERT = StringData.fromString("INSERT"); + private static final StringData OP_DELETE = StringData.fromString("DELETE"); + + private transient GenericRowData reuse; + + /** + * The serializer to serialize Canal JSON data. + */ + private final JsonRowDataSerializationSchema jsonSerializer; + + public CanalJsonSerializationSchema(RowType rowType, TimestampFormat timestampFormat) { + jsonSerializer = new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat); + } + + @Override + public void open(InitializationContext context) { + reuse = new GenericRowData(2); + } + + @Override + public byte[] serialize(RowData row) { + try { + StringData opType = rowKind2String(row.getRowKind()); + ArrayData arrayData = new GenericArrayData(new RowData[]{row}); + reuse.setField(0, arrayData); + reuse.setField(1, opType); + return jsonSerializer.serialize(reuse); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'.", t); + } + + } + + private StringData rowKind2String(RowKind rowKind) { + switch (rowKind) { + case INSERT: + case UPDATE_AFTER: + return OP_INSERT; + case UPDATE_BEFORE: + case DELETE: + return OP_DELETE; + default: + throw new UnsupportedOperationException("Unsupported operation '" + rowKind + "' for row kind."); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CanalJsonSerializationSchema that = (CanalJsonSerializationSchema) o; + return Objects.equals(jsonSerializer, that.jsonSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(jsonSerializer); + } + + private static RowType createJsonRowType(DataType databaseSchema) { + // Canal JSON contains other information, e.g. "database", "ts" + // but we don't need them + // and we don't need "old" , because can not support UPDATE_BEFORE,UPDATE_AFTER + return (RowType) DataTypes.ROW( + DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)), + DataTypes.FIELD("type", DataTypes.STRING())).getLogicalType(); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 2d4761600..2603f5cbb 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.json.canal; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.Configuration; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; @@ -30,6 +31,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; @@ -54,15 +56,17 @@ public class CanalJsonFormatFactoryTest extends TestLogger { public ExpectedException thrown = ExpectedException.none(); private static final TableSchema SCHEMA = TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); @Test public void testSeDeSchema() { + + // test Deser final CanalJsonDeserializationSchema expectedDeser = new CanalJsonDeserializationSchema( ROW_TYPE, InternalTypeInfo.of(ROW_TYPE), @@ -74,18 +78,31 @@ public void testSeDeSchema() { final DynamicTableSource actualSource = createTableSource(options); assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; DeserializationSchema actualDeser = scanSourceMock.valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, - SCHEMA.toRowDataType()); + .createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toRowDataType()); assertEquals(expectedDeser, actualDeser); - thrown.expect(containsCause(new UnsupportedOperationException( - "Canal format doesn't support as a sink format yet."))); - createTableSink(options); + // test Ser + CanalJsonSerializationSchema expectedSer = new CanalJsonSerializationSchema( + ROW_TYPE, + TimestampFormat.ISO_8601); + final DynamicTableSink actualSink = createTableSink(options); + + assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = sinkMock.valueFormat + .createRuntimeEncoder( + new SinkRuntimeProviderContext(false), + SCHEMA.toRowDataType()); + + assertEquals(expectedSer, actualSer); } @Test @@ -94,7 +111,7 @@ public void testInvalidIgnoreParseError() { "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); final Map options = - getModifiedOptions(opts -> opts.put("canal-json.ignore-parse-errors", "abc")); + getModifiedOptions(opts -> opts.put("canal-json.ignore-parse-errors", "abc")); createTableSource(options); } @@ -128,19 +145,19 @@ private Map getAllOptions() { private static DynamicTableSource createTableSource(Map options) { return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - CanalJsonFormatFactoryTest.class.getClassLoader()); + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + CanalJsonFormatFactoryTest.class.getClassLoader()); } private static DynamicTableSink createTableSink(Map options) { return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - CanalJsonFormatFactoryTest.class.getClassLoader()); + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + CanalJsonFormatFactoryTest.class.getClassLoader()); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java similarity index 57% rename from flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java rename to flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index b7135097c..037ffb155 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -37,6 +37,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; import static org.apache.flink.table.api.DataTypes.FIELD; import static org.apache.flink.table.api.DataTypes.FLOAT; @@ -48,7 +49,7 @@ /** * Tests for {@link CanalJsonDeserializationSchema}. */ -public class CanalJsonDeserializationSchemaTest { +public class CanalJsonSerDeSchemaTest { @Rule public ExpectedException thrown = ExpectedException.none(); @@ -61,7 +62,7 @@ public class CanalJsonDeserializationSchemaTest { ).getLogicalType(); @Test - public void testDeserialization() throws Exception { + public void testSerializationDeserialization() throws Exception { List lines = readLines("canal-data.txt"); CanalJsonDeserializationSchema deserializationSchema = new CanalJsonDeserializationSchema( SCHEMA, @@ -131,7 +132,50 @@ public void testDeserialization() throws Exception { "-D(102,car battery,12V car battery,5.17)", "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)" ); - assertEquals(expected, collector.list); + List actual = collector.list.stream() + .map(Object::toString) + .collect(Collectors.toList()); + assertEquals(expected, actual); + + // test Serialization + CanalJsonSerializationSchema serializationSchema = new CanalJsonSerializationSchema( + SCHEMA, + TimestampFormat.ISO_8601); + serializationSchema.open(null); + + List result = new ArrayList<>(); + for (RowData rowData : collector.list) { + result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + + List expectedResult = Arrays.asList("{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"DELETE\"}"); + + assertEquals(expectedResult, result); } // -------------------------------------------------------------------------------------------- @@ -139,7 +183,7 @@ public void testDeserialization() throws Exception { // -------------------------------------------------------------------------------------------- private static List readLines(String resource) throws IOException { - final URL url = CanalJsonDeserializationSchemaTest.class.getClassLoader().getResource(resource); + final URL url = CanalJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); assert url != null; Path path = new File(url.getFile()).toPath(); return Files.readAllLines(path); @@ -147,11 +191,11 @@ private static List readLines(String resource) throws IOException { private static class SimpleCollector implements Collector { - private List list = new ArrayList<>(); + private List list = new ArrayList<>(); @Override public void collect(RowData record) { - list.add(record.toString()); + list.add(record); } @Override From f30075d3549457ec6f84ab79a1ff15d1f9266ba3 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Thu, 10 Sep 2020 16:22:59 +0800 Subject: [PATCH 089/322] [FLINK-18823][format] Support serialization for debezium-json format This closes #13333 --- .../DebeziumJsonDeserializationSchema.java | 2 +- .../debezium/DebeziumJsonFormatFactory.java | 23 +++- .../DebeziumJsonSerializationSchema.java | 112 ++++++++++++++++++ .../json/canal/CanalJsonSerDeSchemaTest.java | 2 +- .../DebeziumJsonFormatFactoryTest.java | 20 +++- ....java => DebeziumJsonSerDeSchemaTest.java} | 69 ++++++++--- 6 files changed, 206 insertions(+), 22 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java rename flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/{DebeziumJsonDeserializationSchemaTest.java => DebeziumJsonSerDeSchemaTest.java} (58%) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java index deb3653e6..7fc734ad2 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java @@ -62,7 +62,7 @@ public final class DebeziumJsonDeserializationSchema implements DeserializationS "if you are using Debezium Postgres Connector, " + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; - /** The deserializer to deserialize Debezium JSON data. */ + /** The deserializer to deserialize Debezium JSON data. **/ private final JsonRowDataDeserializationSchema jsonDeserializer; /** TypeInformation of the produced {@link RowData}. **/ diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java index 17c6ab2e8..d81f7a1bf 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -29,6 +29,7 @@ import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DeserializationFormatFactory; @@ -104,7 +105,27 @@ public ChangelogMode getChangelogMode() { public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { - throw new UnsupportedOperationException("Debezium format doesn't support as a sink format yet."); + FactoryUtil.validateFactoryOptions(this, formatOptions); + TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + + return new EncodingFormat>() { + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + @Override + public SerializationSchema createRuntimeEncoder(DynamicTableSink.Context context, DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new DebeziumJsonSerializationSchema(rowType, timestampFormat); + } + }; } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java new file mode 100644 index 000000000..f94c81371 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.debezium; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.json.JsonRowDataSerializationSchema; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +import java.util.Objects; + +import static java.lang.String.format; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium JSON. + * + * @see Debezium + */ +public class DebeziumJsonSerializationSchema implements SerializationSchema { + private static final long serialVersionUID = 1L; + + private static final StringData OP_INSERT = StringData.fromString("c"); // insert + private static final StringData OP_DELETE = StringData.fromString("d"); // delete + + /** The serializer to serialize Debezium JSON data. **/ + private final JsonRowDataSerializationSchema jsonSerializer; + + private transient GenericRowData genericRowData; + + public DebeziumJsonSerializationSchema(RowType rowType, TimestampFormat timestampFormat) { + jsonSerializer = new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat); + } + + @Override + public void open(InitializationContext context) { + genericRowData = new GenericRowData(3); + } + + @Override + public byte[] serialize(RowData rowData) { + try { + switch (rowData.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + genericRowData.setField(0, null); + genericRowData.setField(1, rowData); + genericRowData.setField(2, OP_INSERT); + return jsonSerializer.serialize(genericRowData); + case UPDATE_BEFORE: + case DELETE: + genericRowData.setField(0, rowData); + genericRowData.setField(1, null); + genericRowData.setField(2, OP_DELETE); + return jsonSerializer.serialize(genericRowData); + default: + throw new UnsupportedOperationException(format("Unsupported operation '%s' for row kind.", rowData.getRowKind())); + } + } catch (Throwable t) { + throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DebeziumJsonSerializationSchema that = (DebeziumJsonSerializationSchema) o; + return Objects.equals(jsonSerializer, that.jsonSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(jsonSerializer); + } + + private static RowType createJsonRowType(DataType databaseSchema) { + // Debezium JSON contains some other information, e.g. "source", "ts_ms" + // but we don't need them. + return (RowType) DataTypes.ROW( + DataTypes.FIELD("before", databaseSchema), + DataTypes.FIELD("after", databaseSchema), + DataTypes.FIELD("op", DataTypes.STRING())).getLogicalType(); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index 037ffb155..af74cc8ea 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -47,7 +47,7 @@ import static org.junit.Assert.assertEquals; /** - * Tests for {@link CanalJsonDeserializationSchema}. + * Tests for {@link CanalJsonSerializationSchema} and {@link CanalJsonDeserializationSchema}. */ public class CanalJsonSerDeSchemaTest { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 71cb13d67..40fcac5dd 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.json.debezium; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.Configuration; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; @@ -30,6 +31,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; @@ -84,9 +86,21 @@ public void testSeDeSchema() { assertEquals(expectedDeser, actualDeser); - thrown.expect(containsCause(new UnsupportedOperationException( - "Debezium format doesn't support as a sink format yet."))); - createTableSink(options); + final DebeziumJsonSerializationSchema expectedSer = new DebeziumJsonSerializationSchema( + ROW_TYPE, + TimestampFormat.ISO_8601); + + final DynamicTableSink actualSink = createTableSink(options); + assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = sinkMock.valueFormat + .createRuntimeEncoder( + new SinkRuntimeProviderContext(false), + SCHEMA.toRowDataType()); + + assertEquals(expectedSer, actualSer); } @Test diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java similarity index 58% rename from flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java rename to flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java index edaddbfcc..0c4b1a620 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -38,6 +38,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; import static org.apache.flink.table.api.DataTypes.FIELD; import static org.apache.flink.table.api.DataTypes.FLOAT; @@ -48,9 +49,9 @@ import static org.junit.Assert.assertTrue; /** - * Tests for {@link DebeziumJsonDeserializationSchema}. + * Tests for {@link DebeziumJsonSerializationSchema} and {@link DebeziumJsonDeserializationSchema}. */ -public class DebeziumJsonDeserializationSchemaTest { +public class DebeziumJsonSerDeSchemaTest { @Rule public ExpectedException thrown = ExpectedException.none(); @@ -63,29 +64,29 @@ public class DebeziumJsonDeserializationSchemaTest { ).getLogicalType(); @Test - public void testSchemaIncludeDeserialization() throws Exception { - testDeserialization("debezium-data-schema-include.txt", true); + public void testSerializationAndSchemaIncludeDeserialization() throws Exception { + testSerializationDeserialization("debezium-data-schema-include.txt", true); } @Test - public void testSchemaExcludeDeserialization() throws Exception { - testDeserialization("debezium-data-schema-exclude.txt", false); + public void testSerializationAndSchemaExcludeDeserialization() throws Exception { + testSerializationDeserialization("debezium-data-schema-exclude.txt", false); } @Test - public void testPostgresSchemaIncludeDeserialization() throws Exception { - testDeserialization("debezium-postgres-data-schema-include.txt", true); + public void testSerializationAndPostgresSchemaIncludeDeserialization() throws Exception { + testSerializationDeserialization("debezium-postgres-data-schema-include.txt", true); } @Test - public void testPostgresSchemaExcludeDeserialization() throws Exception { - testDeserialization("debezium-postgres-data-schema-exclude.txt", false); + public void testSerializationAndPostgresSchemaExcludeDeserialization() throws Exception { + testSerializationDeserialization("debezium-postgres-data-schema-exclude.txt", false); } @Test public void testPostgresDefaultReplicaIdentify() throws Exception { try { - testDeserialization("debezium-postgres-data-replica-identity.txt", false); + testSerializationDeserialization("debezium-postgres-data-replica-identity.txt", false); } catch (Exception e) { assertTrue(ExceptionUtils.findThrowableWithMessage(e, "The \"before\" field of UPDATE message is null, if you are using Debezium Postgres Connector, " + @@ -107,7 +108,7 @@ public void testTombstoneMessages() throws Exception { assertTrue(collector.list.isEmpty()); } - private void testDeserialization(String resourceFile, boolean schemaInclude) throws Exception { + private void testSerializationDeserialization(String resourceFile, boolean schemaInclude) throws Exception { List lines = readLines(resourceFile); DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( SCHEMA, @@ -170,7 +171,43 @@ private void testDeserialization(String resourceFile, boolean schemaInclude) thr "+U(111,scooter,Big 2-wheel scooter ,5.17)", "-D(111,scooter,Big 2-wheel scooter ,5.17)" ); - assertEquals(expected, collector.list); + List actual = collector.list.stream() + .map(Object::toString) + .collect(Collectors.toList()); + assertEquals(expected, actual); + + DebeziumJsonSerializationSchema serializationSchema = new DebeziumJsonSerializationSchema( + SCHEMA, + TimestampFormat.SQL); + serializationSchema.open(null); + actual = new ArrayList<>(); + for (RowData rowData : collector.list) { + actual.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + + expected = Arrays.asList( + "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op\":\"c\"}", + "{\"before\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op\":\"c\"}", + "{\"before\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op\":\"c\"}", + "{\"before\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op\":\"c\"}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op\":\"c\"}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op\":\"d\"}" + ); + assertEquals(expected, actual); } // -------------------------------------------------------------------------------------------- @@ -178,7 +215,7 @@ private void testDeserialization(String resourceFile, boolean schemaInclude) thr // -------------------------------------------------------------------------------------------- private static List readLines(String resource) throws IOException { - final URL url = DebeziumJsonDeserializationSchemaTest.class.getClassLoader().getResource(resource); + final URL url = DebeziumJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); assert url != null; Path path = new File(url.getFile()).toPath(); return Files.readAllLines(path); @@ -186,11 +223,11 @@ private static List readLines(String resource) throws IOException { private static class SimpleCollector implements Collector { - private List list = new ArrayList<>(); + private List list = new ArrayList<>(); @Override public void collect(RowData record) { - list.add(record.toString()); + list.add(record); } @Override From 55f493eb6dc865ab1af22476d420703eebe0c0aa Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 10 Sep 2020 10:28:25 +0200 Subject: [PATCH 090/322] [FLINK-18192] Upgrade avro to 1.10 This commit upgrades the default version of avro that flink-avro will use. It should be possible to downgrade the avro version in a user job as the binary format is compatible and we do not expose any dependencies on avro in the API. Additionally this commit fixes handling of logical types: time-micros and timestamp-micros as well as interpretation of timestamp-millis in the AvroRowDataDeserializationSchema. --- .../registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java index 1cf348a9f..dbcec71cd 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java @@ -148,7 +148,7 @@ private void testRowDataWriteReadWithSchema(Schema schema) throws Exception { byte[] serialized = serializer.serialize(oriData); RowData rowData = deserializer.deserialize(serialized); assertThat(rowData.getArity(), equalTo(schema.getFields().size())); - assertEquals(address.getNum(), Integer.valueOf(rowData.getInt(0))); + assertEquals(address.getNum(), rowData.getInt(0)); assertEquals(address.getStreet(), rowData.getString(1).toString()); if (schema != ADDRESS_SCHEMA_COMPATIBLE) { assertEquals(address.getCity(), rowData.getString(2).toString()); From 7d74f659504441c22bfbd19f629cf73469901ffe Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 7 Sep 2020 11:33:16 +0200 Subject: [PATCH 091/322] [FLINK-19152] Remove Kafka 0.10.x and 0.11.x connectors --- .../pom.xml | 4 ++-- .../registry/test/TestAvroConsumerConfluent.java | 16 ++++++++-------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 3ffec7851..4110f4538 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -48,10 +48,10 @@ under the License. ${project.version} provided - + org.apache.flink - flink-connector-kafka-0.10_${scala.binary.version} + flink-connector-kafka_${scala.binary.version} ${project.version} diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java index dda461797..459a16c9a 100644 --- a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java +++ b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java @@ -25,8 +25,8 @@ import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; import example.avro.User; import org.apache.avro.specific.SpecificRecordBase; @@ -61,7 +61,7 @@ public static void main(String[] args) throws Exception { DataStreamSource input = env .addSource( - new FlinkKafkaConsumer010<>( + new FlinkKafkaConsumer<>( parameterTool.getRequired("input-topic"), ConfluentRegistryAvroDeserializationSchema.forSpecific(User.class, schemaRegistryUrl), config).setStartFromEarliest()); @@ -69,18 +69,18 @@ public static void main(String[] args) throws Exception { SingleOutputStreamOperator mapToString = input .map((MapFunction) SpecificRecordBase::toString); - FlinkKafkaProducer010 stringFlinkKafkaProducer010 = new FlinkKafkaProducer010<>( + FlinkKafkaProducer stringFlinkKafkaProducer = new FlinkKafkaProducer<>( parameterTool.getRequired("output-string-topic"), new SimpleStringSchema(), config); - mapToString.addSink(stringFlinkKafkaProducer010); + mapToString.addSink(stringFlinkKafkaProducer); - FlinkKafkaProducer010 avroFlinkKafkaProducer010 = new FlinkKafkaProducer010<>( + FlinkKafkaProducer avroFlinkKafkaProducer = new FlinkKafkaProducer<>( parameterTool.getRequired("output-avro-topic"), ConfluentRegistryAvroSerializationSchema.forSpecific(User.class, parameterTool.getRequired("output-subject"), schemaRegistryUrl), config); - input.addSink(avroFlinkKafkaProducer010); + input.addSink(avroFlinkKafkaProducer); - env.execute("Kafka 0.10 Confluent Schema Registry AVRO Example"); + env.execute("Kafka Confluent Schema Registry AVRO Example"); } } From fca2c3f5d85e8959dcb9be2bff36052c9b30e855 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Fri, 18 Sep 2020 15:52:20 +0800 Subject: [PATCH 092/322] [FLINK-19002][canal][json] Support to only read changelogs of specific database and table for canal-json format This closes (#13294) --- .../canal/CanalJsonDeserializationSchema.java | 97 ++++++++++++++++++- .../json/canal/CanalJsonFormatFactory.java | 29 +++++- .../canal/CanalJsonFormatFactoryTest.java | 91 ++++++++++------- .../json/canal/CanalJsonSerDeSchemaTest.java | 24 ++++- .../resources/canal-data-filter-table.txt | 14 +++ 5 files changed, 206 insertions(+), 49 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index 761a0f4db..20292fc25 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json.canal; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; @@ -31,6 +32,8 @@ import org.apache.flink.types.RowKind; import org.apache.flink.util.Collector; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.Objects; @@ -62,18 +65,28 @@ public final class CanalJsonDeserializationSchema implements DeserializationSche /** TypeInformation of the produced {@link RowData}. **/ private final TypeInformation resultTypeInfo; + /** Only read changelogs from the specific database. */ + private final @Nullable String database; + + /** Only read changelogs from the specific table. */ + private final @Nullable String table; + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ private final boolean ignoreParseErrors; /** Number of fields. */ private final int fieldCount; - public CanalJsonDeserializationSchema( + private CanalJsonDeserializationSchema( RowType rowType, TypeInformation resultTypeInfo, + @Nullable String database, + @Nullable String table, boolean ignoreParseErrors, TimestampFormat timestampFormatOption) { this.resultTypeInfo = resultTypeInfo; + this.database = database; + this.table = table; this.ignoreParseErrors = ignoreParseErrors; this.fieldCount = rowType.getFieldCount(); this.jsonDeserializer = new JsonRowDataDeserializationSchema( @@ -86,6 +99,69 @@ public CanalJsonDeserializationSchema( } + // ------------------------------------------------------------------------------------------ + // Builder + // ------------------------------------------------------------------------------------------ + + /** + * Creates A builder for building a {@link CanalJsonDeserializationSchema}. + */ + public static Builder builder(RowType rowType, TypeInformation resultTypeInfo) { + return new Builder(rowType, resultTypeInfo); + } + + /** + * A builder for creating a {@link CanalJsonDeserializationSchema}. + */ + @Internal + public static final class Builder { + private final RowType rowType; + private final TypeInformation resultTypeInfo; + private String database = null; + private String table = null; + private boolean ignoreParseErrors = false; + private TimestampFormat timestampFormat = TimestampFormat.SQL; + + private Builder(RowType rowType, TypeInformation resultTypeInfo) { + this.rowType = rowType; + this.resultTypeInfo = resultTypeInfo; + } + + public Builder setDatabase(String database) { + this.database = database; + return this; + } + + public Builder setTable(String table) { + this.table = table; + return this; + } + + public Builder setIgnoreParseErrors(boolean ignoreParseErrors) { + this.ignoreParseErrors = ignoreParseErrors; + return this; + } + + public Builder setTimestampFormat(TimestampFormat timestampFormat) { + this.timestampFormat = timestampFormat; + return this; + } + + public CanalJsonDeserializationSchema build() { + return new CanalJsonDeserializationSchema( + rowType, + resultTypeInfo, + database, + table, + ignoreParseErrors, + timestampFormat + ); + } + } + + + // ------------------------------------------------------------------------------------------ + @Override public RowData deserialize(byte[] message) throws IOException { throw new RuntimeException( @@ -96,6 +172,18 @@ public RowData deserialize(byte[] message) throws IOException { public void deserialize(byte[] message, Collector out) throws IOException { try { RowData row = jsonDeserializer.deserialize(message); + if (database != null) { + String currentDatabase = row.getString(3).toString(); + if (!database.equals(currentDatabase)) { + return; + } + } + if (table != null) { + String currentTable = row.getString(4).toString(); + if (!table.equals(currentTable)) { + return; + } + } String type = row.getString(2).toString(); // "type" field if (OP_INSERT.equals(type)) { // "data" field is an array of row, contains inserted rows @@ -181,11 +269,12 @@ public int hashCode() { } private static RowType createJsonRowType(DataType databaseSchema) { - // Canal JSON contains other information, e.g. "database", "ts" - // but we don't need them + // Canal JSON contains other information, e.g. "ts", "sql", but we don't need them return (RowType) DataTypes.ROW( DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)), DataTypes.FIELD("old", DataTypes.ARRAY(databaseSchema)), - DataTypes.FIELD("type", DataTypes.STRING())).getLogicalType(); + DataTypes.FIELD("type", DataTypes.STRING()), + DataTypes.FIELD("database", DataTypes.STRING()), + DataTypes.FIELD("table", DataTypes.STRING())).getLogicalType(); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java index 0112b8a67..c5d649daa 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; @@ -54,6 +55,18 @@ public class CanalJsonFormatFactory implements DeserializationFormatFactory, Ser public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + public static final ConfigOption DATABASE_INCLUDE = ConfigOptions + .key("database.include") + .stringType() + .noDefaultValue() + .withDescription("Only read changelog rows which match the specific database (by comparing the \"database\" meta field in the record)."); + + public static final ConfigOption TABLE_INCLUDE = ConfigOptions + .key("table.include") + .stringType() + .noDefaultValue() + .withDescription("Only read changelog rows which match the specific table (by comparing the \"table\" meta field in the record)."); + @SuppressWarnings("unchecked") @Override public DecodingFormat> createDecodingFormat( @@ -62,6 +75,8 @@ public DecodingFormat> createDecodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); + String database = formatOptions.getOptional(DATABASE_INCLUDE).orElse(null); + String table = formatOptions.getOptional(TABLE_INCLUDE).orElse(null); return new DecodingFormat>() { @Override @@ -70,11 +85,13 @@ public DeserializationSchema createRuntimeDecoder( final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = (TypeInformation) context.createTypeInformation(producedDataType); - return new CanalJsonDeserializationSchema( - rowType, - rowDataTypeInfo, - ignoreParseErrors, - timestampFormatOption); + return CanalJsonDeserializationSchema + .builder(rowType, rowDataTypeInfo) + .setIgnoreParseErrors(ignoreParseErrors) + .setTimestampFormat(timestampFormatOption) + .setDatabase(database) + .setTable(table) + .build(); } @Override @@ -134,6 +151,8 @@ public Set> optionalOptions() { Set> options = new HashSet<>(); options.add(IGNORE_PARSE_ERRORS); options.add(TIMESTAMP_FORMAT); + options.add(DATABASE_INCLUDE); + options.add(TABLE_INCLUDE); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 2603f5cbb..4767c99f2 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -63,45 +63,53 @@ public class CanalJsonFormatFactoryTest extends TestLogger { private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + private static final InternalTypeInfo ROW_TYPE_INFO = InternalTypeInfo.of(ROW_TYPE); + @Test - public void testSeDeSchema() { + public void testDefaultOptions() { + Map options = getAllOptions(); // test Deser - final CanalJsonDeserializationSchema expectedDeser = new CanalJsonDeserializationSchema( - ROW_TYPE, - InternalTypeInfo.of(ROW_TYPE), - true, - TimestampFormat.ISO_8601); - - final Map options = getAllOptions(); + CanalJsonDeserializationSchema expectedDeser = CanalJsonDeserializationSchema + .builder(ROW_TYPE, ROW_TYPE_INFO) + .setIgnoreParseErrors(false) + .setTimestampFormat(TimestampFormat.SQL) + .build(); + DeserializationSchema actualDeser = createDeserializationSchema(options); + assertEquals(expectedDeser, actualDeser); - final DynamicTableSource actualSource = createTableSource(options); - assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + // test Ser + CanalJsonSerializationSchema expectedSer = new CanalJsonSerializationSchema( + ROW_TYPE, + TimestampFormat.SQL); + SerializationSchema actualSer = createSerializationSchema(options); + assertEquals(expectedSer, actualSer); + } - DeserializationSchema actualDeser = scanSourceMock.valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, - SCHEMA.toRowDataType()); + @Test + public void testUserDefinedOptions() { + Map options = getAllOptions(); + options.put("canal-json.ignore-parse-errors", "true"); + options.put("canal-json.timestamp-format.standard", "ISO-8601"); + options.put("canal-json.database.include", "mydb"); + options.put("canal-json.table.include", "mytable"); + // test Deser + CanalJsonDeserializationSchema expectedDeser = CanalJsonDeserializationSchema + .builder(ROW_TYPE, ROW_TYPE_INFO) + .setIgnoreParseErrors(true) + .setTimestampFormat(TimestampFormat.ISO_8601) + .setDatabase("mydb") + .setTable("mytable") + .build(); + DeserializationSchema actualDeser = createDeserializationSchema(options); assertEquals(expectedDeser, actualDeser); // test Ser CanalJsonSerializationSchema expectedSer = new CanalJsonSerializationSchema( ROW_TYPE, TimestampFormat.ISO_8601); - final DynamicTableSink actualSink = createTableSink(options); - - assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = sinkMock.valueFormat - .createRuntimeEncoder( - new SinkRuntimeProviderContext(false), - SCHEMA.toRowDataType()); - + SerializationSchema actualSer = createSerializationSchema(options); assertEquals(expectedSer, actualSer); } @@ -113,7 +121,7 @@ public void testInvalidIgnoreParseError() { final Map options = getModifiedOptions(opts -> opts.put("canal-json.ignore-parse-errors", "abc")); - createTableSource(options); + createDeserializationSchema(options); } // ------------------------------------------------------------------------ @@ -136,28 +144,41 @@ private Map getAllOptions() { options.put("connector", TestDynamicTableFactory.IDENTIFIER); options.put("target", "MyTarget"); options.put("buffer-size", "1000"); - options.put("format", "canal-json"); - options.put("canal-json.ignore-parse-errors", "true"); - options.put("canal-json.timestamp-format.standard", "ISO-8601"); return options; } - private static DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( + private static DeserializationSchema createDeserializationSchema(Map options) { + DynamicTableSource source = FactoryUtil.createTableSource( null, ObjectIdentifier.of("default", "default", "t1"), new CatalogTableImpl(SCHEMA, options, "mock source"), new Configuration(), CanalJsonFormatFactoryTest.class.getClassLoader()); + + assert source instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) source; + + return scanSourceMock.valueFormat + .createRuntimeDecoder(ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); } - private static DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( + private static SerializationSchema createSerializationSchema(Map options) { + DynamicTableSink sink = FactoryUtil.createTableSink( null, ObjectIdentifier.of("default", "default", "t1"), new CatalogTableImpl(SCHEMA, options, "mock sink"), new Configuration(), CanalJsonFormatFactoryTest.class.getClassLoader()); + + assert sink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) sink; + + return sinkMock.valueFormat + .createRuntimeEncoder( + new SinkRuntimeProviderContext(false), + SCHEMA.toRowDataType()); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index af74cc8ea..e8f3401af 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -61,15 +61,29 @@ public class CanalJsonSerDeSchemaTest { FIELD("weight", FLOAT()) ).getLogicalType(); + @Test + public void testFilteringTables() throws Exception { + List lines = readLines("canal-data-filter-table.txt"); + CanalJsonDeserializationSchema deserializationSchema = CanalJsonDeserializationSchema + .builder(SCHEMA, InternalTypeInfo.of(SCHEMA)) + .setDatabase("mydb") + .setTable("product") + .build(); + runTest(lines, deserializationSchema); + } + @Test public void testSerializationDeserialization() throws Exception { List lines = readLines("canal-data.txt"); - CanalJsonDeserializationSchema deserializationSchema = new CanalJsonDeserializationSchema( - SCHEMA, - InternalTypeInfo.of(SCHEMA), - false, - TimestampFormat.ISO_8601); + CanalJsonDeserializationSchema deserializationSchema = CanalJsonDeserializationSchema + .builder(SCHEMA, InternalTypeInfo.of(SCHEMA)) + .setIgnoreParseErrors(false) + .setTimestampFormat(TimestampFormat.ISO_8601) + .build(); + runTest(lines, deserializationSchema); + } + public void runTest(List lines, CanalJsonDeserializationSchema deserializationSchema) throws Exception { SimpleCollector collector = new SimpleCollector(); for (String line : lines) { deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt new file mode 100644 index 000000000..7db1b224d --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt @@ -0,0 +1,14 @@ +{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"106","name":"hammer","description":"16oz carpenter's hammer","weight":"1.0"},{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"mydb","es":1598944132000,"id":1,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944146308,"type":"INSERT"} +{"data":[{"id":"106","name":"hammer","description":"18oz carpenter hammer","weight":"1.0"}],"database":"mydb","es":1598944202000,"id":2,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"description":"16oz carpenter's hammer"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944202218,"type":"UPDATE"} +{"data":null,"database":"mydb","es":1598944271000,"id":3,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE TABLE orders (\n order_number INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,\n order_date DATE NOT NULL,\n purchaser INTEGER NOT NULL,\n quantity INTEGER NOT NULL,\n product_id INTEGER NOT NULL\n) AUTO_INCREMENT = 10001","sqlType":null,"table":"orders","ts":1598944271192,"type":"CREATE"} +{"data":[{"order_number":"10001","order_date":"2016-01-16","purchaser":"1001","quantity":"1","product_id":"102"},{"order_number":"10002","order_date":"2016-01-17","purchaser":"1002","quantity":"2","product_id":"105"},{"order_number":"10003","order_date":"2016-02-19","purchaser":"1002","quantity":"2","product_id":"106"},{"order_number":"10004","order_date":"2016-02-21","purchaser":"1003","quantity":"1","product_id":"107"}],"database":"mydb","es":1598944275000,"id":4,"isDdl":false,"mysqlType":{"order_number":"INTEGER","order_date":"DATE","purchaser":"INTEGER","quantity":"INTEGER","product_id":"INTEGER"},"old":null,"pkNames":["order_number"],"sql":"","sqlType":{"order_number":4,"order_date":91,"purchaser":4,"quantity":4,"product_id":4},"table":"orders","ts":1598944275018,"type":"INSERT"} +{"data":[{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.1"}],"database":"mydb","es":1598944279000,"id":5,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944279665,"type":"UPDATE"} +{"data":[{"id":"110","name":"jacket","description":"water resistent white wind breaker","weight":"0.2"}],"database":"mydb","es":1598944288000,"id":6,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944288394,"type":"INSERT"} +{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.18"}],"database":"mydb","es":1598944288000,"id":6,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944288394,"type":"INSERT"} +{"data":[{"id":"110","name":"jacket","description":"new water resistent white wind breaker","weight":"0.5"}],"database":"mydb","es":1598944288000,"id":7,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"description":"water resistent white wind breaker","weight":"0.2"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944288717,"type":"UPDATE"} +{"data":[{"order_number":"10001","order_date":"2016-01-16","purchaser":"1001","quantity":"3","product_id":"102"}],"database":"mydb","es":1598944331000,"id":8,"isDdl":false,"mysqlType":{"order_number":"INTEGER","order_date":"DATE","purchaser":"INTEGER","quantity":"INTEGER","product_id":"INTEGER"},"old":[{"quantity":"1"}],"pkNames":["order_number"],"sql":"","sqlType":{"order_number":4,"order_date":91,"purchaser":4,"quantity":4,"product_id":4},"table":"orders","ts":1598944331870,"type":"UPDATE"} +{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"mydb","es":1598944337000,"id":9,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"weight":"5.18"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944337341,"type":"UPDATE"} +{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"mydb","es":1598944337000,"id":9,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944337341,"type":"DELETE"} +{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"5.17"},{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"}],"database":"mydb","es":1598944337000,"id":10,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"weight":"3.14"},{"weight":"8.1"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944337663,"type":"UPDATE"} +{"data":[{"order_number":"10002","order_date":"2016-01-17","purchaser":"1002","quantity":"2","product_id":"105"}],"database":"mydb","es":1598944374000,"id":11,"isDdl":false,"mysqlType":{"order_number":"INTEGER","order_date":"DATE","purchaser":"INTEGER","quantity":"INTEGER","product_id":"INTEGER"},"old":null,"pkNames":["order_number"],"sql":"","sqlType":{"order_number":4,"order_date":91,"purchaser":4,"quantity":4,"product_id":4},"table":"orders","ts":1598944374999,"type":"DELETE"} +{"data":[{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"}],"database":"mydb","es":1598944418000,"id":12,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944418418,"type":"DELETE"} \ No newline at end of file From 1c7ecd9f442f74ee7705377176a275c5c7935be1 Mon Sep 17 00:00:00 2001 From: Brian Zhou <969677093@qq.com> Date: Fri, 25 Sep 2020 22:50:09 +0800 Subject: [PATCH 093/322] [FLINK-19098][json][csv] Make RowData CSV and JSON converters public This closes #13303 Co-authored-by: Jark Wu --- .../JsonRowDataDeserializationSchema.java | 363 +--------------- .../json/JsonRowDataSerializationSchema.java | 260 +----------- .../formats/json/JsonToRowDataConverters.java | 393 ++++++++++++++++++ .../formats/json/RowDataToJsonConverters.java | 291 +++++++++++++ 4 files changed, 704 insertions(+), 603 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index 0804ae3a9..b7c5d0732 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -21,51 +21,16 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeFamily; -import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; -import org.apache.flink.table.types.logical.utils.LogicalTypeUtils; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; import java.io.IOException; -import java.io.Serializable; -import java.lang.reflect.Array; -import java.math.BigDecimal; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.ZoneOffset; -import java.time.temporal.TemporalAccessor; -import java.time.temporal.TemporalQueries; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; import java.util.Objects; import static java.lang.String.format; -import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -83,18 +48,25 @@ public class JsonRowDataDeserializationSchema implements DeserializationSchema resultTypeInfo; /** * Runtime converter that converts {@link JsonNode}s into - * objects of Flink SQL internal data structures. **/ - private final DeserializationRuntimeConverter runtimeConverter; + * objects of Flink SQL internal data structures. + **/ + private final JsonToRowDataConverters.JsonToRowDataConverter runtimeConverter; - /** Object mapper for parsing the JSON. */ + /** + * Object mapper for parsing the JSON. + */ private final ObjectMapper objectMapper = new ObjectMapper(); /** Timestamp format specification which is used to parse timestamp. */ @@ -113,7 +85,8 @@ public JsonRowDataDeserializationSchema( this.resultTypeInfo = checkNotNull(resultTypeInfo); this.failOnMissingField = failOnMissingField; this.ignoreParseErrors = ignoreParseErrors; - this.runtimeConverter = createRowConverter(checkNotNull(rowType)); + this.runtimeConverter = new JsonToRowDataConverters(failOnMissingField, ignoreParseErrors, timestampFormat) + .createRowConverter(checkNotNull(rowType)); this.timestampFormat = timestampFormat; } @@ -150,315 +123,13 @@ public boolean equals(Object o) { } JsonRowDataDeserializationSchema that = (JsonRowDataDeserializationSchema) o; return failOnMissingField == that.failOnMissingField && - ignoreParseErrors == that.ignoreParseErrors && - resultTypeInfo.equals(that.resultTypeInfo) && - timestampFormat.equals(that.timestampFormat); + ignoreParseErrors == that.ignoreParseErrors && + resultTypeInfo.equals(that.resultTypeInfo) && + timestampFormat.equals(that.timestampFormat); } @Override public int hashCode() { return Objects.hash(failOnMissingField, ignoreParseErrors, resultTypeInfo, timestampFormat); } - - // ------------------------------------------------------------------------------------- - // Runtime Converters - // ------------------------------------------------------------------------------------- - - /** - * Runtime converter that converts {@link JsonNode}s into objects of Flink Table & SQL - * internal data structures. - */ - @FunctionalInterface - private interface DeserializationRuntimeConverter extends Serializable { - Object convert(JsonNode jsonNode); - } - - /** - * Creates a runtime converter which is null safe. - */ - private DeserializationRuntimeConverter createConverter(LogicalType type) { - return wrapIntoNullableConverter(createNotNullConverter(type)); - } - - /** - * Creates a runtime converter which assuming input object is not null. - */ - private DeserializationRuntimeConverter createNotNullConverter(LogicalType type) { - switch (type.getTypeRoot()) { - case NULL: - return jsonNode -> null; - case BOOLEAN: - return this::convertToBoolean; - case TINYINT: - return jsonNode -> Byte.parseByte(jsonNode.asText().trim()); - case SMALLINT: - return jsonNode -> Short.parseShort(jsonNode.asText().trim()); - case INTEGER: - case INTERVAL_YEAR_MONTH: - return this::convertToInt; - case BIGINT: - case INTERVAL_DAY_TIME: - return this::convertToLong; - case DATE: - return this::convertToDate; - case TIME_WITHOUT_TIME_ZONE: - return this::convertToTime; - case TIMESTAMP_WITHOUT_TIME_ZONE: - return this::convertToTimestamp; - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return this::convertToTimestampWithLocalZone; - case FLOAT: - return this::convertToFloat; - case DOUBLE: - return this::convertToDouble; - case CHAR: - case VARCHAR: - return this::convertToString; - case BINARY: - case VARBINARY: - return this::convertToBytes; - case DECIMAL: - return createDecimalConverter((DecimalType) type); - case ARRAY: - return createArrayConverter((ArrayType) type); - case MAP: - case MULTISET: - return createMapConverter((MapType) type); - case ROW: - return createRowConverter((RowType) type); - case RAW: - default: - throw new UnsupportedOperationException("Unsupported type: " + type); - } - } - - private boolean convertToBoolean(JsonNode jsonNode) { - if (jsonNode.isBoolean()) { - // avoid redundant toString and parseBoolean, for better performance - return jsonNode.asBoolean(); - } else { - return Boolean.parseBoolean(jsonNode.asText().trim()); - } - } - - private int convertToInt(JsonNode jsonNode) { - if (jsonNode.canConvertToInt()) { - // avoid redundant toString and parseInt, for better performance - return jsonNode.asInt(); - } else { - return Integer.parseInt(jsonNode.asText().trim()); - } - } - - private long convertToLong(JsonNode jsonNode) { - if (jsonNode.canConvertToLong()) { - // avoid redundant toString and parseLong, for better performance - return jsonNode.asLong(); - } else { - return Long.parseLong(jsonNode.asText().trim()); - } - } - - private double convertToDouble(JsonNode jsonNode) { - if (jsonNode.isDouble()) { - // avoid redundant toString and parseDouble, for better performance - return jsonNode.asDouble(); - } else { - return Double.parseDouble(jsonNode.asText().trim()); - } - } - - private float convertToFloat(JsonNode jsonNode) { - if (jsonNode.isDouble()) { - // avoid redundant toString and parseDouble, for better performance - return (float) jsonNode.asDouble(); - } else { - return Float.parseFloat(jsonNode.asText().trim()); - } - } - - private int convertToDate(JsonNode jsonNode) { - LocalDate date = ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); - return (int) date.toEpochDay(); - } - - private int convertToTime(JsonNode jsonNode) { - TemporalAccessor parsedTime = SQL_TIME_FORMAT.parse(jsonNode.asText()); - LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); - - // get number of milliseconds of the day - return localTime.toSecondOfDay() * 1000; - } - - private TimestampData convertToTimestamp(JsonNode jsonNode) { - TemporalAccessor parsedTimestamp; - switch (timestampFormat){ - case SQL: - parsedTimestamp = SQL_TIMESTAMP_FORMAT.parse(jsonNode.asText()); - break; - case ISO_8601: - parsedTimestamp = ISO8601_TIMESTAMP_FORMAT.parse(jsonNode.asText()); - break; - default: - throw new TableException(String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); - } - LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); - LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); - - return TimestampData.fromLocalDateTime(LocalDateTime.of(localDate, localTime)); - } - - private TimestampData convertToTimestampWithLocalZone(JsonNode jsonNode){ - TemporalAccessor parsedTimestampWithLocalZone; - switch (timestampFormat){ - case SQL: - parsedTimestampWithLocalZone = SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); - break; - case ISO_8601: - parsedTimestampWithLocalZone = ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); - break; - default: - throw new TableException(String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); - } - LocalTime localTime = parsedTimestampWithLocalZone.query(TemporalQueries.localTime()); - LocalDate localDate = parsedTimestampWithLocalZone.query(TemporalQueries.localDate()); - - return TimestampData.fromInstant(LocalDateTime.of(localDate, localTime).toInstant(ZoneOffset.UTC)); - } - - private StringData convertToString(JsonNode jsonNode) { - if (jsonNode.isContainerNode()) { - return StringData.fromString(jsonNode.toString()); - } else { - return StringData.fromString(jsonNode.asText()); - } - } - - private byte[] convertToBytes(JsonNode jsonNode) { - try { - return jsonNode.binaryValue(); - } catch (IOException e) { - throw new JsonParseException("Unable to deserialize byte array.", e); - } - } - - private DeserializationRuntimeConverter createDecimalConverter(DecimalType decimalType) { - final int precision = decimalType.getPrecision(); - final int scale = decimalType.getScale(); - return jsonNode -> { - BigDecimal bigDecimal; - if (jsonNode.isBigDecimal()) { - bigDecimal = jsonNode.decimalValue(); - } else { - bigDecimal = new BigDecimal(jsonNode.asText()); - } - return DecimalData.fromBigDecimal(bigDecimal, precision, scale); - }; - } - - private DeserializationRuntimeConverter createArrayConverter(ArrayType arrayType) { - DeserializationRuntimeConverter elementConverter = createConverter(arrayType.getElementType()); - final Class elementClass = LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType()); - return jsonNode -> { - final ArrayNode node = (ArrayNode) jsonNode; - final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); - for (int i = 0; i < node.size(); i++) { - final JsonNode innerNode = node.get(i); - array[i] = elementConverter.convert(innerNode); - } - return new GenericArrayData(array); - }; - } - - private DeserializationRuntimeConverter createMapConverter(MapType mapType) { - LogicalType keyType = mapType.getKeyType(); - if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { - throw new UnsupportedOperationException( - "JSON format doesn't support non-string as key type of map. " + - "The map type is: " + mapType.asSummaryString()); - } - final DeserializationRuntimeConverter keyConverter = createConverter(keyType); - final DeserializationRuntimeConverter valueConverter = createConverter(mapType.getValueType()); - - return jsonNode -> { - Iterator> fields = jsonNode.fields(); - Map result = new HashMap<>(); - while (fields.hasNext()) { - Map.Entry entry = fields.next(); - Object key = keyConverter.convert(TextNode.valueOf(entry.getKey())); - Object value = valueConverter.convert(entry.getValue()); - result.put(key, value); - } - return new GenericMapData(result); - }; - } - - private DeserializationRuntimeConverter createRowConverter(RowType rowType) { - final DeserializationRuntimeConverter[] fieldConverters = rowType.getFields().stream() - .map(RowType.RowField::getType) - .map(this::createConverter) - .toArray(DeserializationRuntimeConverter[]::new); - final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]); - - return jsonNode -> { - ObjectNode node = (ObjectNode) jsonNode; - int arity = fieldNames.length; - GenericRowData row = new GenericRowData(arity); - for (int i = 0; i < arity; i++) { - String fieldName = fieldNames[i]; - JsonNode field = node.get(fieldName); - Object convertedField = convertField(fieldConverters[i], fieldName, field); - row.setField(i, convertedField); - } - return row; - }; - } - - private Object convertField( - DeserializationRuntimeConverter fieldConverter, - String fieldName, - JsonNode field) { - if (field == null) { - if (failOnMissingField) { - throw new JsonParseException( - "Could not find field with name '" + fieldName + "'."); - } else { - return null; - } - } else { - return fieldConverter.convert(field); - } - } - - private DeserializationRuntimeConverter wrapIntoNullableConverter( - DeserializationRuntimeConverter converter) { - return jsonNode -> { - if (jsonNode == null || jsonNode.isNull()) { - return null; - } - try { - return converter.convert(jsonNode); - } catch (Throwable t) { - if (!ignoreParseErrors) { - throw t; - } - return null; - } - }; - } - - /** - * Exception which refers to parse errors in converters. - * */ - private static final class JsonParseException extends RuntimeException { - private static final long serialVersionUID = 1L; - - public JsonParseException(String message) { - super(message); - } - - public JsonParseException(String message, Throwable cause) { - super(message, cause); - } - } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index ba35e7a8a..19bf510c8 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -20,39 +20,14 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeFamily; -import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; -import java.io.Serializable; -import java.math.BigDecimal; -import java.time.LocalDate; -import java.time.LocalTime; -import java.time.ZoneOffset; -import java.util.Arrays; import java.util.Objects; -import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; - /** * Serialization schema that serializes an object of Flink internal data structure into a JSON bytes. * @@ -69,7 +44,7 @@ public class JsonRowDataSerializationSchema implements SerializationSchema mapper.getNodeFactory().nullNode(); - case BOOLEAN: - return (mapper, reuse, value) -> mapper.getNodeFactory().booleanNode((boolean) value); - case TINYINT: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((byte) value); - case SMALLINT: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((short) value); - case INTEGER: - case INTERVAL_YEAR_MONTH: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((int) value); - case BIGINT: - case INTERVAL_DAY_TIME: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((long) value); - case FLOAT: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((float) value); - case DOUBLE: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((double) value); - case CHAR: - case VARCHAR: - // value is BinaryString - return (mapper, reuse, value) -> mapper.getNodeFactory().textNode(value.toString()); - case BINARY: - case VARBINARY: - return (mapper, reuse, value) -> mapper.getNodeFactory().binaryNode((byte[]) value); - case DATE: - return createDateConverter(); - case TIME_WITHOUT_TIME_ZONE: - return createTimeConverter(); - case TIMESTAMP_WITHOUT_TIME_ZONE: - return createTimestampConverter(); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return createTimestampWithLocalZone(); - case DECIMAL: - return createDecimalConverter(); - case ARRAY: - return createArrayConverter((ArrayType) type); - case MAP: - case MULTISET: - return createMapConverter((MapType) type); - case ROW: - return createRowConverter((RowType) type); - case RAW: - default: - throw new UnsupportedOperationException("Not support to parse type: " + type); - } - } - - private SerializationRuntimeConverter createDecimalConverter() { - return (mapper, reuse, value) -> { - BigDecimal bd = ((DecimalData) value).toBigDecimal(); - return mapper.getNodeFactory().numberNode(bd); - }; - } - - private SerializationRuntimeConverter createDateConverter() { - return (mapper, reuse, value) -> { - int days = (int) value; - LocalDate date = LocalDate.ofEpochDay(days); - return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format(date)); - }; - } - - private SerializationRuntimeConverter createTimeConverter() { - return (mapper, reuse, value) -> { - int millisecond = (int) value; - LocalTime time = LocalTime.ofSecondOfDay(millisecond / 1000L); - return mapper.getNodeFactory().textNode(SQL_TIME_FORMAT.format(time)); - }; - } - - private SerializationRuntimeConverter createTimestampConverter() { - switch (timestampFormat){ - case ISO_8601: - return (mapper, reuse, value) -> { - TimestampData timestamp = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(ISO8601_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); - }; - case SQL: - return (mapper, reuse, value) -> { - TimestampData timestamp = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(SQL_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); - }; - default: - throw new TableException("Unsupported timestamp format. Validator should have checked that."); - } - } - - private SerializationRuntimeConverter createTimestampWithLocalZone() { - switch (timestampFormat){ - case ISO_8601: - return (mapper, reuse, value) -> { - TimestampData timestampWithLocalZone = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format(timestampWithLocalZone.toInstant().atOffset(ZoneOffset.UTC))); - }; - case SQL: - return (mapper, reuse, value) -> { - TimestampData timestampWithLocalZone = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format(timestampWithLocalZone.toInstant().atOffset(ZoneOffset.UTC))); - }; - default: - throw new TableException("Unsupported timestamp format. Validator should have checked that."); - } - } - - private SerializationRuntimeConverter createArrayConverter(ArrayType type) { - final LogicalType elementType = type.getElementType(); - final SerializationRuntimeConverter elementConverter = createConverter(elementType); - return (mapper, reuse, value) -> { - ArrayNode node; - - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createArrayNode(); - } else { - node = (ArrayNode) reuse; - node.removeAll(); - } - - ArrayData array = (ArrayData) value; - int numElements = array.size(); - for (int i = 0; i < numElements; i++) { - Object element = ArrayData.get(array, i, elementType); - node.add(elementConverter.convert(mapper, null, element)); - } - - return node; - }; - } - - private SerializationRuntimeConverter createMapConverter(MapType type) { - LogicalType keyType = type.getKeyType(); - if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { - throw new UnsupportedOperationException( - "JSON format doesn't support non-string as key type of map. " + - "The map type is: " + type.asSummaryString()); - } - final LogicalType valueType = type.getValueType(); - final SerializationRuntimeConverter valueConverter = createConverter(valueType); - return (mapper, reuse, object) -> { - ObjectNode node; - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createObjectNode(); - } else { - node = (ObjectNode) reuse; - } - - MapData map = (MapData) object; - ArrayData keyArray = map.keyArray(); - ArrayData valueArray = map.valueArray(); - int numElements = map.size(); - for (int i = 0; i < numElements; i++) { - String fieldName = keyArray.getString(i).toString(); // key must be string - Object value = ArrayData.get(valueArray, i, valueType); - node.set(fieldName, valueConverter.convert(mapper, node.get(fieldName), value)); - } - - return node; - }; - } - - private SerializationRuntimeConverter createRowConverter(RowType type) { - final String[] fieldNames = type.getFieldNames().toArray(new String[0]); - final LogicalType[] fieldTypes = type.getFields().stream() - .map(RowType.RowField::getType) - .toArray(LogicalType[]::new); - final SerializationRuntimeConverter[] fieldConverters = Arrays.stream(fieldTypes) - .map(this::createConverter) - .toArray(SerializationRuntimeConverter[]::new); - final int fieldCount = type.getFieldCount(); - - return (mapper, reuse, value) -> { - ObjectNode node; - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createObjectNode(); - } else { - node = (ObjectNode) reuse; - } - RowData row = (RowData) value; - for (int i = 0; i < fieldCount; i++) { - String fieldName = fieldNames[i]; - Object field = RowData.get(row, i, fieldTypes[i]); - node.set(fieldName, fieldConverters[i].convert(mapper, node.get(fieldName), field)); - } - return node; - }; - } - - private SerializationRuntimeConverter wrapIntoNullableConverter( - SerializationRuntimeConverter converter) { - return (mapper, reuse, object) -> { - if (object == null) { - return mapper.getNodeFactory().nullNode(); - } - - return converter.convert(mapper, reuse, object); - }; - } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java new file mode 100644 index 000000000..875d85eff --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.table.types.logical.utils.LogicalTypeUtils; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; + +import java.io.IOException; +import java.io.Serializable; +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.time.temporal.TemporalAccessor; +import java.time.temporal.TemporalQueries; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; +import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; + +/** Tool class used to convert from {@link JsonNode} to {@link RowData}. **/ +@Internal +public class JsonToRowDataConverters implements Serializable { + + private static final long serialVersionUID = 1L; + + /** + * Flag indicating whether to fail if a field is missing. + */ + private final boolean failOnMissingField; + + /** + * Flag indicating whether to ignore invalid fields/rows (default: throw an exception). + */ + private final boolean ignoreParseErrors; + + /** + * Timestamp format specification which is used to parse timestamp. + */ + private final TimestampFormat timestampFormat; + + public JsonToRowDataConverters( + boolean failOnMissingField, + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { + this.failOnMissingField = failOnMissingField; + this.ignoreParseErrors = ignoreParseErrors; + this.timestampFormat = timestampFormat; + } + + /** + * Runtime converter that converts {@link JsonNode}s into objects of Flink Table & SQL + * internal data structures. + */ + @FunctionalInterface + public interface JsonToRowDataConverter extends Serializable { + Object convert(JsonNode jsonNode); + } + + /** + * Creates a runtime converter which is null safe. + */ + private JsonToRowDataConverter createConverter(LogicalType type) { + return wrapIntoNullableConverter(createNotNullConverter(type)); + } + + /** + * Creates a runtime converter which assuming input object is not null. + */ + private JsonToRowDataConverter createNotNullConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return jsonNode -> null; + case BOOLEAN: + return this::convertToBoolean; + case TINYINT: + return jsonNode -> Byte.parseByte(jsonNode.asText().trim()); + case SMALLINT: + return jsonNode -> Short.parseShort(jsonNode.asText().trim()); + case INTEGER: + case INTERVAL_YEAR_MONTH: + return this::convertToInt; + case BIGINT: + case INTERVAL_DAY_TIME: + return this::convertToLong; + case DATE: + return this::convertToDate; + case TIME_WITHOUT_TIME_ZONE: + return this::convertToTime; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return this::convertToTimestamp; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return this::convertToTimestampWithLocalZone; + case FLOAT: + return this::convertToFloat; + case DOUBLE: + return this::convertToDouble; + case CHAR: + case VARCHAR: + return this::convertToString; + case BINARY: + case VARBINARY: + return this::convertToBytes; + case DECIMAL: + return createDecimalConverter((DecimalType) type); + case ARRAY: + return createArrayConverter((ArrayType) type); + case MAP: + case MULTISET: + return createMapConverter((MapType) type); + case ROW: + return createRowConverter((RowType) type); + case RAW: + default: + throw new UnsupportedOperationException("Unsupported type: " + type); + } + } + + private boolean convertToBoolean(JsonNode jsonNode) { + if (jsonNode.isBoolean()) { + // avoid redundant toString and parseBoolean, for better performance + return jsonNode.asBoolean(); + } else { + return Boolean.parseBoolean(jsonNode.asText().trim()); + } + } + + private int convertToInt(JsonNode jsonNode) { + if (jsonNode.canConvertToInt()) { + // avoid redundant toString and parseInt, for better performance + return jsonNode.asInt(); + } else { + return Integer.parseInt(jsonNode.asText().trim()); + } + } + + private long convertToLong(JsonNode jsonNode) { + if (jsonNode.canConvertToLong()) { + // avoid redundant toString and parseLong, for better performance + return jsonNode.asLong(); + } else { + return Long.parseLong(jsonNode.asText().trim()); + } + } + + private double convertToDouble(JsonNode jsonNode) { + if (jsonNode.isDouble()) { + // avoid redundant toString and parseDouble, for better performance + return jsonNode.asDouble(); + } else { + return Double.parseDouble(jsonNode.asText().trim()); + } + } + + private float convertToFloat(JsonNode jsonNode) { + if (jsonNode.isDouble()) { + // avoid redundant toString and parseDouble, for better performance + return (float) jsonNode.asDouble(); + } else { + return Float.parseFloat(jsonNode.asText().trim()); + } + } + + private int convertToDate(JsonNode jsonNode) { + LocalDate date = ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); + return (int) date.toEpochDay(); + } + + private int convertToTime(JsonNode jsonNode) { + TemporalAccessor parsedTime = SQL_TIME_FORMAT.parse(jsonNode.asText()); + LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); + + // get number of milliseconds of the day + return localTime.toSecondOfDay() * 1000; + } + + private TimestampData convertToTimestamp(JsonNode jsonNode) { + TemporalAccessor parsedTimestamp; + switch (timestampFormat) { + case SQL: + parsedTimestamp = SQL_TIMESTAMP_FORMAT.parse(jsonNode.asText()); + break; + case ISO_8601: + parsedTimestamp = ISO8601_TIMESTAMP_FORMAT.parse(jsonNode.asText()); + break; + default: + throw new TableException(String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); + } + LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); + LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); + + return TimestampData.fromLocalDateTime(LocalDateTime.of(localDate, localTime)); + } + + private TimestampData convertToTimestampWithLocalZone(JsonNode jsonNode){ + TemporalAccessor parsedTimestampWithLocalZone; + switch (timestampFormat){ + case SQL: + parsedTimestampWithLocalZone = SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); + break; + case ISO_8601: + parsedTimestampWithLocalZone = ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); + break; + default: + throw new TableException(String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); + } + LocalTime localTime = parsedTimestampWithLocalZone.query(TemporalQueries.localTime()); + LocalDate localDate = parsedTimestampWithLocalZone.query(TemporalQueries.localDate()); + + return TimestampData.fromInstant(LocalDateTime.of(localDate, localTime).toInstant(ZoneOffset.UTC)); + } + + private StringData convertToString(JsonNode jsonNode) { + if (jsonNode.isContainerNode()) { + return StringData.fromString(jsonNode.toString()); + } else { + return StringData.fromString(jsonNode.asText()); + } + } + + private byte[] convertToBytes(JsonNode jsonNode) { + try { + return jsonNode.binaryValue(); + } catch (IOException e) { + throw new JsonParseException("Unable to deserialize byte array.", e); + } + } + + private JsonToRowDataConverter createDecimalConverter(DecimalType decimalType) { + final int precision = decimalType.getPrecision(); + final int scale = decimalType.getScale(); + return jsonNode -> { + BigDecimal bigDecimal; + if (jsonNode.isBigDecimal()) { + bigDecimal = jsonNode.decimalValue(); + } else { + bigDecimal = new BigDecimal(jsonNode.asText()); + } + return DecimalData.fromBigDecimal(bigDecimal, precision, scale); + }; + } + + private JsonToRowDataConverter createArrayConverter(ArrayType arrayType) { + JsonToRowDataConverter elementConverter = createConverter(arrayType.getElementType()); + final Class elementClass = LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType()); + return jsonNode -> { + final ArrayNode node = (ArrayNode) jsonNode; + final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); + for (int i = 0; i < node.size(); i++) { + final JsonNode innerNode = node.get(i); + array[i] = elementConverter.convert(innerNode); + } + return new GenericArrayData(array); + }; + } + + private JsonToRowDataConverter createMapConverter(MapType mapType) { + LogicalType keyType = mapType.getKeyType(); + if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { + throw new UnsupportedOperationException( + "JSON format doesn't support non-string as key type of map. " + + "The map type is: " + mapType.asSummaryString()); + } + final JsonToRowDataConverter keyConverter = createConverter(keyType); + final JsonToRowDataConverter valueConverter = createConverter(mapType.getValueType()); + + return jsonNode -> { + Iterator> fields = jsonNode.fields(); + Map result = new HashMap<>(); + while (fields.hasNext()) { + Map.Entry entry = fields.next(); + Object key = keyConverter.convert(TextNode.valueOf(entry.getKey())); + Object value = valueConverter.convert(entry.getValue()); + result.put(key, value); + } + return new GenericMapData(result); + }; + } + + public JsonToRowDataConverter createRowConverter(RowType rowType) { + final JsonToRowDataConverter[] fieldConverters = rowType.getFields().stream() + .map(RowType.RowField::getType) + .map(this::createConverter) + .toArray(JsonToRowDataConverter[]::new); + final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]); + + return jsonNode -> { + ObjectNode node = (ObjectNode) jsonNode; + int arity = fieldNames.length; + GenericRowData row = new GenericRowData(arity); + for (int i = 0; i < arity; i++) { + String fieldName = fieldNames[i]; + JsonNode field = node.get(fieldName); + Object convertedField = convertField(fieldConverters[i], fieldName, field); + row.setField(i, convertedField); + } + return row; + }; + } + + private Object convertField( + JsonToRowDataConverter fieldConverter, + String fieldName, + JsonNode field) { + if (field == null) { + if (failOnMissingField) { + throw new JsonParseException( + "Could not find field with name '" + fieldName + "'."); + } else { + return null; + } + } else { + return fieldConverter.convert(field); + } + } + + private JsonToRowDataConverter wrapIntoNullableConverter( + JsonToRowDataConverter converter) { + return jsonNode -> { + if (jsonNode == null || jsonNode.isNull()) { + return null; + } + try { + return converter.convert(jsonNode); + } catch (Throwable t) { + if (!ignoreParseErrors) { + throw t; + } + return null; + } + }; + } + + /** + * Exception which refers to parse errors in converters. + * */ + private static final class JsonParseException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public JsonParseException(String message) { + super(message); + } + + public JsonParseException(String message, Throwable cause) { + super(message, cause); + } + } + +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java new file mode 100644 index 000000000..f1d81f1a4 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.util.Arrays; + +import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; +import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; +import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; + +/** Tool class used to convert from {@link RowData} to {@link JsonNode}. **/ +@Internal +public class RowDataToJsonConverters implements Serializable { + + private static final long serialVersionUID = 1L; + + /** Timestamp format specification which is used to parse timestamp. */ + private final TimestampFormat timestampFormat; + + public RowDataToJsonConverters(TimestampFormat timestampFormat) { + this.timestampFormat = timestampFormat; + } + + /** + * Runtime converter that converts objects of Flink Table & SQL internal data structures + * to corresponding {@link JsonNode}s. + */ + public interface RowDataToJsonConverter extends Serializable { + JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value); + } + + /** + * Creates a runtime converter which is null safe. + */ + public RowDataToJsonConverter createConverter(LogicalType type) { + return wrapIntoNullableConverter(createNotNullConverter(type)); + } + + /** + * Creates a runtime converter which assuming input object is not null. + */ + private RowDataToJsonConverter createNotNullConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return (mapper, reuse, value) -> mapper.getNodeFactory().nullNode(); + case BOOLEAN: + return (mapper, reuse, value) -> mapper.getNodeFactory().booleanNode((boolean) value); + case TINYINT: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((byte) value); + case SMALLINT: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((short) value); + case INTEGER: + case INTERVAL_YEAR_MONTH: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((int) value); + case BIGINT: + case INTERVAL_DAY_TIME: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((long) value); + case FLOAT: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((float) value); + case DOUBLE: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((double) value); + case CHAR: + case VARCHAR: + // value is BinaryString + return (mapper, reuse, value) -> mapper.getNodeFactory().textNode(value.toString()); + case BINARY: + case VARBINARY: + return (mapper, reuse, value) -> mapper.getNodeFactory().binaryNode((byte[]) value); + case DATE: + return createDateConverter(); + case TIME_WITHOUT_TIME_ZONE: + return createTimeConverter(); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return createTimestampConverter(); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return createTimestampWithLocalZone(); + case DECIMAL: + return createDecimalConverter(); + case ARRAY: + return createArrayConverter((ArrayType) type); + case MAP: + case MULTISET: + return createMapConverter((MapType) type); + case ROW: + return createRowConverter((RowType) type); + case RAW: + default: + throw new UnsupportedOperationException("Not support to parse type: " + type); + } + } + + private RowDataToJsonConverter createDecimalConverter() { + return (mapper, reuse, value) -> { + BigDecimal bd = ((DecimalData) value).toBigDecimal(); + return mapper.getNodeFactory().numberNode(bd); + }; + } + + private RowDataToJsonConverter createDateConverter() { + return (mapper, reuse, value) -> { + int days = (int) value; + LocalDate date = LocalDate.ofEpochDay(days); + return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format(date)); + }; + } + + private RowDataToJsonConverter createTimeConverter() { + return (mapper, reuse, value) -> { + int millisecond = (int) value; + LocalTime time = LocalTime.ofSecondOfDay(millisecond / 1000L); + return mapper.getNodeFactory().textNode(SQL_TIME_FORMAT.format(time)); + }; + } + + private RowDataToJsonConverter createTimestampConverter() { + switch (timestampFormat){ + case ISO_8601: + return (mapper, reuse, value) -> { + TimestampData timestamp = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(ISO8601_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); + }; + case SQL: + return (mapper, reuse, value) -> { + TimestampData timestamp = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(SQL_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); + }; + default: + throw new TableException("Unsupported timestamp format. Validator should have checked that."); + } + } + + private RowDataToJsonConverter createTimestampWithLocalZone() { + switch (timestampFormat){ + case ISO_8601: + return (mapper, reuse, value) -> { + TimestampData timestampWithLocalZone = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format(timestampWithLocalZone.toInstant().atOffset(ZoneOffset.UTC))); + }; + case SQL: + return (mapper, reuse, value) -> { + TimestampData timestampWithLocalZone = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format(timestampWithLocalZone.toInstant().atOffset(ZoneOffset.UTC))); + }; + default: + throw new TableException("Unsupported timestamp format. Validator should have checked that."); + } + } + + private RowDataToJsonConverter createArrayConverter(ArrayType type) { + final LogicalType elementType = type.getElementType(); + final RowDataToJsonConverter elementConverter = createConverter(elementType); + return (mapper, reuse, value) -> { + ArrayNode node; + + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createArrayNode(); + } else { + node = (ArrayNode) reuse; + node.removeAll(); + } + + ArrayData array = (ArrayData) value; + int numElements = array.size(); + for (int i = 0; i < numElements; i++) { + Object element = ArrayData.get(array, i, elementType); + node.add(elementConverter.convert(mapper, null, element)); + } + + return node; + }; + } + + private RowDataToJsonConverter createMapConverter(MapType type) { + LogicalType keyType = type.getKeyType(); + if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { + throw new UnsupportedOperationException( + "JSON format doesn't support non-string as key type of map. " + + "The map type is: " + type.asSummaryString()); + } + final LogicalType valueType = type.getValueType(); + final RowDataToJsonConverter valueConverter = createConverter(valueType); + return (mapper, reuse, object) -> { + ObjectNode node; + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createObjectNode(); + } else { + node = (ObjectNode) reuse; + } + + MapData map = (MapData) object; + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + int numElements = map.size(); + for (int i = 0; i < numElements; i++) { + String fieldName = keyArray.getString(i).toString(); // key must be string + Object value = ArrayData.get(valueArray, i, valueType); + node.set(fieldName, valueConverter.convert(mapper, node.get(fieldName), value)); + } + + return node; + }; + } + + private RowDataToJsonConverter createRowConverter(RowType type) { + final String[] fieldNames = type.getFieldNames().toArray(new String[0]); + final LogicalType[] fieldTypes = type.getFields().stream() + .map(RowType.RowField::getType) + .toArray(LogicalType[]::new); + final RowDataToJsonConverter[] fieldConverters = Arrays.stream(fieldTypes) + .map(this::createConverter) + .toArray(RowDataToJsonConverter[]::new); + final int fieldCount = type.getFieldCount(); + + return (mapper, reuse, value) -> { + ObjectNode node; + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createObjectNode(); + } else { + node = (ObjectNode) reuse; + } + RowData row = (RowData) value; + for (int i = 0; i < fieldCount; i++) { + String fieldName = fieldNames[i]; + Object field = RowData.get(row, i, fieldTypes[i]); + node.set(fieldName, fieldConverters[i].convert(mapper, node.get(fieldName), field)); + } + return node; + }; + } + + private RowDataToJsonConverter wrapIntoNullableConverter( + RowDataToJsonConverter converter) { + return (mapper, reuse, object) -> { + if (object == null) { + return mapper.getNodeFactory().nullNode(); + } + + return converter.convert(mapper, reuse, object); + }; + } +} From edf51f6d3163a5881197fd62d57494c86c24aab2 Mon Sep 17 00:00:00 2001 From: wangxlong <18868816710@163.com> Date: Sat, 10 Oct 2020 15:56:01 +0800 Subject: [PATCH 094/322] [FLINK-19509][json] Support MULTISET type for JSON format This closes #13543 --- .../formats/json/JsonToRowDataConverters.java | 17 ++++++++++++----- .../formats/json/RowDataToJsonConverters.java | 16 +++++++++++----- .../json/JsonRowDataSerDeSchemaTest.java | 11 +++++++++-- 3 files changed, 32 insertions(+), 12 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java index 875d85eff..25df3df80 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java @@ -29,9 +29,11 @@ import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeFamily; import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.MultisetType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; import org.apache.flink.table.types.logical.utils.LogicalTypeUtils; @@ -150,8 +152,13 @@ private JsonToRowDataConverter createNotNullConverter(LogicalType type) { case ARRAY: return createArrayConverter((ArrayType) type); case MAP: + MapType mapType = (MapType) type; + return createMapConverter( + mapType.asSummaryString(), mapType.getKeyType(), mapType.getValueType()); case MULTISET: - return createMapConverter((MapType) type); + MultisetType multisetType = (MultisetType) type; + return createMapConverter( + multisetType.asSummaryString(), multisetType.getElementType(), new IntType()); case ROW: return createRowConverter((RowType) type); case RAW: @@ -298,15 +305,15 @@ private JsonToRowDataConverter createArrayConverter(ArrayType arrayType) { }; } - private JsonToRowDataConverter createMapConverter(MapType mapType) { - LogicalType keyType = mapType.getKeyType(); + private JsonToRowDataConverter createMapConverter( + String typeSummary, LogicalType keyType, LogicalType valueType) { if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { throw new UnsupportedOperationException( "JSON format doesn't support non-string as key type of map. " + - "The map type is: " + mapType.asSummaryString()); + "The type is: " + typeSummary); } final JsonToRowDataConverter keyConverter = createConverter(keyType); - final JsonToRowDataConverter valueConverter = createConverter(mapType.getValueType()); + final JsonToRowDataConverter valueConverter = createConverter(valueType); return jsonNode -> { Iterator> fields = jsonNode.fields(); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java index f1d81f1a4..62c9fa486 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java @@ -26,9 +26,11 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeFamily; import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.MultisetType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; @@ -122,8 +124,13 @@ private RowDataToJsonConverter createNotNullConverter(LogicalType type) { case ARRAY: return createArrayConverter((ArrayType) type); case MAP: + MapType mapType = (MapType) type; + return createMapConverter( + mapType.asSummaryString(), mapType.getKeyType(), mapType.getValueType()); case MULTISET: - return createMapConverter((MapType) type); + MultisetType multisetType = (MultisetType) type; + return createMapConverter + (multisetType.asSummaryString(), multisetType.getElementType(), new IntType()); case ROW: return createRowConverter((RowType) type); case RAW: @@ -218,14 +225,13 @@ private RowDataToJsonConverter createArrayConverter(ArrayType type) { }; } - private RowDataToJsonConverter createMapConverter(MapType type) { - LogicalType keyType = type.getKeyType(); + private RowDataToJsonConverter createMapConverter( + String typeSummary, LogicalType keyType, LogicalType valueType) { if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { throw new UnsupportedOperationException( "JSON format doesn't support non-string as key type of map. " + - "The map type is: " + type.asSummaryString()); + "The type is: " + typeSummary); } - final LogicalType valueType = type.getValueType(); final RowDataToJsonConverter valueConverter = createConverter(valueType); return (mapper, reuse, object) -> { ObjectNode node; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 7a150c355..e2a476208 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -58,6 +58,7 @@ import static org.apache.flink.table.api.DataTypes.FLOAT; import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.MAP; +import static org.apache.flink.table.api.DataTypes.MULTISET; import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.SMALLINT; import static org.apache.flink.table.api.DataTypes.STRING; @@ -97,6 +98,9 @@ public void testSerDe() throws Exception { Map map = new HashMap<>(); map.put("flink", 123L); + Map multiSet = new HashMap<>(); + multiSet.put("blink", 2); + Map> nestedMap = new HashMap<>(); Map innerMap = new HashMap<>(); innerMap.put("key", 234); @@ -123,6 +127,7 @@ public void testSerDe() throws Exception { root.put("timestamp9", "1990-10-14T12:12:43.123456789"); root.put("timestampWithLocalZone", "1990-10-14T12:12:43.123456789Z"); root.putObject("map").put("flink", 123); + root.putObject("multiSet").put("blink", 2); root.putObject("map2map").putObject("inner_map").put("key", 234); byte[] serializedJson = objectMapper.writeValueAsBytes(root); @@ -144,6 +149,7 @@ public void testSerDe() throws Exception { FIELD("timestamp9", TIMESTAMP(9)), FIELD("timestampWithLocalZone", TIMESTAMP_WITH_LOCAL_TIME_ZONE(9)), FIELD("map", MAP(STRING(), BIGINT())), + FIELD("multiSet", MULTISET(STRING())), FIELD("map2map", MAP(STRING(), MAP(STRING(), INT())))); RowType schema = (RowType) dataType.getLogicalType(); TypeInformation resultTypeInfo = InternalTypeInfo.of(schema); @@ -151,7 +157,7 @@ public void testSerDe() throws Exception { JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); - Row expected = new Row(17); + Row expected = new Row(18); expected.setField(0, true); expected.setField(1, tinyint); expected.setField(2, smallint); @@ -168,7 +174,8 @@ public void testSerDe() throws Exception { expected.setField(13, timestamp9.toLocalDateTime()); expected.setField(14, timestampWithLocalZone); expected.setField(15, map); - expected.setField(16, nestedMap); + expected.setField(16, multiSet); + expected.setField(17, nestedMap); RowData rowData = deserializationSchema.deserialize(serializedJson); Row actual = convertToExternal(rowData, dataType); From 839f96ab038b6d37ab62d95ab2b9740427786b75 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Fri, 21 Aug 2020 20:17:33 +0800 Subject: [PATCH 095/322] [FLINK-18999][table-common][table-planner-blink] Add isTemporary flag to table factory context --- .../registry/confluent/RegistryAvroFormatFactoryTest.java | 6 ++++-- .../apache/flink/formats/json/JsonFormatFactoryTest.java | 6 ++++-- .../formats/json/canal/CanalJsonFormatFactoryTest.java | 6 ++++-- .../json/debezium/DebeziumJsonFormatFactoryTest.java | 6 ++++-- .../formats/json/maxwell/MaxwellJsonFormatFactoryTest.java | 6 ++++-- 5 files changed, 20 insertions(+), 10 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java index dbfcc794c..cef104ce4 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java @@ -163,7 +163,8 @@ private DynamicTableSource createTableSource(Map options) { ObjectIdentifier.of("default", "default", "t1"), new CatalogTableImpl(SCHEMA, options, "mock source"), new Configuration(), - RegistryAvroFormatFactoryTest.class.getClassLoader()); + RegistryAvroFormatFactoryTest.class.getClassLoader(), + false); } private DynamicTableSink createTableSink(Map options) { @@ -172,6 +173,7 @@ private DynamicTableSink createTableSink(Map options) { ObjectIdentifier.of("default", "default", "t1"), new CatalogTableImpl(SCHEMA, options, "mock sink"), new Configuration(), - RegistryAvroFormatFactoryTest.class.getClassLoader()); + RegistryAvroFormatFactoryTest.class.getClassLoader(), + false); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index 5c0747933..9bb4ba9e1 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -183,7 +183,8 @@ private static DynamicTableSource createTableSource(Map options) ObjectIdentifier.of("default", "default", "t1"), new CatalogTableImpl(SCHEMA, options, "Mock scan table"), new Configuration(), - JsonFormatFactoryTest.class.getClassLoader()); + JsonFormatFactoryTest.class.getClassLoader(), + false); } private static DynamicTableSink createTableSink(Map options) { @@ -192,6 +193,7 @@ private static DynamicTableSink createTableSink(Map options) { ObjectIdentifier.of("default", "default", "t1"), new CatalogTableImpl(SCHEMA, options, "Mock sink table"), new Configuration(), - JsonFormatFactoryTest.class.getClassLoader()); + JsonFormatFactoryTest.class.getClassLoader(), + false); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 4767c99f2..8beb8eabc 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -154,7 +154,8 @@ private static DeserializationSchema createDeserializationSchema(Map createSerializationSchema(Map options) ObjectIdentifier.of("default", "default", "t1"), new CatalogTableImpl(SCHEMA, options, "mock source"), new Configuration(), - DebeziumJsonFormatFactoryTest.class.getClassLoader()); + DebeziumJsonFormatFactoryTest.class.getClassLoader(), + false); } private static DynamicTableSink createTableSink(Map options) { @@ -157,6 +158,7 @@ private static DynamicTableSink createTableSink(Map options) { ObjectIdentifier.of("default", "default", "t1"), new CatalogTableImpl(SCHEMA, options, "mock sink"), new Configuration(), - DebeziumJsonFormatFactoryTest.class.getClassLoader()); + DebeziumJsonFormatFactoryTest.class.getClassLoader(), + false); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java index d9dcc06d9..569954a00 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java @@ -145,7 +145,8 @@ private static DynamicTableSource createTableSource(Map options) ObjectIdentifier.of("default", "default", "t1"), new CatalogTableImpl(SCHEMA, options, "mock source"), new Configuration(), - MaxwellJsonFormatFactoryTest.class.getClassLoader()); + MaxwellJsonFormatFactoryTest.class.getClassLoader(), + false); } private static DynamicTableSink createTableSink(Map options) { @@ -154,6 +155,7 @@ private static DynamicTableSink createTableSink(Map options) { ObjectIdentifier.of("default", "default", "t1"), new CatalogTableImpl(SCHEMA, options, "mock sink"), new Configuration(), - MaxwellJsonFormatFactoryTest.class.getClassLoader()); + MaxwellJsonFormatFactoryTest.class.getClassLoader(), + false); } } From 13d7fbbdb6594aa6822b249a6c9ff2e8699c5ef0 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 15 Oct 2020 20:56:14 +0800 Subject: [PATCH 096/322] [FLINK-17528][table] Remove RowData#get() API and use FieldGetter instead This closes #13653 --- .../apache/flink/formats/json/RowDataToJsonConverters.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java index 62c9fa486..a446fd890 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java @@ -265,6 +265,10 @@ private RowDataToJsonConverter createRowConverter(RowType type) { .map(this::createConverter) .toArray(RowDataToJsonConverter[]::new); final int fieldCount = type.getFieldCount(); + final RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[fieldTypes.length]; + for (int i = 0; i < fieldCount; i++) { + fieldGetters[i] = RowData.createFieldGetter(fieldTypes[i], i); + } return (mapper, reuse, value) -> { ObjectNode node; @@ -277,7 +281,7 @@ private RowDataToJsonConverter createRowConverter(RowType type) { RowData row = (RowData) value; for (int i = 0; i < fieldCount; i++) { String fieldName = fieldNames[i]; - Object field = RowData.get(row, i, fieldTypes[i]); + Object field = fieldGetters[i].getFieldOrNull(row); node.set(fieldName, fieldConverters[i].convert(mapper, node.get(fieldName), field)); } return node; From 2e4a694198d06c4d928c9b8e97627d90d4cfae54 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 15 Oct 2020 21:17:20 +0800 Subject: [PATCH 097/322] [FLINK-17528][table] Remove ArrayData#get() API and use ElementGetter instead This closes #13653 --- .../apache/flink/formats/json/RowDataToJsonConverters.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java index a446fd890..e7530f14c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java @@ -203,6 +203,7 @@ private RowDataToJsonConverter createTimestampWithLocalZone() { private RowDataToJsonConverter createArrayConverter(ArrayType type) { final LogicalType elementType = type.getElementType(); final RowDataToJsonConverter elementConverter = createConverter(elementType); + final ArrayData.ElementGetter elementGetter = ArrayData.createElementGetter(elementType); return (mapper, reuse, value) -> { ArrayNode node; @@ -217,7 +218,7 @@ private RowDataToJsonConverter createArrayConverter(ArrayType type) { ArrayData array = (ArrayData) value; int numElements = array.size(); for (int i = 0; i < numElements; i++) { - Object element = ArrayData.get(array, i, elementType); + Object element = elementGetter.getElementOrNull(array, i); node.add(elementConverter.convert(mapper, null, element)); } @@ -233,6 +234,7 @@ private RowDataToJsonConverter createMapConverter( "The type is: " + typeSummary); } final RowDataToJsonConverter valueConverter = createConverter(valueType); + final ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(valueType); return (mapper, reuse, object) -> { ObjectNode node; // reuse could be a NullNode if last record is null. @@ -248,7 +250,7 @@ private RowDataToJsonConverter createMapConverter( int numElements = map.size(); for (int i = 0; i < numElements; i++) { String fieldName = keyArray.getString(i).toString(); // key must be string - Object value = ArrayData.get(valueArray, i, valueType); + Object value = valueGetter.getElementOrNull(valueArray, i); node.set(fieldName, valueConverter.convert(mapper, node.get(fieldName), value)); } From c883034fed98028c1dd044bbebd550481afab12b Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 19 Oct 2020 14:18:46 +0200 Subject: [PATCH 098/322] [hotfix][table-common] Avoid unnecessary casting when creating type information in sources and sinks This it is not a compatible change. But given that those interfaces are still relatively new and not many people have changed to the new sources/sinks. We should do this change now or never and avoid @SuppressWarning in almost all implementations. --- .../avro/registry/confluent/RegistryAvroFormatFactory.java | 2 +- .../java/org/apache/flink/formats/json/JsonFormatFactory.java | 3 +-- .../flink/formats/json/canal/CanalJsonFormatFactory.java | 3 +-- .../flink/formats/json/debezium/DebeziumJsonFormatFactory.java | 3 +-- .../flink/formats/json/maxwell/MaxwellJsonFormatFactory.java | 3 +-- 5 files changed, 5 insertions(+), 9 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index 9a134254e..8b3c2bcaf 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -73,7 +73,7 @@ public DeserializationSchema createRuntimeDecoder( DataType producedDataType) { final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = - (TypeInformation) context.createTypeInformation(producedDataType); + context.createTypeInformation(producedDataType); return new AvroRowDataDeserializationSchema( ConfluentRegistryAvroDeserializationSchema.forGeneric( AvroSchemaConverter.convertToSchema(rowType), diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index 57952b920..1fe5da128 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -56,7 +56,6 @@ public class JsonFormatFactory implements public static final String IDENTIFIER = "json"; - @SuppressWarnings("unchecked") @Override public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, @@ -75,7 +74,7 @@ public DeserializationSchema createRuntimeDecoder( DataType producedDataType) { final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = - (TypeInformation) context.createTypeInformation(producedDataType); + context.createTypeInformation(producedDataType); return new JsonRowDataDeserializationSchema( rowType, rowDataTypeInfo, diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java index c5d649daa..0afc96c29 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java @@ -67,7 +67,6 @@ public class CanalJsonFormatFactory implements DeserializationFormatFactory, Ser .noDefaultValue() .withDescription("Only read changelog rows which match the specific table (by comparing the \"table\" meta field in the record)."); - @SuppressWarnings("unchecked") @Override public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, @@ -84,7 +83,7 @@ public DeserializationSchema createRuntimeDecoder( DynamicTableSource.Context context, DataType producedDataType) { final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = - (TypeInformation) context.createTypeInformation(producedDataType); + context.createTypeInformation(producedDataType); return CanalJsonDeserializationSchema .builder(rowType, rowDataTypeInfo) .setIgnoreParseErrors(ignoreParseErrors) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java index d81f7a1bf..794b1a49e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -64,7 +64,6 @@ public class DebeziumJsonFormatFactory implements DeserializationFormatFactory, public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; - @SuppressWarnings("unchecked") @Override public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, @@ -80,7 +79,7 @@ public DeserializationSchema createRuntimeDecoder( DynamicTableSource.Context context, DataType producedDataType) { final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = - (TypeInformation) context.createTypeInformation(producedDataType); + context.createTypeInformation(producedDataType); return new DebeziumJsonDeserializationSchema( rowType, rowDataTypeInfo, diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java index e3df0434c..f51e5e2ec 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java @@ -54,7 +54,6 @@ public class MaxwellJsonFormatFactory implements DeserializationFormatFactory, S public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; - @SuppressWarnings("unchecked") @Override public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, @@ -69,7 +68,7 @@ public DeserializationSchema createRuntimeDecoder( DynamicTableSource.Context context, DataType producedDataType) { final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = - (TypeInformation) context.createTypeInformation(producedDataType); + context.createTypeInformation(producedDataType); return new MaxwellJsonDeserializationSchema( rowType, rowDataTypeInfo, From ef72378f059ede89c9f84f4ad59b7a92ada822be Mon Sep 17 00:00:00 2001 From: wangxlong <18868816710@163.com> Date: Thu, 29 Oct 2020 12:43:09 +0800 Subject: [PATCH 099/322] [hotfix][json] Add serialVersionUID to JsonInputFormat class This closes #13809 --- .../apache/flink/formats/json/JsonFileSystemFormatFactory.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java index 1ecdc735d..831b8ca5d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java @@ -135,6 +135,9 @@ public Optional> createBulkWriterFactory(WriterConte * from json format files. */ public static class JsonInputFormat extends DelimitedInputFormat { + + private static final long serialVersionUID = 1L; + /** * Code of \r, used to remove \r from a line when the line ends with \r\n. */ From b86a6e2caae7c9e6cd0aacfe0f79fca3b01ded6a Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 27 Oct 2020 10:45:50 +0800 Subject: [PATCH 100/322] [FLINK-19779][avro] Remove the "record_" field name prefix for Avro format deserialization Never modify and prefix the field name, instead, we now use the {rowName}_{fieldName} as the nested row type name because Avro schema does not allow same name row type with different schema. --- .../registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java index dbcec71cd..ce3d630b5 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java @@ -139,7 +139,9 @@ private void testRowDataWriteReadWithSchema(Schema schema) throws Exception { RowType rowType = (RowType) dataType.getLogicalType(); AvroRowDataSerializationSchema serializer = getSerializationSchema(rowType, schema); - AvroRowDataDeserializationSchema deserializer = getDeserializationSchema(rowType, schema); + Schema writeSchema = AvroSchemaConverter.convertToSchema(dataType.getLogicalType()); + AvroRowDataDeserializationSchema deserializer = + getDeserializationSchema(rowType, writeSchema); serializer.open(null); deserializer.open(null); From 11dcb83c541e92d3c8c2ab3251063f1babcac158 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 27 Oct 2020 10:48:14 +0800 Subject: [PATCH 101/322] [FLINK-19786][avro] Fix the nullability and precision for Avro format deserialization * Fix the TIME schema precision as 3 * Fix the nullability of type: TIMESTAMP_WITHOUT_TIME_ZONE, DATE, TIME_WITHOUT_TIME_ZONE, DECIMAL, MAP, ARRAY * The table schema row type should be always non-nullable --- .../registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java index ce3d630b5..69378f040 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java @@ -139,7 +139,8 @@ private void testRowDataWriteReadWithSchema(Schema schema) throws Exception { RowType rowType = (RowType) dataType.getLogicalType(); AvroRowDataSerializationSchema serializer = getSerializationSchema(rowType, schema); - Schema writeSchema = AvroSchemaConverter.convertToSchema(dataType.getLogicalType()); + Schema writeSchema = AvroSchemaConverter + .convertToSchema(dataType.getLogicalType()); AvroRowDataDeserializationSchema deserializer = getDeserializationSchema(rowType, writeSchema); From 7fbc65dd501c3e99c75faf15ac37abf77ab42b65 Mon Sep 17 00:00:00 2001 From: fangliang <568693125@qq.com> Date: Wed, 4 Nov 2020 10:13:53 +0800 Subject: [PATCH 102/322] [FLINK-19873][canal-json] Skip DDL change events for Canal data This closes #13872 --- .../formats/json/canal/CanalJsonDeserializationSchema.java | 5 +++++ .../flink-json-debezium/src/test/resources/canal-data.txt | 3 ++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index 20292fc25..fe9227a30 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -58,6 +58,7 @@ public final class CanalJsonDeserializationSchema implements DeserializationSche private static final String OP_INSERT = "INSERT"; private static final String OP_UPDATE = "UPDATE"; private static final String OP_DELETE = "DELETE"; + private static final String OP_CREATE = "CREATE"; /** The deserializer to deserialize Debezium JSON data. */ private final JsonRowDataDeserializationSchema jsonDeserializer; @@ -223,6 +224,10 @@ public void deserialize(byte[] message, Collector out) throws IOExcepti insert.setRowKind(RowKind.DELETE); out.collect(insert); } + } else if (OP_CREATE.equals(type)){ + // "data" field is null and "type" is "CREATE" which means + // this is a DDL change event, and we should skip it. + return; } else { if (!ignoreParseErrors) { throw new IOException(format( diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt index e81fad25c..530468aa4 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt @@ -7,4 +7,5 @@ {"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"inventory","es":1589373560000,"id":9,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"5.18"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373560798,"type":"UPDATE"} {"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"inventory","es":1589373563000,"id":10,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373563798,"type":"DELETE"} {"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"5.17"},{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"}],"database":"inventory","es":1589373753000,"id":11,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"3.14"},{"weight":"8.1"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373753939,"type":"UPDATE"} -{"data":[{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"}],"database":"inventory","es":1589374013000,"id":12,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589374013680,"type":"DELETE"} \ No newline at end of file +{"data":null,"database":"inventory","es":1589373566000,"id":13,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE TABLE `xj_`.`user02` (`uid` int(0) NOT NULL,`uname` varchar(255) NULL, PRIMARY KEY (`uid`))","sqlType":null,"table":"user02","ts":1589373566000,"type":"CREATE"} +{"data":[{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"}],"database":"inventory","es":1589374013000,"id":12,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589374013680,"type":"DELETE"} From 26e9b1800a185c074df596992c40371613b8562c Mon Sep 17 00:00:00 2001 From: Benchao Li Date: Fri, 23 Oct 2020 23:03:44 +0800 Subject: [PATCH 103/322] [FLINK-19790][json] Clear reused ObjectNode's content for map converter in RowDataToJsonConverters This closes #13777 --- .../formats/json/RowDataToJsonConverters.java | 1 + .../json/JsonRowDataSerDeSchemaTest.java | 23 ++++++++++++++++++- 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java index e7530f14c..132efc5ef 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java @@ -242,6 +242,7 @@ private RowDataToJsonConverter createMapConverter( node = mapper.createObjectNode(); } else { node = (ObjectNode) reuse; + node.removeAll(); } MapData map = (MapData) object; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index e2a476208..bba9cc1d2 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -246,7 +246,12 @@ public void testSerDeMultiRows() throws Exception { RowType rowType = (RowType) ROW( FIELD("f1", INT()), FIELD("f2", BOOLEAN()), - FIELD("f3", STRING()) + FIELD("f3", STRING()), + FIELD("f4", MAP(STRING(), STRING())), + FIELD("f5", ARRAY(STRING())), + FIELD("f6", ROW( + FIELD("f1", STRING()), + FIELD("f2", INT()))) ).getLogicalType(); JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( @@ -261,6 +266,14 @@ public void testSerDeMultiRows() throws Exception { root.put("f1", 1); root.put("f2", true); root.put("f3", "str"); + ObjectNode map = root.putObject("f4"); + map.put("hello1", "flink"); + ArrayNode array = root.putArray("f5"); + array.add("element1"); + array.add("element2"); + ObjectNode row = root.putObject("f6"); + row.put("f1", "this is row1"); + row.put("f2", 12); byte[] serializedJson = objectMapper.writeValueAsBytes(root); RowData rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); @@ -273,6 +286,14 @@ public void testSerDeMultiRows() throws Exception { root.put("f1", 10); root.put("f2", false); root.put("f3", "newStr"); + ObjectNode map = root.putObject("f4"); + map.put("hello2", "json"); + ArrayNode array = root.putArray("f5"); + array.add("element3"); + array.add("element4"); + ObjectNode row = root.putObject("f6"); + row.put("f1", "this is row2"); + row.putNull("f2"); byte[] serializedJson = objectMapper.writeValueAsBytes(root); RowData rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); From 9b8b7b4c012485ba9e4f058e3b172cae82ef43f8 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Fri, 30 Oct 2020 10:52:40 +0100 Subject: [PATCH 104/322] [FLINK-19849] Fix NOTICE files for 1.12 release This closes #13796 --- .../src/main/resources/META-INF/NOTICE | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE index a5f4b4f3c..84f48af14 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -6,10 +6,9 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.101tec:zkclient:0.10 - com.fasterxml.jackson.core:jackson-databind:2.10.1 - com.fasterxml.jackson.core:jackson-annotations:2.10.1 - com.fasterxml.jackson.core:jackson-core:2.10.1 - io.confluent:common-utils:5.4.2 - io.confluent:kafka-schema-registry-client:5.4.2 -- org.apache.zookeeper:zookeeper:3.4.10 +- io.confluent:common-config:5.4.2 From 719b390a65509e5d91b5b2c4b72ad121af3b2179 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 6 Nov 2020 13:32:28 +0100 Subject: [PATCH 105/322] [FLINK-18546] Upgrade to Kafka Schema Registry Client 5.5.2 The commit changes the packaging of flink-avro-confluent-registry module. We do no longer ship a fat-jar. Instead users should build the fat-jar themselves. It is because the upgraded version of registry client accesses fields from Avro of Jackson types. Because of the fact we can either bundle and shade both Avro and Jackson, or not shade Jackson. I decided it will be best not to ship a fat-jar and leave that decision to users. On the other hand for the sql-client it is better to provide fat-jars, however because of 1. I added a separate flink-sql-avro-confluent-registry module that builds a fat-jar (including Avro, and shaded jackson). --- .../pom.xml | 1 - .../flink-avro-confluent-registry/pom.xml | 75 +-- .../src/main/resources/META-INF/NOTICE | 14 - .../flink-sql-avro-confluent-registry/pom.xml | 118 ++++ .../src/main/resources/META-INF/NOTICE | 25 + .../licenses/LICENSE.jakarta-annotation-api | 637 ++++++++++++++++++ .../META-INF/licenses/LICENSE.jakarta-inject | 637 ++++++++++++++++++ .../licenses/LICENSE.jakarta-ws-rs-api | 637 ++++++++++++++++++ .../licenses/LICENSE.osgi-resource-locator | 637 ++++++++++++++++++ 9 files changed, 2705 insertions(+), 76 deletions(-) delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE create mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml create mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE create mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-annotation-api create mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-inject create mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-ws-rs-api create mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.osgi-resource-locator diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 4110f4538..1b43ecd96 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -31,7 +31,6 @@ under the License. UTF-8 - 4.1.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 235e45190..3dac36deb 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -33,7 +33,7 @@ under the License. 2.4.1 - 5.4.2 + 5.5.2 @@ -84,12 +84,12 @@ under the License. ${project.version} provided - - org.apache.flink - flink-table-common - ${project.version} - provided - + + org.apache.flink + flink-table-common + ${project.version} + provided + @@ -99,12 +99,12 @@ under the License. test test-jar - - org.apache.flink - flink-table-runtime-blink_${scala.binary.version} - ${project.version} - test - + + org.apache.flink + flink-table-runtime-blink_${scala.binary.version} + ${project.version} + test + org.apache.flink flink-table-common @@ -119,53 +119,6 @@ under the License. test test-jar - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - false - - - io.confluent:* - com.fasterxml.jackson.core:* - org.apache.zookeeper:zookeeper - com.101tec:zkclient - - - - - com.fasterxml.jackson - org.apache.flink.formats.avro.registry.confluent.shaded.com.fasterxml.jackson - - - org.apache.zookeeper - org.apache.flink.formats.avro.registry.confluent.shaded.org.apache.zookeeper - - - org.apache.jute - org.apache.flink.formats.avro.registry.confluent.shaded.org.apache.jute - - - org.I0Itec.zkclient - org.apache.flink.formats.avro.registry.confluent.shaded.org.101tec - - - - - - - - + diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE deleted file mode 100644 index 84f48af14..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ /dev/null @@ -1,14 +0,0 @@ -flink-avro-confluent-registry -Copyright 2014-2020 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - -- com.fasterxml.jackson.core:jackson-databind:2.10.1 -- com.fasterxml.jackson.core:jackson-annotations:2.10.1 -- com.fasterxml.jackson.core:jackson-core:2.10.1 -- io.confluent:common-utils:5.4.2 -- io.confluent:kafka-schema-registry-client:5.4.2 -- io.confluent:common-config:5.4.2 diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml new file mode 100644 index 000000000..aac58240f --- /dev/null +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -0,0 +1,118 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-formats + 1.12-SNAPSHOT + .. + + + flink-sql-avro-confluent-registry + Flink : Formats : SQL Avro Confluent Registry + + jar + + + + org.apache.flink + flink-avro-confluent-registry + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + + + io.confluent:* + + org.apache.kafka:kafka-clients + org.apache.flink:flink-avro + org.apache.flink:flink-avro-confluent-registry + org.apache.avro:avro + com.fasterxml.jackson.core:* + jakarta.ws.rs:jakarta.ws.rs-api + jakarta.annotation:jakarta.annotation-api + org.glassfish.jersey.core:jersey-common + org.glassfish.hk2.external:jakarta.inject + org.glassfish.hk2:osgi-resource-locator + org.apache.commons:commons-compress + + + + + com.fasterxml.jackson + org.apache.flink.formats.avro.registry.confluent.shaded.com.fasterxml.jackson + + + org.apache.commons.compress + org.apache.flink.formats.avro.registry.confluent.shaded.org.apache.commons.compress + + + org.apache.kafka + org.apache.flink.formats.avro.registry.confluent.shaded.org.apache.kafka + + + + + org.glassfish.jersey.core:jersey-common + + + META-INF/versions/** + + + + org.apache.kafka:* + + kafka/kafka-version.properties + LICENSE + + NOTICE + common/** + + + + + + + + + + diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE new file mode 100644 index 000000000..0d9003214 --- /dev/null +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -0,0 +1,25 @@ +flink-sql-avro-confluent-registry +Copyright 2014-2020 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.avro:avro:1.10.0 +- com.fasterxml.jackson.core:jackson-core:2.10.1 +- com.fasterxml.jackson.core:jackson-databind:2.10.1 +- com.fasterxml.jackson.core:jackson-annotations:2.10.1 +- org.apache.commons:commons-compress:1.20 +- io.confluent:kafka-schema-registry-client:5.5.2 +- org.apache.kafka:kafka-clients:5.5.2-ccs +- io.confluent:common-config:5.5.2 +- io.confluent:common-utils:5.5.2 +- org.glassfish.jersey.core:jersey-common:2.30 + +The binary distribution of this product bundles these dependencies under the Eclipse Public License - v 2.0 (https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt) + +- jakarta.annotation:jakarta.annotation-api:1.3.5 +- jakarta.ws.rs:jakarta.ws.rs-api:2.1.6 +- org.glassfish.hk2.external:jakarta.inject:2.6.1 +- org.glassfish.hk2:osgi-resource-locator:1.0.3 diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-annotation-api b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-annotation-api new file mode 100644 index 000000000..5de3d1b40 --- /dev/null +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-annotation-api @@ -0,0 +1,637 @@ +# Eclipse Public License - v 2.0 + + THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE + PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION + OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + + 1. DEFINITIONS + + "Contribution" means: + + a) in the case of the initial Contributor, the initial content + Distributed under this Agreement, and + + b) in the case of each subsequent Contributor: + i) changes to the Program, and + ii) additions to the Program; + where such changes and/or additions to the Program originate from + and are Distributed by that particular Contributor. A Contribution + "originates" from a Contributor if it was added to the Program by + such Contributor itself or anyone acting on such Contributor's behalf. + Contributions do not include changes or additions to the Program that + are not Modified Works. + + "Contributor" means any person or entity that Distributes the Program. + + "Licensed Patents" mean patent claims licensable by a Contributor which + are necessarily infringed by the use or sale of its Contribution alone + or when combined with the Program. + + "Program" means the Contributions Distributed in accordance with this + Agreement. + + "Recipient" means anyone who receives the Program under this Agreement + or any Secondary License (as applicable), including Contributors. + + "Derivative Works" shall mean any work, whether in Source Code or other + form, that is based on (or derived from) the Program and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. + + "Modified Works" shall mean any work in Source Code or other form that + results from an addition to, deletion from, or modification of the + contents of the Program, including, for purposes of clarity any new file + in Source Code form that contains any contents of the Program. Modified + Works shall not include works that contain only declarations, + interfaces, types, classes, structures, or files of the Program solely + in each case in order to link to, bind by name, or subclass the Program + or Modified Works thereof. + + "Distribute" means the acts of a) distributing or b) making available + in any manner that enables the transfer of a copy. + + "Source Code" means the form of a Program preferred for making + modifications, including but not limited to software source code, + documentation source, and configuration files. + + "Secondary License" means either the GNU General Public License, + Version 2.0, or any later versions of that license, including any + exceptions or additional permissions as identified by the initial + Contributor. + + 2. GRANT OF RIGHTS + + a) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free copyright + license to reproduce, prepare Derivative Works of, publicly display, + publicly perform, Distribute and sublicense the Contribution of such + Contributor, if any, and such Derivative Works. + + b) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free patent + license under Licensed Patents to make, use, sell, offer to sell, + import and otherwise transfer the Contribution of such Contributor, + if any, in Source Code or other form. This patent license shall + apply to the combination of the Contribution and the Program if, at + the time the Contribution is added by the Contributor, such addition + of the Contribution causes such combination to be covered by the + Licensed Patents. The patent license shall not apply to any other + combinations which include the Contribution. No hardware per se is + licensed hereunder. + + c) Recipient understands that although each Contributor grants the + licenses to its Contributions set forth herein, no assurances are + provided by any Contributor that the Program does not infringe the + patent or other intellectual property rights of any other entity. + Each Contributor disclaims any liability to Recipient for claims + brought by any other entity based on infringement of intellectual + property rights or otherwise. As a condition to exercising the + rights and licenses granted hereunder, each Recipient hereby + assumes sole responsibility to secure any other intellectual + property rights needed, if any. For example, if a third party + patent license is required to allow Recipient to Distribute the + Program, it is Recipient's responsibility to acquire that license + before distributing the Program. + + d) Each Contributor represents that to its knowledge it has + sufficient copyright rights in its Contribution, if any, to grant + the copyright license set forth in this Agreement. + + e) Notwithstanding the terms of any Secondary License, no + Contributor makes additional grants to any Recipient (other than + those set forth in this Agreement) as a result of such Recipient's + receipt of the Program under the terms of a Secondary License + (if permitted under the terms of Section 3). + + 3. REQUIREMENTS + + 3.1 If a Contributor Distributes the Program in any form, then: + + a) the Program must also be made available as Source Code, in + accordance with section 3.2, and the Contributor must accompany + the Program with a statement that the Source Code for the Program + is available under this Agreement, and informs Recipients how to + obtain it in a reasonable manner on or through a medium customarily + used for software exchange; and + + b) the Contributor may Distribute the Program under a license + different than this Agreement, provided that such license: + i) effectively disclaims on behalf of all other Contributors all + warranties and conditions, express and implied, including + warranties or conditions of title and non-infringement, and + implied warranties or conditions of merchantability and fitness + for a particular purpose; + + ii) effectively excludes on behalf of all other Contributors all + liability for damages, including direct, indirect, special, + incidental and consequential damages, such as lost profits; + + iii) does not attempt to limit or alter the recipients' rights + in the Source Code under section 3.2; and + + iv) requires any subsequent distribution of the Program by any + party to be under a license that satisfies the requirements + of this section 3. + + 3.2 When the Program is Distributed as Source Code: + + a) it must be made available under this Agreement, or if the + Program (i) is combined with other material in a separate file or + files made available under a Secondary License, and (ii) the initial + Contributor attached to the Source Code the notice described in + Exhibit A of this Agreement, then the Program may be made available + under the terms of such Secondary Licenses, and + + b) a copy of this Agreement must be included with each copy of + the Program. + + 3.3 Contributors may not remove or alter any copyright, patent, + trademark, attribution notices, disclaimers of warranty, or limitations + of liability ("notices") contained within the Program from any copy of + the Program which they Distribute, provided that Contributors may add + their own appropriate notices. + + 4. COMMERCIAL DISTRIBUTION + + Commercial distributors of software may accept certain responsibilities + with respect to end users, business partners and the like. While this + license is intended to facilitate the commercial use of the Program, + the Contributor who includes the Program in a commercial product + offering should do so in a manner which does not create potential + liability for other Contributors. Therefore, if a Contributor includes + the Program in a commercial product offering, such Contributor + ("Commercial Contributor") hereby agrees to defend and indemnify every + other Contributor ("Indemnified Contributor") against any losses, + damages and costs (collectively "Losses") arising from claims, lawsuits + and other legal actions brought by a third party against the Indemnified + Contributor to the extent caused by the acts or omissions of such + Commercial Contributor in connection with its distribution of the Program + in a commercial product offering. The obligations in this section do not + apply to any claims or Losses relating to any actual or alleged + intellectual property infringement. In order to qualify, an Indemnified + Contributor must: a) promptly notify the Commercial Contributor in + writing of such claim, and b) allow the Commercial Contributor to control, + and cooperate with the Commercial Contributor in, the defense and any + related settlement negotiations. The Indemnified Contributor may + participate in any such claim at its own expense. + + For example, a Contributor might include the Program in a commercial + product offering, Product X. That Contributor is then a Commercial + Contributor. If that Commercial Contributor then makes performance + claims, or offers warranties related to Product X, those performance + claims and warranties are such Commercial Contributor's responsibility + alone. Under this section, the Commercial Contributor would have to + defend claims against the other Contributors related to those performance + claims and warranties, and if a court requires any other Contributor to + pay any damages as a result, the Commercial Contributor must pay + those damages. + + 5. NO WARRANTY + + EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT + PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED ON AN "AS IS" + BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR + IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF + TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR + PURPOSE. Each Recipient is solely responsible for determining the + appropriateness of using and distributing the Program and assumes all + risks associated with its exercise of rights under this Agreement, + including but not limited to the risks and costs of program errors, + compliance with applicable laws, damage to or loss of data, programs + or equipment, and unavailability or interruption of operations. + + 6. DISCLAIMER OF LIABILITY + + EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT + PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS + SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, + EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST + PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE + EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE + POSSIBILITY OF SUCH DAMAGES. + + 7. GENERAL + + If any provision of this Agreement is invalid or unenforceable under + applicable law, it shall not affect the validity or enforceability of + the remainder of the terms of this Agreement, and without further + action by the parties hereto, such provision shall be reformed to the + minimum extent necessary to make such provision valid and enforceable. + + If Recipient institutes patent litigation against any entity + (including a cross-claim or counterclaim in a lawsuit) alleging that the + Program itself (excluding combinations of the Program with other software + or hardware) infringes such Recipient's patent(s), then such Recipient's + rights granted under Section 2(b) shall terminate as of the date such + litigation is filed. + + All Recipient's rights under this Agreement shall terminate if it + fails to comply with any of the material terms or conditions of this + Agreement and does not cure such failure in a reasonable period of + time after becoming aware of such noncompliance. If all Recipient's + rights under this Agreement terminate, Recipient agrees to cease use + and distribution of the Program as soon as reasonably practicable. + However, Recipient's obligations under this Agreement and any licenses + granted by Recipient relating to the Program shall continue and survive. + + Everyone is permitted to copy and distribute copies of this Agreement, + but in order to avoid inconsistency the Agreement is copyrighted and + may only be modified in the following manner. The Agreement Steward + reserves the right to publish new versions (including revisions) of + this Agreement from time to time. No one other than the Agreement + Steward has the right to modify this Agreement. The Eclipse Foundation + is the initial Agreement Steward. The Eclipse Foundation may assign the + responsibility to serve as the Agreement Steward to a suitable separate + entity. Each new version of the Agreement will be given a distinguishing + version number. The Program (including Contributions) may always be + Distributed subject to the version of the Agreement under which it was + received. In addition, after a new version of the Agreement is published, + Contributor may elect to Distribute the Program (including its + Contributions) under the new version. + + Except as expressly stated in Sections 2(a) and 2(b) above, Recipient + receives no rights or licenses to the intellectual property of any + Contributor under this Agreement, whether expressly, by implication, + estoppel or otherwise. All rights in the Program not expressly granted + under this Agreement are reserved. Nothing in this Agreement is intended + to be enforceable by any entity that is not a Contributor or Recipient. + No third-party beneficiary rights are created under this Agreement. + + Exhibit A - Form of Secondary Licenses Notice + + "This Source Code may also be made available under the following + Secondary Licenses when the conditions for such availability set forth + in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), + version(s), and exceptions or additional permissions here}." + + Simply including a copy of this Agreement, including this Exhibit A + is not sufficient to license the Source Code under Secondary Licenses. + + If it is not possible or desirable to put the notice in a particular + file, then You may include the notice in a location (such as a LICENSE + file in a relevant directory) where a recipient would be likely to + look for such a notice. + + You may add additional accurate notices of copyright ownership. + +--- + +## The GNU General Public License (GPL) Version 2, June 1991 + + Copyright (C) 1989, 1991 Free Software Foundation, Inc. + 51 Franklin Street, Fifth Floor + Boston, MA 02110-1335 + USA + + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + + Preamble + + The licenses for most software are designed to take away your freedom to + share and change it. By contrast, the GNU General Public License is + intended to guarantee your freedom to share and change free software--to + make sure the software is free for all its users. This General Public + License applies to most of the Free Software Foundation's software and + to any other program whose authors commit to using it. (Some other Free + Software Foundation software is covered by the GNU Library General + Public License instead.) You can apply it to your programs, too. + + When we speak of free software, we are referring to freedom, not price. + Our General Public Licenses are designed to make sure that you have the + freedom to distribute copies of free software (and charge for this + service if you wish), that you receive source code or can get it if you + want it, that you can change the software or use pieces of it in new + free programs; and that you know you can do these things. + + To protect your rights, we need to make restrictions that forbid anyone + to deny you these rights or to ask you to surrender the rights. These + restrictions translate to certain responsibilities for you if you + distribute copies of the software, or if you modify it. + + For example, if you distribute copies of such a program, whether gratis + or for a fee, you must give the recipients all the rights that you have. + You must make sure that they, too, receive or can get the source code. + And you must show them these terms so they know their rights. + + We protect your rights with two steps: (1) copyright the software, and + (2) offer you this license which gives you legal permission to copy, + distribute and/or modify the software. + + Also, for each author's protection and ours, we want to make certain + that everyone understands that there is no warranty for this free + software. If the software is modified by someone else and passed on, we + want its recipients to know that what they have is not the original, so + that any problems introduced by others will not reflect on the original + authors' reputations. + + Finally, any free program is threatened constantly by software patents. + We wish to avoid the danger that redistributors of a free program will + individually obtain patent licenses, in effect making the program + proprietary. To prevent this, we have made it clear that any patent must + be licensed for everyone's free use or not licensed at all. + + The precise terms and conditions for copying, distribution and + modification follow. + + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License applies to any program or other work which contains a + notice placed by the copyright holder saying it may be distributed under + the terms of this General Public License. The "Program", below, refers + to any such program or work, and a "work based on the Program" means + either the Program or any derivative work under copyright law: that is + to say, a work containing the Program or a portion of it, either + verbatim or with modifications and/or translated into another language. + (Hereinafter, translation is included without limitation in the term + "modification".) Each licensee is addressed as "you". + + Activities other than copying, distribution and modification are not + covered by this License; they are outside its scope. The act of running + the Program is not restricted, and the output from the Program is + covered only if its contents constitute a work based on the Program + (independent of having been made by running the Program). Whether that + is true depends on what the Program does. + + 1. You may copy and distribute verbatim copies of the Program's source + code as you receive it, in any medium, provided that you conspicuously + and appropriately publish on each copy an appropriate copyright notice + and disclaimer of warranty; keep intact all the notices that refer to + this License and to the absence of any warranty; and give any other + recipients of the Program a copy of this License along with the Program. + + You may charge a fee for the physical act of transferring a copy, and + you may at your option offer warranty protection in exchange for a fee. + + 2. You may modify your copy or copies of the Program or any portion of + it, thus forming a work based on the Program, and copy and distribute + such modifications or work under the terms of Section 1 above, provided + that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any part + thereof, to be licensed as a whole at no charge to all third parties + under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a notice + that there is no warranty (or else, saying that you provide a + warranty) and that users may redistribute the program under these + conditions, and telling the user how to view a copy of this License. + (Exception: if the Program itself is interactive but does not + normally print such an announcement, your work based on the Program + is not required to print an announcement.) + + These requirements apply to the modified work as a whole. If + identifiable sections of that work are not derived from the Program, and + can be reasonably considered independent and separate works in + themselves, then this License, and its terms, do not apply to those + sections when you distribute them as separate works. But when you + distribute the same sections as part of a whole which is a work based on + the Program, the distribution of the whole must be on the terms of this + License, whose permissions for other licensees extend to the entire + whole, and thus to each and every part regardless of who wrote it. + + Thus, it is not the intent of this section to claim rights or contest + your rights to work written entirely by you; rather, the intent is to + exercise the right to control the distribution of derivative or + collective works based on the Program. + + In addition, mere aggregation of another work not based on the Program + with the Program (or with a work based on the Program) on a volume of a + storage or distribution medium does not bring the other work under the + scope of this License. + + 3. You may copy and distribute the Program (or a work based on it, + under Section 2) in object code or executable form under the terms of + Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections 1 + and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your cost + of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer to + distribute corresponding source code. (This alternative is allowed + only for noncommercial distribution and only if you received the + program in object code or executable form with such an offer, in + accord with Subsection b above.) + + The source code for a work means the preferred form of the work for + making modifications to it. For an executable work, complete source code + means all the source code for all modules it contains, plus any + associated interface definition files, plus the scripts used to control + compilation and installation of the executable. However, as a special + exception, the source code distributed need not include anything that is + normally distributed (in either source or binary form) with the major + components (compiler, kernel, and so on) of the operating system on + which the executable runs, unless that component itself accompanies the + executable. + + If distribution of executable or object code is made by offering access + to copy from a designated place, then offering equivalent access to copy + the source code from the same place counts as distribution of the source + code, even though third parties are not compelled to copy the source + along with the object code. + + 4. You may not copy, modify, sublicense, or distribute the Program + except as expressly provided under this License. Any attempt otherwise + to copy, modify, sublicense or distribute the Program is void, and will + automatically terminate your rights under this License. However, parties + who have received copies, or rights, from you under this License will + not have their licenses terminated so long as such parties remain in + full compliance. + + 5. You are not required to accept this License, since you have not + signed it. However, nothing else grants you permission to modify or + distribute the Program or its derivative works. These actions are + prohibited by law if you do not accept this License. Therefore, by + modifying or distributing the Program (or any work based on the + Program), you indicate your acceptance of this License to do so, and all + its terms and conditions for copying, distributing or modifying the + Program or works based on it. + + 6. Each time you redistribute the Program (or any work based on the + Program), the recipient automatically receives a license from the + original licensor to copy, distribute or modify the Program subject to + these terms and conditions. You may not impose any further restrictions + on the recipients' exercise of the rights granted herein. You are not + responsible for enforcing compliance by third parties to this License. + + 7. If, as a consequence of a court judgment or allegation of patent + infringement or for any other reason (not limited to patent issues), + conditions are imposed on you (whether by court order, agreement or + otherwise) that contradict the conditions of this License, they do not + excuse you from the conditions of this License. If you cannot distribute + so as to satisfy simultaneously your obligations under this License and + any other pertinent obligations, then as a consequence you may not + distribute the Program at all. For example, if a patent license would + not permit royalty-free redistribution of the Program by all those who + receive copies directly or indirectly through you, then the only way you + could satisfy both it and this License would be to refrain entirely from + distribution of the Program. + + If any portion of this section is held invalid or unenforceable under + any particular circumstance, the balance of the section is intended to + apply and the section as a whole is intended to apply in other + circumstances. + + It is not the purpose of this section to induce you to infringe any + patents or other property right claims or to contest validity of any + such claims; this section has the sole purpose of protecting the + integrity of the free software distribution system, which is implemented + by public license practices. Many people have made generous + contributions to the wide range of software distributed through that + system in reliance on consistent application of that system; it is up to + the author/donor to decide if he or she is willing to distribute + software through any other system and a licensee cannot impose that choice. + + This section is intended to make thoroughly clear what is believed to be + a consequence of the rest of this License. + + 8. If the distribution and/or use of the Program is restricted in + certain countries either by patents or by copyrighted interfaces, the + original copyright holder who places the Program under this License may + add an explicit geographical distribution limitation excluding those + countries, so that distribution is permitted only in or among countries + not thus excluded. In such case, this License incorporates the + limitation as if written in the body of this License. + + 9. The Free Software Foundation may publish revised and/or new + versions of the General Public License from time to time. Such new + versions will be similar in spirit to the present version, but may + differ in detail to address new problems or concerns. + + Each version is given a distinguishing version number. If the Program + specifies a version number of this License which applies to it and "any + later version", you have the option of following the terms and + conditions either of that version or of any later version published by + the Free Software Foundation. If the Program does not specify a version + number of this License, you may choose any version ever published by the + Free Software Foundation. + + 10. If you wish to incorporate parts of the Program into other free + programs whose distribution conditions are different, write to the + author to ask for permission. For software which is copyrighted by the + Free Software Foundation, write to the Free Software Foundation; we + sometimes make exceptions for this. Our decision will be guided by the + two goals of preserving the free status of all derivatives of our free + software and of promoting the sharing and reuse of software generally. + + NO WARRANTY + + 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO + WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. + EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR + OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, + EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE + ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH + YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL + NECESSARY SERVICING, REPAIR OR CORRECTION. + + 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN + WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY + AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR + DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL + DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM + (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED + INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF + THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR + OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Programs + + If you develop a new program, and you want it to be of the greatest + possible use to the public, the best way to achieve this is to make it + free software which everyone can redistribute and change under these terms. + + To do so, attach the following notices to the program. It is safest to + attach them to the start of each source file to most effectively convey + the exclusion of warranty; and each file should have at least the + "copyright" line and a pointer to where the full notice is found. + + One line to give the program's name and a brief idea of what it does. + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, but + WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + General Public License for more details. + + You should have received a copy of the GNU General Public License + along with this program; if not, write to the Free Software + Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA + + Also add information on how to contact you by electronic and paper mail. + + If the program is interactive, make it output a short notice like this + when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type + `show w'. This is free software, and you are welcome to redistribute + it under certain conditions; type `show c' for details. + + The hypothetical commands `show w' and `show c' should show the + appropriate parts of the General Public License. Of course, the commands + you use may be called something other than `show w' and `show c'; they + could even be mouse-clicks or menu items--whatever suits your program. + + You should also get your employer (if you work as a programmer) or your + school, if any, to sign a "copyright disclaimer" for the program, if + necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the + program `Gnomovision' (which makes passes at compilers) written by + James Hacker. + + signature of Ty Coon, 1 April 1989 + Ty Coon, President of Vice + + This General Public License does not permit incorporating your program + into proprietary programs. If your program is a subroutine library, you + may consider it more useful to permit linking proprietary applications + with the library. If this is what you want to do, use the GNU Library + General Public License instead of this License. + +--- + +## CLASSPATH EXCEPTION + + Linking this library statically or dynamically with other modules is + making a combined work based on this library. Thus, the terms and + conditions of the GNU General Public License version 2 cover the whole + combination. + + As a special exception, the copyright holders of this library give you + permission to link this library with independent modules to produce an + executable, regardless of the license terms of these independent + modules, and to copy and distribute the resulting executable under + terms of your choice, provided that you also meet, for each linked + independent module, the terms and conditions of the license of that + module. An independent module is a module which is not derived from or + based on this library. If you modify this library, you may extend this + exception to your version of the library, but you are not obligated to + do so. If you do not wish to do so, delete this exception statement + from your version. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-inject b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-inject new file mode 100644 index 000000000..6667b0612 --- /dev/null +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-inject @@ -0,0 +1,637 @@ +# Eclipse Public License - v 2.0 + + THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE + PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION + OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + + 1. DEFINITIONS + + "Contribution" means: + + a) in the case of the initial Contributor, the initial content + Distributed under this Agreement, and + + b) in the case of each subsequent Contributor: + i) changes to the Program, and + ii) additions to the Program; + where such changes and/or additions to the Program originate from + and are Distributed by that particular Contributor. A Contribution + "originates" from a Contributor if it was added to the Program by + such Contributor itself or anyone acting on such Contributor's behalf. + Contributions do not include changes or additions to the Program that + are not Modified Works. + + "Contributor" means any person or entity that Distributes the Program. + + "Licensed Patents" mean patent claims licensable by a Contributor which + are necessarily infringed by the use or sale of its Contribution alone + or when combined with the Program. + + "Program" means the Contributions Distributed in accordance with this + Agreement. + + "Recipient" means anyone who receives the Program under this Agreement + or any Secondary License (as applicable), including Contributors. + + "Derivative Works" shall mean any work, whether in Source Code or other + form, that is based on (or derived from) the Program and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. + + "Modified Works" shall mean any work in Source Code or other form that + results from an addition to, deletion from, or modification of the + contents of the Program, including, for purposes of clarity any new file + in Source Code form that contains any contents of the Program. Modified + Works shall not include works that contain only declarations, + interfaces, types, classes, structures, or files of the Program solely + in each case in order to link to, bind by name, or subclass the Program + or Modified Works thereof. + + "Distribute" means the acts of a) distributing or b) making available + in any manner that enables the transfer of a copy. + + "Source Code" means the form of a Program preferred for making + modifications, including but not limited to software source code, + documentation source, and configuration files. + + "Secondary License" means either the GNU General Public License, + Version 2.0, or any later versions of that license, including any + exceptions or additional permissions as identified by the initial + Contributor. + + 2. GRANT OF RIGHTS + + a) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free copyright + license to reproduce, prepare Derivative Works of, publicly display, + publicly perform, Distribute and sublicense the Contribution of such + Contributor, if any, and such Derivative Works. + + b) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free patent + license under Licensed Patents to make, use, sell, offer to sell, + import and otherwise transfer the Contribution of such Contributor, + if any, in Source Code or other form. This patent license shall + apply to the combination of the Contribution and the Program if, at + the time the Contribution is added by the Contributor, such addition + of the Contribution causes such combination to be covered by the + Licensed Patents. The patent license shall not apply to any other + combinations which include the Contribution. No hardware per se is + licensed hereunder. + + c) Recipient understands that although each Contributor grants the + licenses to its Contributions set forth herein, no assurances are + provided by any Contributor that the Program does not infringe the + patent or other intellectual property rights of any other entity. + Each Contributor disclaims any liability to Recipient for claims + brought by any other entity based on infringement of intellectual + property rights or otherwise. As a condition to exercising the + rights and licenses granted hereunder, each Recipient hereby + assumes sole responsibility to secure any other intellectual + property rights needed, if any. For example, if a third party + patent license is required to allow Recipient to Distribute the + Program, it is Recipient's responsibility to acquire that license + before distributing the Program. + + d) Each Contributor represents that to its knowledge it has + sufficient copyright rights in its Contribution, if any, to grant + the copyright license set forth in this Agreement. + + e) Notwithstanding the terms of any Secondary License, no + Contributor makes additional grants to any Recipient (other than + those set forth in this Agreement) as a result of such Recipient's + receipt of the Program under the terms of a Secondary License + (if permitted under the terms of Section 3). + + 3. REQUIREMENTS + + 3.1 If a Contributor Distributes the Program in any form, then: + + a) the Program must also be made available as Source Code, in + accordance with section 3.2, and the Contributor must accompany + the Program with a statement that the Source Code for the Program + is available under this Agreement, and informs Recipients how to + obtain it in a reasonable manner on or through a medium customarily + used for software exchange; and + + b) the Contributor may Distribute the Program under a license + different than this Agreement, provided that such license: + i) effectively disclaims on behalf of all other Contributors all + warranties and conditions, express and implied, including + warranties or conditions of title and non-infringement, and + implied warranties or conditions of merchantability and fitness + for a particular purpose; + + ii) effectively excludes on behalf of all other Contributors all + liability for damages, including direct, indirect, special, + incidental and consequential damages, such as lost profits; + + iii) does not attempt to limit or alter the recipients' rights + in the Source Code under section 3.2; and + + iv) requires any subsequent distribution of the Program by any + party to be under a license that satisfies the requirements + of this section 3. + + 3.2 When the Program is Distributed as Source Code: + + a) it must be made available under this Agreement, or if the + Program (i) is combined with other material in a separate file or + files made available under a Secondary License, and (ii) the initial + Contributor attached to the Source Code the notice described in + Exhibit A of this Agreement, then the Program may be made available + under the terms of such Secondary Licenses, and + + b) a copy of this Agreement must be included with each copy of + the Program. + + 3.3 Contributors may not remove or alter any copyright, patent, + trademark, attribution notices, disclaimers of warranty, or limitations + of liability ("notices") contained within the Program from any copy of + the Program which they Distribute, provided that Contributors may add + their own appropriate notices. + + 4. COMMERCIAL DISTRIBUTION + + Commercial distributors of software may accept certain responsibilities + with respect to end users, business partners and the like. While this + license is intended to facilitate the commercial use of the Program, + the Contributor who includes the Program in a commercial product + offering should do so in a manner which does not create potential + liability for other Contributors. Therefore, if a Contributor includes + the Program in a commercial product offering, such Contributor + ("Commercial Contributor") hereby agrees to defend and indemnify every + other Contributor ("Indemnified Contributor") against any losses, + damages and costs (collectively "Losses") arising from claims, lawsuits + and other legal actions brought by a third party against the Indemnified + Contributor to the extent caused by the acts or omissions of such + Commercial Contributor in connection with its distribution of the Program + in a commercial product offering. The obligations in this section do not + apply to any claims or Losses relating to any actual or alleged + intellectual property infringement. In order to qualify, an Indemnified + Contributor must: a) promptly notify the Commercial Contributor in + writing of such claim, and b) allow the Commercial Contributor to control, + and cooperate with the Commercial Contributor in, the defense and any + related settlement negotiations. The Indemnified Contributor may + participate in any such claim at its own expense. + + For example, a Contributor might include the Program in a commercial + product offering, Product X. That Contributor is then a Commercial + Contributor. If that Commercial Contributor then makes performance + claims, or offers warranties related to Product X, those performance + claims and warranties are such Commercial Contributor's responsibility + alone. Under this section, the Commercial Contributor would have to + defend claims against the other Contributors related to those performance + claims and warranties, and if a court requires any other Contributor to + pay any damages as a result, the Commercial Contributor must pay + those damages. + + 5. NO WARRANTY + + EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT + PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED ON AN "AS IS" + BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR + IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF + TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR + PURPOSE. Each Recipient is solely responsible for determining the + appropriateness of using and distributing the Program and assumes all + risks associated with its exercise of rights under this Agreement, + including but not limited to the risks and costs of program errors, + compliance with applicable laws, damage to or loss of data, programs + or equipment, and unavailability or interruption of operations. + + 6. DISCLAIMER OF LIABILITY + + EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT + PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS + SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, + EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST + PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE + EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE + POSSIBILITY OF SUCH DAMAGES. + + 7. GENERAL + + If any provision of this Agreement is invalid or unenforceable under + applicable law, it shall not affect the validity or enforceability of + the remainder of the terms of this Agreement, and without further + action by the parties hereto, such provision shall be reformed to the + minimum extent necessary to make such provision valid and enforceable. + + If Recipient institutes patent litigation against any entity + (including a cross-claim or counterclaim in a lawsuit) alleging that the + Program itself (excluding combinations of the Program with other software + or hardware) infringes such Recipient's patent(s), then such Recipient's + rights granted under Section 2(b) shall terminate as of the date such + litigation is filed. + + All Recipient's rights under this Agreement shall terminate if it + fails to comply with any of the material terms or conditions of this + Agreement and does not cure such failure in a reasonable period of + time after becoming aware of such noncompliance. If all Recipient's + rights under this Agreement terminate, Recipient agrees to cease use + and distribution of the Program as soon as reasonably practicable. + However, Recipient's obligations under this Agreement and any licenses + granted by Recipient relating to the Program shall continue and survive. + + Everyone is permitted to copy and distribute copies of this Agreement, + but in order to avoid inconsistency the Agreement is copyrighted and + may only be modified in the following manner. The Agreement Steward + reserves the right to publish new versions (including revisions) of + this Agreement from time to time. No one other than the Agreement + Steward has the right to modify this Agreement. The Eclipse Foundation + is the initial Agreement Steward. The Eclipse Foundation may assign the + responsibility to serve as the Agreement Steward to a suitable separate + entity. Each new version of the Agreement will be given a distinguishing + version number. The Program (including Contributions) may always be + Distributed subject to the version of the Agreement under which it was + received. In addition, after a new version of the Agreement is published, + Contributor may elect to Distribute the Program (including its + Contributions) under the new version. + + Except as expressly stated in Sections 2(a) and 2(b) above, Recipient + receives no rights or licenses to the intellectual property of any + Contributor under this Agreement, whether expressly, by implication, + estoppel or otherwise. All rights in the Program not expressly granted + under this Agreement are reserved. Nothing in this Agreement is intended + to be enforceable by any entity that is not a Contributor or Recipient. + No third-party beneficiary rights are created under this Agreement. + + Exhibit A - Form of Secondary Licenses Notice + + "This Source Code may also be made available under the following + Secondary Licenses when the conditions for such availability set forth + in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), + version(s), and exceptions or additional permissions here}." + + Simply including a copy of this Agreement, including this Exhibit A + is not sufficient to license the Source Code under Secondary Licenses. + + If it is not possible or desirable to put the notice in a particular + file, then You may include the notice in a location (such as a LICENSE + file in a relevant directory) where a recipient would be likely to + look for such a notice. + + You may add additional accurate notices of copyright ownership. + +--- + +## The GNU General Public License (GPL) Version 2, June 1991 + + Copyright (C) 1989, 1991 Free Software Foundation, Inc. + 51 Franklin Street, Fifth Floor + Boston, MA 02110-1335 + USA + + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + + Preamble + + The licenses for most software are designed to take away your freedom to + share and change it. By contrast, the GNU General Public License is + intended to guarantee your freedom to share and change free software--to + make sure the software is free for all its users. This General Public + License applies to most of the Free Software Foundation's software and + to any other program whose authors commit to using it. (Some other Free + Software Foundation software is covered by the GNU Library General + Public License instead.) You can apply it to your programs, too. + + When we speak of free software, we are referring to freedom, not price. + Our General Public Licenses are designed to make sure that you have the + freedom to distribute copies of free software (and charge for this + service if you wish), that you receive source code or can get it if you + want it, that you can change the software or use pieces of it in new + free programs; and that you know you can do these things. + + To protect your rights, we need to make restrictions that forbid anyone + to deny you these rights or to ask you to surrender the rights. These + restrictions translate to certain responsibilities for you if you + distribute copies of the software, or if you modify it. + + For example, if you distribute copies of such a program, whether gratis + or for a fee, you must give the recipients all the rights that you have. + You must make sure that they, too, receive or can get the source code. + And you must show them these terms so they know their rights. + + We protect your rights with two steps: (1) copyright the software, and + (2) offer you this license which gives you legal permission to copy, + distribute and/or modify the software. + + Also, for each author's protection and ours, we want to make certain + that everyone understands that there is no warranty for this free + software. If the software is modified by someone else and passed on, we + want its recipients to know that what they have is not the original, so + that any problems introduced by others will not reflect on the original + authors' reputations. + + Finally, any free program is threatened constantly by software patents. + We wish to avoid the danger that redistributors of a free program will + individually obtain patent licenses, in effect making the program + proprietary. To prevent this, we have made it clear that any patent must + be licensed for everyone's free use or not licensed at all. + + The precise terms and conditions for copying, distribution and + modification follow. + + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License applies to any program or other work which contains a + notice placed by the copyright holder saying it may be distributed under + the terms of this General Public License. The "Program", below, refers + to any such program or work, and a "work based on the Program" means + either the Program or any derivative work under copyright law: that is + to say, a work containing the Program or a portion of it, either + verbatim or with modifications and/or translated into another language. + (Hereinafter, translation is included without limitation in the term + "modification".) Each licensee is addressed as "you". + + Activities other than copying, distribution and modification are not + covered by this License; they are outside its scope. The act of running + the Program is not restricted, and the output from the Program is + covered only if its contents constitute a work based on the Program + (independent of having been made by running the Program). Whether that + is true depends on what the Program does. + + 1. You may copy and distribute verbatim copies of the Program's source + code as you receive it, in any medium, provided that you conspicuously + and appropriately publish on each copy an appropriate copyright notice + and disclaimer of warranty; keep intact all the notices that refer to + this License and to the absence of any warranty; and give any other + recipients of the Program a copy of this License along with the Program. + + You may charge a fee for the physical act of transferring a copy, and + you may at your option offer warranty protection in exchange for a fee. + + 2. You may modify your copy or copies of the Program or any portion of + it, thus forming a work based on the Program, and copy and distribute + such modifications or work under the terms of Section 1 above, provided + that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any part + thereof, to be licensed as a whole at no charge to all third parties + under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a notice + that there is no warranty (or else, saying that you provide a + warranty) and that users may redistribute the program under these + conditions, and telling the user how to view a copy of this License. + (Exception: if the Program itself is interactive but does not + normally print such an announcement, your work based on the Program + is not required to print an announcement.) + + These requirements apply to the modified work as a whole. If + identifiable sections of that work are not derived from the Program, and + can be reasonably considered independent and separate works in + themselves, then this License, and its terms, do not apply to those + sections when you distribute them as separate works. But when you + distribute the same sections as part of a whole which is a work based on + the Program, the distribution of the whole must be on the terms of this + License, whose permissions for other licensees extend to the entire + whole, and thus to each and every part regardless of who wrote it. + + Thus, it is not the intent of this section to claim rights or contest + your rights to work written entirely by you; rather, the intent is to + exercise the right to control the distribution of derivative or + collective works based on the Program. + + In addition, mere aggregation of another work not based on the Program + with the Program (or with a work based on the Program) on a volume of a + storage or distribution medium does not bring the other work under the + scope of this License. + + 3. You may copy and distribute the Program (or a work based on it, + under Section 2) in object code or executable form under the terms of + Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections 1 + and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your cost + of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer to + distribute corresponding source code. (This alternative is allowed + only for noncommercial distribution and only if you received the + program in object code or executable form with such an offer, in + accord with Subsection b above.) + + The source code for a work means the preferred form of the work for + making modifications to it. For an executable work, complete source code + means all the source code for all modules it contains, plus any + associated interface definition files, plus the scripts used to control + compilation and installation of the executable. However, as a special + exception, the source code distributed need not include anything that is + normally distributed (in either source or binary form) with the major + components (compiler, kernel, and so on) of the operating system on + which the executable runs, unless that component itself accompanies the + executable. + + If distribution of executable or object code is made by offering access + to copy from a designated place, then offering equivalent access to copy + the source code from the same place counts as distribution of the source + code, even though third parties are not compelled to copy the source + along with the object code. + + 4. You may not copy, modify, sublicense, or distribute the Program + except as expressly provided under this License. Any attempt otherwise + to copy, modify, sublicense or distribute the Program is void, and will + automatically terminate your rights under this License. However, parties + who have received copies, or rights, from you under this License will + not have their licenses terminated so long as such parties remain in + full compliance. + + 5. You are not required to accept this License, since you have not + signed it. However, nothing else grants you permission to modify or + distribute the Program or its derivative works. These actions are + prohibited by law if you do not accept this License. Therefore, by + modifying or distributing the Program (or any work based on the + Program), you indicate your acceptance of this License to do so, and all + its terms and conditions for copying, distributing or modifying the + Program or works based on it. + + 6. Each time you redistribute the Program (or any work based on the + Program), the recipient automatically receives a license from the + original licensor to copy, distribute or modify the Program subject to + these terms and conditions. You may not impose any further restrictions + on the recipients' exercise of the rights granted herein. You are not + responsible for enforcing compliance by third parties to this License. + + 7. If, as a consequence of a court judgment or allegation of patent + infringement or for any other reason (not limited to patent issues), + conditions are imposed on you (whether by court order, agreement or + otherwise) that contradict the conditions of this License, they do not + excuse you from the conditions of this License. If you cannot distribute + so as to satisfy simultaneously your obligations under this License and + any other pertinent obligations, then as a consequence you may not + distribute the Program at all. For example, if a patent license would + not permit royalty-free redistribution of the Program by all those who + receive copies directly or indirectly through you, then the only way you + could satisfy both it and this License would be to refrain entirely from + distribution of the Program. + + If any portion of this section is held invalid or unenforceable under + any particular circumstance, the balance of the section is intended to + apply and the section as a whole is intended to apply in other + circumstances. + + It is not the purpose of this section to induce you to infringe any + patents or other property right claims or to contest validity of any + such claims; this section has the sole purpose of protecting the + integrity of the free software distribution system, which is implemented + by public license practices. Many people have made generous + contributions to the wide range of software distributed through that + system in reliance on consistent application of that system; it is up to + the author/donor to decide if he or she is willing to distribute + software through any other system and a licensee cannot impose that choice. + + This section is intended to make thoroughly clear what is believed to be + a consequence of the rest of this License. + + 8. If the distribution and/or use of the Program is restricted in + certain countries either by patents or by copyrighted interfaces, the + original copyright holder who places the Program under this License may + add an explicit geographical distribution limitation excluding those + countries, so that distribution is permitted only in or among countries + not thus excluded. In such case, this License incorporates the + limitation as if written in the body of this License. + + 9. The Free Software Foundation may publish revised and/or new + versions of the General Public License from time to time. Such new + versions will be similar in spirit to the present version, but may + differ in detail to address new problems or concerns. + + Each version is given a distinguishing version number. If the Program + specifies a version number of this License which applies to it and "any + later version", you have the option of following the terms and + conditions either of that version or of any later version published by + the Free Software Foundation. If the Program does not specify a version + number of this License, you may choose any version ever published by the + Free Software Foundation. + + 10. If you wish to incorporate parts of the Program into other free + programs whose distribution conditions are different, write to the + author to ask for permission. For software which is copyrighted by the + Free Software Foundation, write to the Free Software Foundation; we + sometimes make exceptions for this. Our decision will be guided by the + two goals of preserving the free status of all derivatives of our free + software and of promoting the sharing and reuse of software generally. + + NO WARRANTY + + 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO + WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. + EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR + OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, + EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE + ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH + YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL + NECESSARY SERVICING, REPAIR OR CORRECTION. + + 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN + WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY + AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR + DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL + DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM + (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED + INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF + THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR + OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Programs + + If you develop a new program, and you want it to be of the greatest + possible use to the public, the best way to achieve this is to make it + free software which everyone can redistribute and change under these terms. + + To do so, attach the following notices to the program. It is safest to + attach them to the start of each source file to most effectively convey + the exclusion of warranty; and each file should have at least the + "copyright" line and a pointer to where the full notice is found. + + One line to give the program's name and a brief idea of what it does. + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, but + WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + General Public License for more details. + + You should have received a copy of the GNU General Public License + along with this program; if not, write to the Free Software + Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA + + Also add information on how to contact you by electronic and paper mail. + + If the program is interactive, make it output a short notice like this + when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type + `show w'. This is free software, and you are welcome to redistribute + it under certain conditions; type `show c' for details. + + The hypothetical commands `show w' and `show c' should show the + appropriate parts of the General Public License. Of course, the commands + you use may be called something other than `show w' and `show c'; they + could even be mouse-clicks or menu items--whatever suits your program. + + You should also get your employer (if you work as a programmer) or your + school, if any, to sign a "copyright disclaimer" for the program, if + necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the + program `Gnomovision' (which makes passes at compilers) written by + James Hacker. + + signature of Ty Coon, 1 April 1989 + Ty Coon, President of Vice + + This General Public License does not permit incorporating your program + into proprietary programs. If your program is a subroutine library, you + may consider it more useful to permit linking proprietary applications + with the library. If this is what you want to do, use the GNU Library + General Public License instead of this License. + +--- + +## CLASSPATH EXCEPTION + + Linking this library statically or dynamically with other modules is + making a combined work based on this library. Thus, the terms and + conditions of the GNU General Public License version 2 cover the whole + combination. + + As a special exception, the copyright holders of this library give you + permission to link this library with independent modules to produce an + executable, regardless of the license terms of these independent + modules, and to copy and distribute the resulting executable under + terms of your choice, provided that you also meet, for each linked + independent module, the terms and conditions of the license of that + module. An independent module is a module which is not derived from or + based on this library. If you modify this library, you may extend this + exception to your version of the library, but you are not obligated to + do so. If you do not wish to do so, delete this exception statement + from your version. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-ws-rs-api b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-ws-rs-api new file mode 100644 index 000000000..5de3d1b40 --- /dev/null +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-ws-rs-api @@ -0,0 +1,637 @@ +# Eclipse Public License - v 2.0 + + THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE + PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION + OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + + 1. DEFINITIONS + + "Contribution" means: + + a) in the case of the initial Contributor, the initial content + Distributed under this Agreement, and + + b) in the case of each subsequent Contributor: + i) changes to the Program, and + ii) additions to the Program; + where such changes and/or additions to the Program originate from + and are Distributed by that particular Contributor. A Contribution + "originates" from a Contributor if it was added to the Program by + such Contributor itself or anyone acting on such Contributor's behalf. + Contributions do not include changes or additions to the Program that + are not Modified Works. + + "Contributor" means any person or entity that Distributes the Program. + + "Licensed Patents" mean patent claims licensable by a Contributor which + are necessarily infringed by the use or sale of its Contribution alone + or when combined with the Program. + + "Program" means the Contributions Distributed in accordance with this + Agreement. + + "Recipient" means anyone who receives the Program under this Agreement + or any Secondary License (as applicable), including Contributors. + + "Derivative Works" shall mean any work, whether in Source Code or other + form, that is based on (or derived from) the Program and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. + + "Modified Works" shall mean any work in Source Code or other form that + results from an addition to, deletion from, or modification of the + contents of the Program, including, for purposes of clarity any new file + in Source Code form that contains any contents of the Program. Modified + Works shall not include works that contain only declarations, + interfaces, types, classes, structures, or files of the Program solely + in each case in order to link to, bind by name, or subclass the Program + or Modified Works thereof. + + "Distribute" means the acts of a) distributing or b) making available + in any manner that enables the transfer of a copy. + + "Source Code" means the form of a Program preferred for making + modifications, including but not limited to software source code, + documentation source, and configuration files. + + "Secondary License" means either the GNU General Public License, + Version 2.0, or any later versions of that license, including any + exceptions or additional permissions as identified by the initial + Contributor. + + 2. GRANT OF RIGHTS + + a) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free copyright + license to reproduce, prepare Derivative Works of, publicly display, + publicly perform, Distribute and sublicense the Contribution of such + Contributor, if any, and such Derivative Works. + + b) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free patent + license under Licensed Patents to make, use, sell, offer to sell, + import and otherwise transfer the Contribution of such Contributor, + if any, in Source Code or other form. This patent license shall + apply to the combination of the Contribution and the Program if, at + the time the Contribution is added by the Contributor, such addition + of the Contribution causes such combination to be covered by the + Licensed Patents. The patent license shall not apply to any other + combinations which include the Contribution. No hardware per se is + licensed hereunder. + + c) Recipient understands that although each Contributor grants the + licenses to its Contributions set forth herein, no assurances are + provided by any Contributor that the Program does not infringe the + patent or other intellectual property rights of any other entity. + Each Contributor disclaims any liability to Recipient for claims + brought by any other entity based on infringement of intellectual + property rights or otherwise. As a condition to exercising the + rights and licenses granted hereunder, each Recipient hereby + assumes sole responsibility to secure any other intellectual + property rights needed, if any. For example, if a third party + patent license is required to allow Recipient to Distribute the + Program, it is Recipient's responsibility to acquire that license + before distributing the Program. + + d) Each Contributor represents that to its knowledge it has + sufficient copyright rights in its Contribution, if any, to grant + the copyright license set forth in this Agreement. + + e) Notwithstanding the terms of any Secondary License, no + Contributor makes additional grants to any Recipient (other than + those set forth in this Agreement) as a result of such Recipient's + receipt of the Program under the terms of a Secondary License + (if permitted under the terms of Section 3). + + 3. REQUIREMENTS + + 3.1 If a Contributor Distributes the Program in any form, then: + + a) the Program must also be made available as Source Code, in + accordance with section 3.2, and the Contributor must accompany + the Program with a statement that the Source Code for the Program + is available under this Agreement, and informs Recipients how to + obtain it in a reasonable manner on or through a medium customarily + used for software exchange; and + + b) the Contributor may Distribute the Program under a license + different than this Agreement, provided that such license: + i) effectively disclaims on behalf of all other Contributors all + warranties and conditions, express and implied, including + warranties or conditions of title and non-infringement, and + implied warranties or conditions of merchantability and fitness + for a particular purpose; + + ii) effectively excludes on behalf of all other Contributors all + liability for damages, including direct, indirect, special, + incidental and consequential damages, such as lost profits; + + iii) does not attempt to limit or alter the recipients' rights + in the Source Code under section 3.2; and + + iv) requires any subsequent distribution of the Program by any + party to be under a license that satisfies the requirements + of this section 3. + + 3.2 When the Program is Distributed as Source Code: + + a) it must be made available under this Agreement, or if the + Program (i) is combined with other material in a separate file or + files made available under a Secondary License, and (ii) the initial + Contributor attached to the Source Code the notice described in + Exhibit A of this Agreement, then the Program may be made available + under the terms of such Secondary Licenses, and + + b) a copy of this Agreement must be included with each copy of + the Program. + + 3.3 Contributors may not remove or alter any copyright, patent, + trademark, attribution notices, disclaimers of warranty, or limitations + of liability ("notices") contained within the Program from any copy of + the Program which they Distribute, provided that Contributors may add + their own appropriate notices. + + 4. COMMERCIAL DISTRIBUTION + + Commercial distributors of software may accept certain responsibilities + with respect to end users, business partners and the like. While this + license is intended to facilitate the commercial use of the Program, + the Contributor who includes the Program in a commercial product + offering should do so in a manner which does not create potential + liability for other Contributors. Therefore, if a Contributor includes + the Program in a commercial product offering, such Contributor + ("Commercial Contributor") hereby agrees to defend and indemnify every + other Contributor ("Indemnified Contributor") against any losses, + damages and costs (collectively "Losses") arising from claims, lawsuits + and other legal actions brought by a third party against the Indemnified + Contributor to the extent caused by the acts or omissions of such + Commercial Contributor in connection with its distribution of the Program + in a commercial product offering. The obligations in this section do not + apply to any claims or Losses relating to any actual or alleged + intellectual property infringement. In order to qualify, an Indemnified + Contributor must: a) promptly notify the Commercial Contributor in + writing of such claim, and b) allow the Commercial Contributor to control, + and cooperate with the Commercial Contributor in, the defense and any + related settlement negotiations. The Indemnified Contributor may + participate in any such claim at its own expense. + + For example, a Contributor might include the Program in a commercial + product offering, Product X. That Contributor is then a Commercial + Contributor. If that Commercial Contributor then makes performance + claims, or offers warranties related to Product X, those performance + claims and warranties are such Commercial Contributor's responsibility + alone. Under this section, the Commercial Contributor would have to + defend claims against the other Contributors related to those performance + claims and warranties, and if a court requires any other Contributor to + pay any damages as a result, the Commercial Contributor must pay + those damages. + + 5. NO WARRANTY + + EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT + PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED ON AN "AS IS" + BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR + IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF + TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR + PURPOSE. Each Recipient is solely responsible for determining the + appropriateness of using and distributing the Program and assumes all + risks associated with its exercise of rights under this Agreement, + including but not limited to the risks and costs of program errors, + compliance with applicable laws, damage to or loss of data, programs + or equipment, and unavailability or interruption of operations. + + 6. DISCLAIMER OF LIABILITY + + EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT + PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS + SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, + EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST + PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE + EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE + POSSIBILITY OF SUCH DAMAGES. + + 7. GENERAL + + If any provision of this Agreement is invalid or unenforceable under + applicable law, it shall not affect the validity or enforceability of + the remainder of the terms of this Agreement, and without further + action by the parties hereto, such provision shall be reformed to the + minimum extent necessary to make such provision valid and enforceable. + + If Recipient institutes patent litigation against any entity + (including a cross-claim or counterclaim in a lawsuit) alleging that the + Program itself (excluding combinations of the Program with other software + or hardware) infringes such Recipient's patent(s), then such Recipient's + rights granted under Section 2(b) shall terminate as of the date such + litigation is filed. + + All Recipient's rights under this Agreement shall terminate if it + fails to comply with any of the material terms or conditions of this + Agreement and does not cure such failure in a reasonable period of + time after becoming aware of such noncompliance. If all Recipient's + rights under this Agreement terminate, Recipient agrees to cease use + and distribution of the Program as soon as reasonably practicable. + However, Recipient's obligations under this Agreement and any licenses + granted by Recipient relating to the Program shall continue and survive. + + Everyone is permitted to copy and distribute copies of this Agreement, + but in order to avoid inconsistency the Agreement is copyrighted and + may only be modified in the following manner. The Agreement Steward + reserves the right to publish new versions (including revisions) of + this Agreement from time to time. No one other than the Agreement + Steward has the right to modify this Agreement. The Eclipse Foundation + is the initial Agreement Steward. The Eclipse Foundation may assign the + responsibility to serve as the Agreement Steward to a suitable separate + entity. Each new version of the Agreement will be given a distinguishing + version number. The Program (including Contributions) may always be + Distributed subject to the version of the Agreement under which it was + received. In addition, after a new version of the Agreement is published, + Contributor may elect to Distribute the Program (including its + Contributions) under the new version. + + Except as expressly stated in Sections 2(a) and 2(b) above, Recipient + receives no rights or licenses to the intellectual property of any + Contributor under this Agreement, whether expressly, by implication, + estoppel or otherwise. All rights in the Program not expressly granted + under this Agreement are reserved. Nothing in this Agreement is intended + to be enforceable by any entity that is not a Contributor or Recipient. + No third-party beneficiary rights are created under this Agreement. + + Exhibit A - Form of Secondary Licenses Notice + + "This Source Code may also be made available under the following + Secondary Licenses when the conditions for such availability set forth + in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), + version(s), and exceptions or additional permissions here}." + + Simply including a copy of this Agreement, including this Exhibit A + is not sufficient to license the Source Code under Secondary Licenses. + + If it is not possible or desirable to put the notice in a particular + file, then You may include the notice in a location (such as a LICENSE + file in a relevant directory) where a recipient would be likely to + look for such a notice. + + You may add additional accurate notices of copyright ownership. + +--- + +## The GNU General Public License (GPL) Version 2, June 1991 + + Copyright (C) 1989, 1991 Free Software Foundation, Inc. + 51 Franklin Street, Fifth Floor + Boston, MA 02110-1335 + USA + + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + + Preamble + + The licenses for most software are designed to take away your freedom to + share and change it. By contrast, the GNU General Public License is + intended to guarantee your freedom to share and change free software--to + make sure the software is free for all its users. This General Public + License applies to most of the Free Software Foundation's software and + to any other program whose authors commit to using it. (Some other Free + Software Foundation software is covered by the GNU Library General + Public License instead.) You can apply it to your programs, too. + + When we speak of free software, we are referring to freedom, not price. + Our General Public Licenses are designed to make sure that you have the + freedom to distribute copies of free software (and charge for this + service if you wish), that you receive source code or can get it if you + want it, that you can change the software or use pieces of it in new + free programs; and that you know you can do these things. + + To protect your rights, we need to make restrictions that forbid anyone + to deny you these rights or to ask you to surrender the rights. These + restrictions translate to certain responsibilities for you if you + distribute copies of the software, or if you modify it. + + For example, if you distribute copies of such a program, whether gratis + or for a fee, you must give the recipients all the rights that you have. + You must make sure that they, too, receive or can get the source code. + And you must show them these terms so they know their rights. + + We protect your rights with two steps: (1) copyright the software, and + (2) offer you this license which gives you legal permission to copy, + distribute and/or modify the software. + + Also, for each author's protection and ours, we want to make certain + that everyone understands that there is no warranty for this free + software. If the software is modified by someone else and passed on, we + want its recipients to know that what they have is not the original, so + that any problems introduced by others will not reflect on the original + authors' reputations. + + Finally, any free program is threatened constantly by software patents. + We wish to avoid the danger that redistributors of a free program will + individually obtain patent licenses, in effect making the program + proprietary. To prevent this, we have made it clear that any patent must + be licensed for everyone's free use or not licensed at all. + + The precise terms and conditions for copying, distribution and + modification follow. + + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License applies to any program or other work which contains a + notice placed by the copyright holder saying it may be distributed under + the terms of this General Public License. The "Program", below, refers + to any such program or work, and a "work based on the Program" means + either the Program or any derivative work under copyright law: that is + to say, a work containing the Program or a portion of it, either + verbatim or with modifications and/or translated into another language. + (Hereinafter, translation is included without limitation in the term + "modification".) Each licensee is addressed as "you". + + Activities other than copying, distribution and modification are not + covered by this License; they are outside its scope. The act of running + the Program is not restricted, and the output from the Program is + covered only if its contents constitute a work based on the Program + (independent of having been made by running the Program). Whether that + is true depends on what the Program does. + + 1. You may copy and distribute verbatim copies of the Program's source + code as you receive it, in any medium, provided that you conspicuously + and appropriately publish on each copy an appropriate copyright notice + and disclaimer of warranty; keep intact all the notices that refer to + this License and to the absence of any warranty; and give any other + recipients of the Program a copy of this License along with the Program. + + You may charge a fee for the physical act of transferring a copy, and + you may at your option offer warranty protection in exchange for a fee. + + 2. You may modify your copy or copies of the Program or any portion of + it, thus forming a work based on the Program, and copy and distribute + such modifications or work under the terms of Section 1 above, provided + that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any part + thereof, to be licensed as a whole at no charge to all third parties + under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a notice + that there is no warranty (or else, saying that you provide a + warranty) and that users may redistribute the program under these + conditions, and telling the user how to view a copy of this License. + (Exception: if the Program itself is interactive but does not + normally print such an announcement, your work based on the Program + is not required to print an announcement.) + + These requirements apply to the modified work as a whole. If + identifiable sections of that work are not derived from the Program, and + can be reasonably considered independent and separate works in + themselves, then this License, and its terms, do not apply to those + sections when you distribute them as separate works. But when you + distribute the same sections as part of a whole which is a work based on + the Program, the distribution of the whole must be on the terms of this + License, whose permissions for other licensees extend to the entire + whole, and thus to each and every part regardless of who wrote it. + + Thus, it is not the intent of this section to claim rights or contest + your rights to work written entirely by you; rather, the intent is to + exercise the right to control the distribution of derivative or + collective works based on the Program. + + In addition, mere aggregation of another work not based on the Program + with the Program (or with a work based on the Program) on a volume of a + storage or distribution medium does not bring the other work under the + scope of this License. + + 3. You may copy and distribute the Program (or a work based on it, + under Section 2) in object code or executable form under the terms of + Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections 1 + and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your cost + of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer to + distribute corresponding source code. (This alternative is allowed + only for noncommercial distribution and only if you received the + program in object code or executable form with such an offer, in + accord with Subsection b above.) + + The source code for a work means the preferred form of the work for + making modifications to it. For an executable work, complete source code + means all the source code for all modules it contains, plus any + associated interface definition files, plus the scripts used to control + compilation and installation of the executable. However, as a special + exception, the source code distributed need not include anything that is + normally distributed (in either source or binary form) with the major + components (compiler, kernel, and so on) of the operating system on + which the executable runs, unless that component itself accompanies the + executable. + + If distribution of executable or object code is made by offering access + to copy from a designated place, then offering equivalent access to copy + the source code from the same place counts as distribution of the source + code, even though third parties are not compelled to copy the source + along with the object code. + + 4. You may not copy, modify, sublicense, or distribute the Program + except as expressly provided under this License. Any attempt otherwise + to copy, modify, sublicense or distribute the Program is void, and will + automatically terminate your rights under this License. However, parties + who have received copies, or rights, from you under this License will + not have their licenses terminated so long as such parties remain in + full compliance. + + 5. You are not required to accept this License, since you have not + signed it. However, nothing else grants you permission to modify or + distribute the Program or its derivative works. These actions are + prohibited by law if you do not accept this License. Therefore, by + modifying or distributing the Program (or any work based on the + Program), you indicate your acceptance of this License to do so, and all + its terms and conditions for copying, distributing or modifying the + Program or works based on it. + + 6. Each time you redistribute the Program (or any work based on the + Program), the recipient automatically receives a license from the + original licensor to copy, distribute or modify the Program subject to + these terms and conditions. You may not impose any further restrictions + on the recipients' exercise of the rights granted herein. You are not + responsible for enforcing compliance by third parties to this License. + + 7. If, as a consequence of a court judgment or allegation of patent + infringement or for any other reason (not limited to patent issues), + conditions are imposed on you (whether by court order, agreement or + otherwise) that contradict the conditions of this License, they do not + excuse you from the conditions of this License. If you cannot distribute + so as to satisfy simultaneously your obligations under this License and + any other pertinent obligations, then as a consequence you may not + distribute the Program at all. For example, if a patent license would + not permit royalty-free redistribution of the Program by all those who + receive copies directly or indirectly through you, then the only way you + could satisfy both it and this License would be to refrain entirely from + distribution of the Program. + + If any portion of this section is held invalid or unenforceable under + any particular circumstance, the balance of the section is intended to + apply and the section as a whole is intended to apply in other + circumstances. + + It is not the purpose of this section to induce you to infringe any + patents or other property right claims or to contest validity of any + such claims; this section has the sole purpose of protecting the + integrity of the free software distribution system, which is implemented + by public license practices. Many people have made generous + contributions to the wide range of software distributed through that + system in reliance on consistent application of that system; it is up to + the author/donor to decide if he or she is willing to distribute + software through any other system and a licensee cannot impose that choice. + + This section is intended to make thoroughly clear what is believed to be + a consequence of the rest of this License. + + 8. If the distribution and/or use of the Program is restricted in + certain countries either by patents or by copyrighted interfaces, the + original copyright holder who places the Program under this License may + add an explicit geographical distribution limitation excluding those + countries, so that distribution is permitted only in or among countries + not thus excluded. In such case, this License incorporates the + limitation as if written in the body of this License. + + 9. The Free Software Foundation may publish revised and/or new + versions of the General Public License from time to time. Such new + versions will be similar in spirit to the present version, but may + differ in detail to address new problems or concerns. + + Each version is given a distinguishing version number. If the Program + specifies a version number of this License which applies to it and "any + later version", you have the option of following the terms and + conditions either of that version or of any later version published by + the Free Software Foundation. If the Program does not specify a version + number of this License, you may choose any version ever published by the + Free Software Foundation. + + 10. If you wish to incorporate parts of the Program into other free + programs whose distribution conditions are different, write to the + author to ask for permission. For software which is copyrighted by the + Free Software Foundation, write to the Free Software Foundation; we + sometimes make exceptions for this. Our decision will be guided by the + two goals of preserving the free status of all derivatives of our free + software and of promoting the sharing and reuse of software generally. + + NO WARRANTY + + 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO + WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. + EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR + OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, + EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE + ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH + YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL + NECESSARY SERVICING, REPAIR OR CORRECTION. + + 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN + WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY + AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR + DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL + DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM + (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED + INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF + THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR + OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Programs + + If you develop a new program, and you want it to be of the greatest + possible use to the public, the best way to achieve this is to make it + free software which everyone can redistribute and change under these terms. + + To do so, attach the following notices to the program. It is safest to + attach them to the start of each source file to most effectively convey + the exclusion of warranty; and each file should have at least the + "copyright" line and a pointer to where the full notice is found. + + One line to give the program's name and a brief idea of what it does. + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, but + WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + General Public License for more details. + + You should have received a copy of the GNU General Public License + along with this program; if not, write to the Free Software + Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA + + Also add information on how to contact you by electronic and paper mail. + + If the program is interactive, make it output a short notice like this + when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type + `show w'. This is free software, and you are welcome to redistribute + it under certain conditions; type `show c' for details. + + The hypothetical commands `show w' and `show c' should show the + appropriate parts of the General Public License. Of course, the commands + you use may be called something other than `show w' and `show c'; they + could even be mouse-clicks or menu items--whatever suits your program. + + You should also get your employer (if you work as a programmer) or your + school, if any, to sign a "copyright disclaimer" for the program, if + necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the + program `Gnomovision' (which makes passes at compilers) written by + James Hacker. + + signature of Ty Coon, 1 April 1989 + Ty Coon, President of Vice + + This General Public License does not permit incorporating your program + into proprietary programs. If your program is a subroutine library, you + may consider it more useful to permit linking proprietary applications + with the library. If this is what you want to do, use the GNU Library + General Public License instead of this License. + +--- + +## CLASSPATH EXCEPTION + + Linking this library statically or dynamically with other modules is + making a combined work based on this library. Thus, the terms and + conditions of the GNU General Public License version 2 cover the whole + combination. + + As a special exception, the copyright holders of this library give you + permission to link this library with independent modules to produce an + executable, regardless of the license terms of these independent + modules, and to copy and distribute the resulting executable under + terms of your choice, provided that you also meet, for each linked + independent module, the terms and conditions of the license of that + module. An independent module is a module which is not derived from or + based on this library. If you modify this library, you may extend this + exception to your version of the library, but you are not obligated to + do so. If you do not wish to do so, delete this exception statement + from your version. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.osgi-resource-locator b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.osgi-resource-locator new file mode 100644 index 000000000..5de3d1b40 --- /dev/null +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.osgi-resource-locator @@ -0,0 +1,637 @@ +# Eclipse Public License - v 2.0 + + THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE + PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION + OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + + 1. DEFINITIONS + + "Contribution" means: + + a) in the case of the initial Contributor, the initial content + Distributed under this Agreement, and + + b) in the case of each subsequent Contributor: + i) changes to the Program, and + ii) additions to the Program; + where such changes and/or additions to the Program originate from + and are Distributed by that particular Contributor. A Contribution + "originates" from a Contributor if it was added to the Program by + such Contributor itself or anyone acting on such Contributor's behalf. + Contributions do not include changes or additions to the Program that + are not Modified Works. + + "Contributor" means any person or entity that Distributes the Program. + + "Licensed Patents" mean patent claims licensable by a Contributor which + are necessarily infringed by the use or sale of its Contribution alone + or when combined with the Program. + + "Program" means the Contributions Distributed in accordance with this + Agreement. + + "Recipient" means anyone who receives the Program under this Agreement + or any Secondary License (as applicable), including Contributors. + + "Derivative Works" shall mean any work, whether in Source Code or other + form, that is based on (or derived from) the Program and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. + + "Modified Works" shall mean any work in Source Code or other form that + results from an addition to, deletion from, or modification of the + contents of the Program, including, for purposes of clarity any new file + in Source Code form that contains any contents of the Program. Modified + Works shall not include works that contain only declarations, + interfaces, types, classes, structures, or files of the Program solely + in each case in order to link to, bind by name, or subclass the Program + or Modified Works thereof. + + "Distribute" means the acts of a) distributing or b) making available + in any manner that enables the transfer of a copy. + + "Source Code" means the form of a Program preferred for making + modifications, including but not limited to software source code, + documentation source, and configuration files. + + "Secondary License" means either the GNU General Public License, + Version 2.0, or any later versions of that license, including any + exceptions or additional permissions as identified by the initial + Contributor. + + 2. GRANT OF RIGHTS + + a) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free copyright + license to reproduce, prepare Derivative Works of, publicly display, + publicly perform, Distribute and sublicense the Contribution of such + Contributor, if any, and such Derivative Works. + + b) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free patent + license under Licensed Patents to make, use, sell, offer to sell, + import and otherwise transfer the Contribution of such Contributor, + if any, in Source Code or other form. This patent license shall + apply to the combination of the Contribution and the Program if, at + the time the Contribution is added by the Contributor, such addition + of the Contribution causes such combination to be covered by the + Licensed Patents. The patent license shall not apply to any other + combinations which include the Contribution. No hardware per se is + licensed hereunder. + + c) Recipient understands that although each Contributor grants the + licenses to its Contributions set forth herein, no assurances are + provided by any Contributor that the Program does not infringe the + patent or other intellectual property rights of any other entity. + Each Contributor disclaims any liability to Recipient for claims + brought by any other entity based on infringement of intellectual + property rights or otherwise. As a condition to exercising the + rights and licenses granted hereunder, each Recipient hereby + assumes sole responsibility to secure any other intellectual + property rights needed, if any. For example, if a third party + patent license is required to allow Recipient to Distribute the + Program, it is Recipient's responsibility to acquire that license + before distributing the Program. + + d) Each Contributor represents that to its knowledge it has + sufficient copyright rights in its Contribution, if any, to grant + the copyright license set forth in this Agreement. + + e) Notwithstanding the terms of any Secondary License, no + Contributor makes additional grants to any Recipient (other than + those set forth in this Agreement) as a result of such Recipient's + receipt of the Program under the terms of a Secondary License + (if permitted under the terms of Section 3). + + 3. REQUIREMENTS + + 3.1 If a Contributor Distributes the Program in any form, then: + + a) the Program must also be made available as Source Code, in + accordance with section 3.2, and the Contributor must accompany + the Program with a statement that the Source Code for the Program + is available under this Agreement, and informs Recipients how to + obtain it in a reasonable manner on or through a medium customarily + used for software exchange; and + + b) the Contributor may Distribute the Program under a license + different than this Agreement, provided that such license: + i) effectively disclaims on behalf of all other Contributors all + warranties and conditions, express and implied, including + warranties or conditions of title and non-infringement, and + implied warranties or conditions of merchantability and fitness + for a particular purpose; + + ii) effectively excludes on behalf of all other Contributors all + liability for damages, including direct, indirect, special, + incidental and consequential damages, such as lost profits; + + iii) does not attempt to limit or alter the recipients' rights + in the Source Code under section 3.2; and + + iv) requires any subsequent distribution of the Program by any + party to be under a license that satisfies the requirements + of this section 3. + + 3.2 When the Program is Distributed as Source Code: + + a) it must be made available under this Agreement, or if the + Program (i) is combined with other material in a separate file or + files made available under a Secondary License, and (ii) the initial + Contributor attached to the Source Code the notice described in + Exhibit A of this Agreement, then the Program may be made available + under the terms of such Secondary Licenses, and + + b) a copy of this Agreement must be included with each copy of + the Program. + + 3.3 Contributors may not remove or alter any copyright, patent, + trademark, attribution notices, disclaimers of warranty, or limitations + of liability ("notices") contained within the Program from any copy of + the Program which they Distribute, provided that Contributors may add + their own appropriate notices. + + 4. COMMERCIAL DISTRIBUTION + + Commercial distributors of software may accept certain responsibilities + with respect to end users, business partners and the like. While this + license is intended to facilitate the commercial use of the Program, + the Contributor who includes the Program in a commercial product + offering should do so in a manner which does not create potential + liability for other Contributors. Therefore, if a Contributor includes + the Program in a commercial product offering, such Contributor + ("Commercial Contributor") hereby agrees to defend and indemnify every + other Contributor ("Indemnified Contributor") against any losses, + damages and costs (collectively "Losses") arising from claims, lawsuits + and other legal actions brought by a third party against the Indemnified + Contributor to the extent caused by the acts or omissions of such + Commercial Contributor in connection with its distribution of the Program + in a commercial product offering. The obligations in this section do not + apply to any claims or Losses relating to any actual or alleged + intellectual property infringement. In order to qualify, an Indemnified + Contributor must: a) promptly notify the Commercial Contributor in + writing of such claim, and b) allow the Commercial Contributor to control, + and cooperate with the Commercial Contributor in, the defense and any + related settlement negotiations. The Indemnified Contributor may + participate in any such claim at its own expense. + + For example, a Contributor might include the Program in a commercial + product offering, Product X. That Contributor is then a Commercial + Contributor. If that Commercial Contributor then makes performance + claims, or offers warranties related to Product X, those performance + claims and warranties are such Commercial Contributor's responsibility + alone. Under this section, the Commercial Contributor would have to + defend claims against the other Contributors related to those performance + claims and warranties, and if a court requires any other Contributor to + pay any damages as a result, the Commercial Contributor must pay + those damages. + + 5. NO WARRANTY + + EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT + PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED ON AN "AS IS" + BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR + IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF + TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR + PURPOSE. Each Recipient is solely responsible for determining the + appropriateness of using and distributing the Program and assumes all + risks associated with its exercise of rights under this Agreement, + including but not limited to the risks and costs of program errors, + compliance with applicable laws, damage to or loss of data, programs + or equipment, and unavailability or interruption of operations. + + 6. DISCLAIMER OF LIABILITY + + EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT + PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS + SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, + EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST + PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE + EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE + POSSIBILITY OF SUCH DAMAGES. + + 7. GENERAL + + If any provision of this Agreement is invalid or unenforceable under + applicable law, it shall not affect the validity or enforceability of + the remainder of the terms of this Agreement, and without further + action by the parties hereto, such provision shall be reformed to the + minimum extent necessary to make such provision valid and enforceable. + + If Recipient institutes patent litigation against any entity + (including a cross-claim or counterclaim in a lawsuit) alleging that the + Program itself (excluding combinations of the Program with other software + or hardware) infringes such Recipient's patent(s), then such Recipient's + rights granted under Section 2(b) shall terminate as of the date such + litigation is filed. + + All Recipient's rights under this Agreement shall terminate if it + fails to comply with any of the material terms or conditions of this + Agreement and does not cure such failure in a reasonable period of + time after becoming aware of such noncompliance. If all Recipient's + rights under this Agreement terminate, Recipient agrees to cease use + and distribution of the Program as soon as reasonably practicable. + However, Recipient's obligations under this Agreement and any licenses + granted by Recipient relating to the Program shall continue and survive. + + Everyone is permitted to copy and distribute copies of this Agreement, + but in order to avoid inconsistency the Agreement is copyrighted and + may only be modified in the following manner. The Agreement Steward + reserves the right to publish new versions (including revisions) of + this Agreement from time to time. No one other than the Agreement + Steward has the right to modify this Agreement. The Eclipse Foundation + is the initial Agreement Steward. The Eclipse Foundation may assign the + responsibility to serve as the Agreement Steward to a suitable separate + entity. Each new version of the Agreement will be given a distinguishing + version number. The Program (including Contributions) may always be + Distributed subject to the version of the Agreement under which it was + received. In addition, after a new version of the Agreement is published, + Contributor may elect to Distribute the Program (including its + Contributions) under the new version. + + Except as expressly stated in Sections 2(a) and 2(b) above, Recipient + receives no rights or licenses to the intellectual property of any + Contributor under this Agreement, whether expressly, by implication, + estoppel or otherwise. All rights in the Program not expressly granted + under this Agreement are reserved. Nothing in this Agreement is intended + to be enforceable by any entity that is not a Contributor or Recipient. + No third-party beneficiary rights are created under this Agreement. + + Exhibit A - Form of Secondary Licenses Notice + + "This Source Code may also be made available under the following + Secondary Licenses when the conditions for such availability set forth + in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), + version(s), and exceptions or additional permissions here}." + + Simply including a copy of this Agreement, including this Exhibit A + is not sufficient to license the Source Code under Secondary Licenses. + + If it is not possible or desirable to put the notice in a particular + file, then You may include the notice in a location (such as a LICENSE + file in a relevant directory) where a recipient would be likely to + look for such a notice. + + You may add additional accurate notices of copyright ownership. + +--- + +## The GNU General Public License (GPL) Version 2, June 1991 + + Copyright (C) 1989, 1991 Free Software Foundation, Inc. + 51 Franklin Street, Fifth Floor + Boston, MA 02110-1335 + USA + + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + + Preamble + + The licenses for most software are designed to take away your freedom to + share and change it. By contrast, the GNU General Public License is + intended to guarantee your freedom to share and change free software--to + make sure the software is free for all its users. This General Public + License applies to most of the Free Software Foundation's software and + to any other program whose authors commit to using it. (Some other Free + Software Foundation software is covered by the GNU Library General + Public License instead.) You can apply it to your programs, too. + + When we speak of free software, we are referring to freedom, not price. + Our General Public Licenses are designed to make sure that you have the + freedom to distribute copies of free software (and charge for this + service if you wish), that you receive source code or can get it if you + want it, that you can change the software or use pieces of it in new + free programs; and that you know you can do these things. + + To protect your rights, we need to make restrictions that forbid anyone + to deny you these rights or to ask you to surrender the rights. These + restrictions translate to certain responsibilities for you if you + distribute copies of the software, or if you modify it. + + For example, if you distribute copies of such a program, whether gratis + or for a fee, you must give the recipients all the rights that you have. + You must make sure that they, too, receive or can get the source code. + And you must show them these terms so they know their rights. + + We protect your rights with two steps: (1) copyright the software, and + (2) offer you this license which gives you legal permission to copy, + distribute and/or modify the software. + + Also, for each author's protection and ours, we want to make certain + that everyone understands that there is no warranty for this free + software. If the software is modified by someone else and passed on, we + want its recipients to know that what they have is not the original, so + that any problems introduced by others will not reflect on the original + authors' reputations. + + Finally, any free program is threatened constantly by software patents. + We wish to avoid the danger that redistributors of a free program will + individually obtain patent licenses, in effect making the program + proprietary. To prevent this, we have made it clear that any patent must + be licensed for everyone's free use or not licensed at all. + + The precise terms and conditions for copying, distribution and + modification follow. + + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License applies to any program or other work which contains a + notice placed by the copyright holder saying it may be distributed under + the terms of this General Public License. The "Program", below, refers + to any such program or work, and a "work based on the Program" means + either the Program or any derivative work under copyright law: that is + to say, a work containing the Program or a portion of it, either + verbatim or with modifications and/or translated into another language. + (Hereinafter, translation is included without limitation in the term + "modification".) Each licensee is addressed as "you". + + Activities other than copying, distribution and modification are not + covered by this License; they are outside its scope. The act of running + the Program is not restricted, and the output from the Program is + covered only if its contents constitute a work based on the Program + (independent of having been made by running the Program). Whether that + is true depends on what the Program does. + + 1. You may copy and distribute verbatim copies of the Program's source + code as you receive it, in any medium, provided that you conspicuously + and appropriately publish on each copy an appropriate copyright notice + and disclaimer of warranty; keep intact all the notices that refer to + this License and to the absence of any warranty; and give any other + recipients of the Program a copy of this License along with the Program. + + You may charge a fee for the physical act of transferring a copy, and + you may at your option offer warranty protection in exchange for a fee. + + 2. You may modify your copy or copies of the Program or any portion of + it, thus forming a work based on the Program, and copy and distribute + such modifications or work under the terms of Section 1 above, provided + that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any part + thereof, to be licensed as a whole at no charge to all third parties + under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a notice + that there is no warranty (or else, saying that you provide a + warranty) and that users may redistribute the program under these + conditions, and telling the user how to view a copy of this License. + (Exception: if the Program itself is interactive but does not + normally print such an announcement, your work based on the Program + is not required to print an announcement.) + + These requirements apply to the modified work as a whole. If + identifiable sections of that work are not derived from the Program, and + can be reasonably considered independent and separate works in + themselves, then this License, and its terms, do not apply to those + sections when you distribute them as separate works. But when you + distribute the same sections as part of a whole which is a work based on + the Program, the distribution of the whole must be on the terms of this + License, whose permissions for other licensees extend to the entire + whole, and thus to each and every part regardless of who wrote it. + + Thus, it is not the intent of this section to claim rights or contest + your rights to work written entirely by you; rather, the intent is to + exercise the right to control the distribution of derivative or + collective works based on the Program. + + In addition, mere aggregation of another work not based on the Program + with the Program (or with a work based on the Program) on a volume of a + storage or distribution medium does not bring the other work under the + scope of this License. + + 3. You may copy and distribute the Program (or a work based on it, + under Section 2) in object code or executable form under the terms of + Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections 1 + and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your cost + of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer to + distribute corresponding source code. (This alternative is allowed + only for noncommercial distribution and only if you received the + program in object code or executable form with such an offer, in + accord with Subsection b above.) + + The source code for a work means the preferred form of the work for + making modifications to it. For an executable work, complete source code + means all the source code for all modules it contains, plus any + associated interface definition files, plus the scripts used to control + compilation and installation of the executable. However, as a special + exception, the source code distributed need not include anything that is + normally distributed (in either source or binary form) with the major + components (compiler, kernel, and so on) of the operating system on + which the executable runs, unless that component itself accompanies the + executable. + + If distribution of executable or object code is made by offering access + to copy from a designated place, then offering equivalent access to copy + the source code from the same place counts as distribution of the source + code, even though third parties are not compelled to copy the source + along with the object code. + + 4. You may not copy, modify, sublicense, or distribute the Program + except as expressly provided under this License. Any attempt otherwise + to copy, modify, sublicense or distribute the Program is void, and will + automatically terminate your rights under this License. However, parties + who have received copies, or rights, from you under this License will + not have their licenses terminated so long as such parties remain in + full compliance. + + 5. You are not required to accept this License, since you have not + signed it. However, nothing else grants you permission to modify or + distribute the Program or its derivative works. These actions are + prohibited by law if you do not accept this License. Therefore, by + modifying or distributing the Program (or any work based on the + Program), you indicate your acceptance of this License to do so, and all + its terms and conditions for copying, distributing or modifying the + Program or works based on it. + + 6. Each time you redistribute the Program (or any work based on the + Program), the recipient automatically receives a license from the + original licensor to copy, distribute or modify the Program subject to + these terms and conditions. You may not impose any further restrictions + on the recipients' exercise of the rights granted herein. You are not + responsible for enforcing compliance by third parties to this License. + + 7. If, as a consequence of a court judgment or allegation of patent + infringement or for any other reason (not limited to patent issues), + conditions are imposed on you (whether by court order, agreement or + otherwise) that contradict the conditions of this License, they do not + excuse you from the conditions of this License. If you cannot distribute + so as to satisfy simultaneously your obligations under this License and + any other pertinent obligations, then as a consequence you may not + distribute the Program at all. For example, if a patent license would + not permit royalty-free redistribution of the Program by all those who + receive copies directly or indirectly through you, then the only way you + could satisfy both it and this License would be to refrain entirely from + distribution of the Program. + + If any portion of this section is held invalid or unenforceable under + any particular circumstance, the balance of the section is intended to + apply and the section as a whole is intended to apply in other + circumstances. + + It is not the purpose of this section to induce you to infringe any + patents or other property right claims or to contest validity of any + such claims; this section has the sole purpose of protecting the + integrity of the free software distribution system, which is implemented + by public license practices. Many people have made generous + contributions to the wide range of software distributed through that + system in reliance on consistent application of that system; it is up to + the author/donor to decide if he or she is willing to distribute + software through any other system and a licensee cannot impose that choice. + + This section is intended to make thoroughly clear what is believed to be + a consequence of the rest of this License. + + 8. If the distribution and/or use of the Program is restricted in + certain countries either by patents or by copyrighted interfaces, the + original copyright holder who places the Program under this License may + add an explicit geographical distribution limitation excluding those + countries, so that distribution is permitted only in or among countries + not thus excluded. In such case, this License incorporates the + limitation as if written in the body of this License. + + 9. The Free Software Foundation may publish revised and/or new + versions of the General Public License from time to time. Such new + versions will be similar in spirit to the present version, but may + differ in detail to address new problems or concerns. + + Each version is given a distinguishing version number. If the Program + specifies a version number of this License which applies to it and "any + later version", you have the option of following the terms and + conditions either of that version or of any later version published by + the Free Software Foundation. If the Program does not specify a version + number of this License, you may choose any version ever published by the + Free Software Foundation. + + 10. If you wish to incorporate parts of the Program into other free + programs whose distribution conditions are different, write to the + author to ask for permission. For software which is copyrighted by the + Free Software Foundation, write to the Free Software Foundation; we + sometimes make exceptions for this. Our decision will be guided by the + two goals of preserving the free status of all derivatives of our free + software and of promoting the sharing and reuse of software generally. + + NO WARRANTY + + 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO + WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. + EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR + OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, + EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE + ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH + YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL + NECESSARY SERVICING, REPAIR OR CORRECTION. + + 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN + WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY + AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR + DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL + DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM + (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED + INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF + THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR + OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Programs + + If you develop a new program, and you want it to be of the greatest + possible use to the public, the best way to achieve this is to make it + free software which everyone can redistribute and change under these terms. + + To do so, attach the following notices to the program. It is safest to + attach them to the start of each source file to most effectively convey + the exclusion of warranty; and each file should have at least the + "copyright" line and a pointer to where the full notice is found. + + One line to give the program's name and a brief idea of what it does. + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, but + WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + General Public License for more details. + + You should have received a copy of the GNU General Public License + along with this program; if not, write to the Free Software + Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA + + Also add information on how to contact you by electronic and paper mail. + + If the program is interactive, make it output a short notice like this + when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type + `show w'. This is free software, and you are welcome to redistribute + it under certain conditions; type `show c' for details. + + The hypothetical commands `show w' and `show c' should show the + appropriate parts of the General Public License. Of course, the commands + you use may be called something other than `show w' and `show c'; they + could even be mouse-clicks or menu items--whatever suits your program. + + You should also get your employer (if you work as a programmer) or your + school, if any, to sign a "copyright disclaimer" for the program, if + necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the + program `Gnomovision' (which makes passes at compilers) written by + James Hacker. + + signature of Ty Coon, 1 April 1989 + Ty Coon, President of Vice + + This General Public License does not permit incorporating your program + into proprietary programs. If your program is a subroutine library, you + may consider it more useful to permit linking proprietary applications + with the library. If this is what you want to do, use the GNU Library + General Public License instead of this License. + +--- + +## CLASSPATH EXCEPTION + + Linking this library statically or dynamically with other modules is + making a combined work based on this library. Thus, the terms and + conditions of the GNU General Public License version 2 cover the whole + combination. + + As a special exception, the copyright holders of this library give you + permission to link this library with independent modules to produce an + executable, regardless of the license terms of these independent + modules, and to copy and distribute the resulting executable under + terms of your choice, provided that you also meet, for each linked + independent module, the terms and conditions of the license of that + module. An independent module is a module which is not derived from or + based on this library. If you modify this library, you may extend this + exception to your version of the library, but you are not obligated to + do so. If you do not wish to do so, delete this exception statement + from your version. From 7bfb2b4c257b0670882addf018685d3a9c259a1d Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 3 Nov 2020 15:55:58 +0100 Subject: [PATCH 106/322] [FLINK-19276][json][connector-kafka] Support reading Debezium metadata This exposes metadata for the Debezium JSON format according to FLIP-107. - Update the Kafka connector to expose format specific metadata. - Reconfigure the internal JsonRowDataDeserializationSchema to read additional fields. - Let DebeziumJsonDeserializationSchema access and convert those additional fields to metadata columns. This closes #13910. --- .../debezium/DebeziumJsonDecodingFormat.java | 247 ++++++++++++++++++ .../DebeziumJsonDeserializationSchema.java | 192 +++++++++++--- .../debezium/DebeziumJsonFormatFactory.java | 33 +-- .../DebeziumJsonFormatFactoryTest.java | 18 +- .../debezium/DebeziumJsonSerDeSchemaTest.java | 117 ++++++++- 5 files changed, 526 insertions(+), 81 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java new file mode 100644 index 000000000..9590e22d5 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.debezium; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.formats.json.debezium.DebeziumJsonDeserializationSchema.MetadataConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.types.RowKind; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * {@link DecodingFormat} for Debezium using JSON encoding. + */ +public class DebeziumJsonDecodingFormat implements DecodingFormat> { + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + private List metadataKeys; + + // -------------------------------------------------------------------------------------------- + // Debezium-specific attributes + // -------------------------------------------------------------------------------------------- + + private final boolean schemaInclude; + + private final boolean ignoreParseErrors; + + private final TimestampFormat timestampFormat; + + public DebeziumJsonDecodingFormat( + boolean schemaInclude, + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { + this.schemaInclude = schemaInclude; + this.ignoreParseErrors = ignoreParseErrors; + this.timestampFormat = timestampFormat; + this.metadataKeys = Collections.emptyList(); + } + + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, + DataType physicalDataType) { + + final List readableMetadata = metadataKeys.stream() + .map(k -> + Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .collect(Collectors.toList()); + + final List metadataFields = readableMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList()); + + final DataType producedDataType = DataTypeUtils.appendRowFields(physicalDataType, metadataFields); + + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + + return new DebeziumJsonDeserializationSchema( + physicalDataType, + readableMetadata, + producedTypeInfo, + schemaInclude, + ignoreParseErrors, + timestampFormat); + } + + @Override + public Map listReadableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + Stream.of(ReadableMetadata.values()).forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyReadableMetadata(List metadataKeys) { + this.metadataKeys = metadataKeys; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + + /** + * List of metadata that can be read with this format. + */ + enum ReadableMetadata { + SCHEMA( + "schema", + DataTypes.STRING().nullable(), + false, + DataTypes.FIELD("schema", DataTypes.STRING()), + GenericRowData::getString + ), + + INGESTION_TIMESTAMP( + "ingestion-timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + true, + DataTypes.FIELD("ts_ms", DataTypes.BIGINT()), + (row, pos) -> { + return TimestampData.fromEpochMillis(row.getLong(pos)); + } + ), + + SOURCE_TIMESTAMP( + "source.timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + (row, pos) -> { + final StringData timestamp = (StringData) readProperty(row, pos, KEY_SOURCE_TIMESTAMP); + if (timestamp == null) { + return null; + } + return TimestampData.fromEpochMillis(Long.parseLong(timestamp.toString())); + } + ), + + SOURCE_DATABASE( + "source.database", + DataTypes.STRING().nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + (row, pos) -> { + return readProperty(row, pos, KEY_SOURCE_DATABASE); + } + ), + + SOURCE_SCHEMA( + "source.schema", + DataTypes.STRING().nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + (row, pos) -> { + return readProperty(row, pos, KEY_SOURCE_SCHEMA); + } + ), + + SOURCE_TABLE( + "source.table", + DataTypes.STRING().nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + (row, pos) -> { + return readProperty(row, pos, KEY_SOURCE_TABLE); + } + ), + + SOURCE_PROPERTIES( + "source.properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()).nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + GenericRowData::getMap + ); + + final String key; + + final DataType dataType; + + final boolean isJsonPayload; + + final DataTypes.Field requiredJsonField; + + final MetadataConverter converter; + + ReadableMetadata( + String key, + DataType dataType, + boolean isJsonPayload, + DataTypes.Field requiredJsonField, + MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.isJsonPayload = isJsonPayload; + this.requiredJsonField = requiredJsonField; + this.converter = converter; + } + } + + private static final StringData KEY_SOURCE_TIMESTAMP = StringData.fromString("ts_ms"); + + private static final StringData KEY_SOURCE_DATABASE = StringData.fromString("db"); + + private static final StringData KEY_SOURCE_SCHEMA = StringData.fromString("schema"); + + private static final StringData KEY_SOURCE_TABLE = StringData.fromString("table"); + + private static Object readProperty(GenericRowData row, int pos, StringData key) { + final GenericMapData map = (GenericMapData) row.getMap(pos); + if (map == null) { + return null; + } + return map.get(key); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java index 7fc734ad2..43bd94efa 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java @@ -23,19 +23,23 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.formats.json.debezium.DebeziumJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; import org.apache.flink.types.RowKind; import org.apache.flink.util.Collector; import java.io.IOException; +import java.io.Serializable; +import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; import static java.lang.String.format; -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** * Deserialization schema from Debezium JSON to Flink Table/SQL internal data structure {@link RowData}. @@ -62,11 +66,17 @@ public final class DebeziumJsonDeserializationSchema implements DeserializationS "if you are using Debezium Postgres Connector, " + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; - /** The deserializer to deserialize Debezium JSON data. **/ + /** The deserializer to deserialize Debezium JSON data. */ private final JsonRowDataDeserializationSchema jsonDeserializer; - /** TypeInformation of the produced {@link RowData}. **/ - private final TypeInformation resultTypeInfo; + /** Flag that indicates that an additional projection is required for metadata. */ + private final boolean hasMetadata; + + /** Metadata to be extracted for every record. */ + private final MetadataConverter[] metadataConverters; + + /** {@link TypeInformation} of the produced {@link RowData} (physical + meta data). */ + private final TypeInformation producedTypeInfo; /** * Flag indicating whether the Debezium JSON data contains schema part or not. @@ -80,25 +90,29 @@ public final class DebeziumJsonDeserializationSchema implements DeserializationS private final boolean ignoreParseErrors; public DebeziumJsonDeserializationSchema( - RowType rowType, - TypeInformation resultTypeInfo, + DataType physicalDataType, + List requestedMetadata, + TypeInformation producedTypeInfo, boolean schemaInclude, boolean ignoreParseErrors, - TimestampFormat timestampFormatOption) { - this.resultTypeInfo = resultTypeInfo; - this.schemaInclude = schemaInclude; - this.ignoreParseErrors = ignoreParseErrors; + TimestampFormat timestampFormat) { + final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata, schemaInclude); this.jsonDeserializer = new JsonRowDataDeserializationSchema( - createJsonRowType(fromLogicalToDataType(rowType), schemaInclude), - // the result type is never used, so it's fine to pass in Debezium's result type - resultTypeInfo, + jsonRowType, + // the result type is never used, so it's fine to pass in the produced type info + producedTypeInfo, false, // ignoreParseErrors already contains the functionality of failOnMissingField ignoreParseErrors, - timestampFormatOption); + timestampFormat); + this.hasMetadata = requestedMetadata.size() > 0; + this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata, schemaInclude); + this.producedTypeInfo = producedTypeInfo; + this.schemaInclude = schemaInclude; + this.ignoreParseErrors = ignoreParseErrors; } @Override - public RowData deserialize(byte[] message) throws IOException { + public RowData deserialize(byte[] message) { throw new RuntimeException( "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); } @@ -123,21 +137,21 @@ public void deserialize(byte[] message, Collector out) throws IOExcepti String op = payload.getField(2).toString(); if (OP_CREATE.equals(op) || OP_READ.equals(op)) { after.setRowKind(RowKind.INSERT); - out.collect(after); + emitRow(row, after, out); } else if (OP_UPDATE.equals(op)) { if (before == null) { throw new IllegalStateException(String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); } before.setRowKind(RowKind.UPDATE_BEFORE); after.setRowKind(RowKind.UPDATE_AFTER); - out.collect(before); - out.collect(after); + emitRow(row, before, out); + emitRow(row, after, out); } else if (OP_DELETE.equals(op)) { if (before == null) { throw new IllegalStateException(String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); } before.setRowKind(RowKind.DELETE); - out.collect(before); + emitRow(row, before, out); } else { if (!ignoreParseErrors) { throw new IOException(format( @@ -153,6 +167,31 @@ public void deserialize(byte[] message, Collector out) throws IOExcepti } } + private void emitRow(GenericRowData rootRow, GenericRowData physicalRow, Collector out) { + // shortcut in case no output projection is required + if (!hasMetadata) { + out.collect(physicalRow); + return; + } + + final int physicalArity = physicalRow.getArity(); + final int metadataArity = metadataConverters.length; + + final GenericRowData producedRow = new GenericRowData( + physicalRow.getRowKind(), + physicalArity + metadataArity); + + for (int physicalPos = 0; physicalPos < physicalArity; physicalPos++) { + producedRow.setField(physicalPos, physicalRow.getField(physicalPos)); + } + + for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { + producedRow.setField(physicalArity + metadataPos, metadataConverters[metadataPos].convert(rootRow)); + } + + out.collect(producedRow); + } + @Override public boolean isEndOfStream(RowData nextElement) { return false; @@ -160,7 +199,7 @@ public boolean isEndOfStream(RowData nextElement) { @Override public TypeInformation getProducedType() { - return resultTypeInfo; + return producedTypeInfo; } @Override @@ -172,32 +211,113 @@ public boolean equals(Object o) { return false; } DebeziumJsonDeserializationSchema that = (DebeziumJsonDeserializationSchema) o; - return schemaInclude == that.schemaInclude && - ignoreParseErrors == that.ignoreParseErrors && - Objects.equals(jsonDeserializer, that.jsonDeserializer) && - Objects.equals(resultTypeInfo, that.resultTypeInfo); + return Objects.equals(jsonDeserializer, that.jsonDeserializer) + && hasMetadata == that.hasMetadata + && Objects.equals(producedTypeInfo, that.producedTypeInfo) + && schemaInclude == that.schemaInclude + && ignoreParseErrors == that.ignoreParseErrors; } @Override public int hashCode() { - return Objects.hash(jsonDeserializer, resultTypeInfo, schemaInclude, ignoreParseErrors); + return Objects.hash( + jsonDeserializer, + hasMetadata, + producedTypeInfo, + schemaInclude, + ignoreParseErrors); } - private static RowType createJsonRowType(DataType databaseSchema, boolean schemaInclude) { + // -------------------------------------------------------------------------------------------- + + private static RowType createJsonRowType( + DataType physicalDataType, + List readableMetadata, + boolean schemaInclude) { DataType payload = DataTypes.ROW( - DataTypes.FIELD("before", databaseSchema), - DataTypes.FIELD("after", databaseSchema), + DataTypes.FIELD("before", physicalDataType), + DataTypes.FIELD("after", physicalDataType), DataTypes.FIELD("op", DataTypes.STRING())); + + // append fields that are required for reading metadata in the payload + final List payloadMetadataFields = readableMetadata.stream() + .filter(m -> m.isJsonPayload) + .map(m -> m.requiredJsonField) + .distinct() + .collect(Collectors.toList()); + payload = DataTypeUtils.appendRowFields(payload, payloadMetadataFields); + + DataType root = payload; if (schemaInclude) { // when Debezium Kafka connect enables "value.converter.schemas.enable", - // the JSON will contain "schema" information, but we just ignore "schema" - // and extract data from "payload". - return (RowType) DataTypes.ROW( - DataTypes.FIELD("payload", payload)).getLogicalType(); - } else { - // payload contains some other information, e.g. "source", "ts_ms" - // but we don't need them. - return (RowType) payload.getLogicalType(); + // the JSON will contain "schema" information and we need to extract data from "payload". + root = DataTypes.ROW(DataTypes.FIELD("payload", payload)); } + + // append fields that are required for reading metadata in the root + final List rootMetadataFields = readableMetadata.stream() + .filter(m -> !m.isJsonPayload) + .map(m -> m.requiredJsonField) + .distinct() + .collect(Collectors.toList()); + root = DataTypeUtils.appendRowFields(root, rootMetadataFields); + + return (RowType) root.getLogicalType(); + } + + private static MetadataConverter[] createMetadataConverters( + RowType jsonRowType, + List requestedMetadata, + boolean schemaInclude) { + return requestedMetadata.stream() + .map(m -> { + if (m.isJsonPayload) { + return convertInPayload(jsonRowType, m, schemaInclude); + } else { + return convertInRoot(jsonRowType, m); + } + }) + .toArray(MetadataConverter[]::new); + } + + private static MetadataConverter convertInRoot( + RowType jsonRowType, + ReadableMetadata metadata) { + final int pos = findFieldPos(metadata, jsonRowType); + return (root, unused) -> metadata.converter.convert(root, pos); + } + + private static MetadataConverter convertInPayload( + RowType jsonRowType, + ReadableMetadata metadata, + boolean schemaInclude) { + if (schemaInclude) { + final int pos = findFieldPos(metadata, (RowType) jsonRowType.getChildren().get(0)); + return (root, unused) -> { + final GenericRowData payload = (GenericRowData) root.getField(0); + return metadata.converter.convert(payload, pos); + }; + } + return convertInRoot(jsonRowType, metadata); + } + + private static int findFieldPos(ReadableMetadata metadata, RowType jsonRowType) { + return jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); + } + + // -------------------------------------------------------------------------------------------- + + /** + * Converter that extracts a metadata field from the row (root or payload) that comes out of the + * JSON schema and converts it to the desired data type. + */ + interface MetadataConverter extends Serializable { + + // Method for top-level access. + default Object convert(GenericRowData row) { + return convert(row, -1); + } + + Object convert(GenericRowData row, int pos); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java index 794b1a49e..63171ad66 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.ReadableConfig; @@ -30,7 +29,6 @@ import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DeserializationFormatFactory; import org.apache.flink.table.factories.DynamicTableFactory; @@ -68,36 +66,16 @@ public class DebeziumJsonFormatFactory implements DeserializationFormatFactory, public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + final boolean schemaInclude = formatOptions.get(SCHEMA_INCLUDE); + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); - return new DecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType producedDataType) { - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation rowDataTypeInfo = - context.createTypeInformation(producedDataType); - return new DebeziumJsonDeserializationSchema( - rowType, - rowDataTypeInfo, - schemaInclude, - ignoreParseErrors, - timestampFormatOption); - } + final TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - }; + return new DebeziumJsonDecodingFormat(schemaInclude, ignoreParseErrors, timestampFormat); } @Override @@ -145,5 +123,4 @@ public Set> optionalOptions() { options.add(TIMESTAMP_FORMAT); return options; } - } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 47df75c88..ae343c195 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -34,6 +34,7 @@ import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; @@ -41,6 +42,7 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; @@ -61,13 +63,14 @@ public class DebeziumJsonFormatFactoryTest extends TestLogger { .field("c", DataTypes.BOOLEAN()) .build(); - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + private static final DataType PHYSICAL_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); @Test public void testSeDeSchema() { final DebeziumJsonDeserializationSchema expectedDeser = new DebeziumJsonDeserializationSchema( - ROW_TYPE, - InternalTypeInfo.of(ROW_TYPE), + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), true, true, TimestampFormat.ISO_8601); @@ -81,14 +84,13 @@ public void testSeDeSchema() { DeserializationSchema actualDeser = scanSourceMock.valueFormat .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, - SCHEMA.toRowDataType()); + ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); assertEquals(expectedDeser, actualDeser); final DebeziumJsonSerializationSchema expectedSer = new DebeziumJsonSerializationSchema( - ROW_TYPE, - TimestampFormat.ISO_8601); + (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), + TimestampFormat.ISO_8601); final DynamicTableSink actualSink = createTableSink(options); assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; @@ -98,7 +100,7 @@ public void testSeDeSchema() { SerializationSchema actualSer = sinkMock.valueFormat .createRuntimeEncoder( new SinkRuntimeProviderContext(false), - SCHEMA.toRowDataType()); + PHYSICAL_DATA_TYPE); assertEquals(expectedSer, actualSer); } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java index 0c4b1a620..00e6ffbcd 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -19,9 +19,13 @@ package org.apache.flink.formats.json.debezium; import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.formats.json.debezium.DebeziumJsonDecodingFormat.ReadableMetadata; +import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; import org.apache.flink.util.Collector; import org.apache.flink.util.ExceptionUtils; @@ -37,7 +41,9 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.function.Consumer; import java.util.stream.Collectors; import static org.apache.flink.table.api.DataTypes.FIELD; @@ -45,7 +51,10 @@ import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.startsWith; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -56,12 +65,11 @@ public class DebeziumJsonSerDeSchemaTest { @Rule public ExpectedException thrown = ExpectedException.none(); - private static final RowType SCHEMA = (RowType) ROW( + private static final DataType PHYSICAL_DATA_TYPE = ROW( FIELD("id", INT().notNull()), FIELD("name", STRING()), FIELD("description", STRING()), - FIELD("weight", FLOAT()) - ).getLogicalType(); + FIELD("weight", FLOAT())); @Test public void testSerializationAndSchemaIncludeDeserialization() throws Exception { @@ -84,7 +92,7 @@ public void testSerializationAndPostgresSchemaExcludeDeserialization() throws Ex } @Test - public void testPostgresDefaultReplicaIdentify() throws Exception { + public void testPostgresDefaultReplicaIdentify() { try { testSerializationDeserialization("debezium-postgres-data-replica-identity.txt", false); } catch (Exception e) { @@ -97,8 +105,9 @@ public void testPostgresDefaultReplicaIdentify() throws Exception { @Test public void testTombstoneMessages() throws Exception { DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( - SCHEMA, - InternalTypeInfo.of(SCHEMA), + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), false, false, TimestampFormat.ISO_8601); @@ -108,11 +117,71 @@ public void testTombstoneMessages() throws Exception { assertTrue(collector.list.isEmpty()); } + @Test + public void testDeserializationWithMetadata() throws Exception { + testDeserializationWithMetadata( + "debezium-data-schema-include.txt", + true, + row -> { + assertThat(row.getInt(0), equalTo(101)); + assertThat(row.getString(1).toString(), equalTo("scooter")); + assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); + assertThat(row.getFloat(3), equalTo(3.14f)); + assertThat( + row.getString(4).toString(), + startsWith("{\"type\":\"struct\",\"fields\":[{\"type\":\"struct\",\"fields\":[{\"type\":\"int32\",\"optional\":false,\"field\":\"id\"},")); + assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1589355606100L)); + assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(0L)); + assertThat(row.getString(7).toString(), equalTo("inventory")); + assertThat(row.isNullAt(8), equalTo(true)); + assertThat(row.getString(9).toString(), equalTo("products")); + assertThat(row.getMap(10).size(), equalTo(14)); + } + ); + + testDeserializationWithMetadata( + "debezium-data-schema-exclude.txt", + false, + row -> { + assertThat(row.getInt(0), equalTo(101)); + assertThat(row.getString(1).toString(), equalTo("scooter")); + assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); + assertThat(row.getFloat(3), equalTo(3.14f)); + assertThat(row.isNullAt(4), equalTo(true)); + assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1589355606100L)); + assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(0L)); + assertThat(row.getString(7).toString(), equalTo("inventory")); + assertThat(row.isNullAt(8), equalTo(true)); + assertThat(row.getString(9).toString(), equalTo("products")); + assertThat(row.getMap(10).size(), equalTo(14)); + } + ); + + testDeserializationWithMetadata( + "debezium-postgres-data-schema-exclude.txt", + false, + row -> { + assertThat(row.getInt(0), equalTo(101)); + assertThat(row.getString(1).toString(), equalTo("scooter")); + assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); + assertThat(row.getFloat(3), equalTo(3.14f)); + assertThat(row.isNullAt(4), equalTo(true)); + assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1596001099434L)); + assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(1596001099434L)); + assertThat(row.getString(7).toString(), equalTo("postgres")); + assertThat(row.getString(8).toString(), equalTo("inventory")); + assertThat(row.getString(9).toString(), equalTo("products")); + assertThat(row.getMap(10).size(), equalTo(11)); + } + ); + } + private void testSerializationDeserialization(String resourceFile, boolean schemaInclude) throws Exception { List lines = readLines(resourceFile); DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( - SCHEMA, - InternalTypeInfo.of(SCHEMA), + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), schemaInclude, false, TimestampFormat.ISO_8601); @@ -177,7 +246,7 @@ private void testSerializationDeserialization(String resourceFile, boolean schem assertEquals(expected, actual); DebeziumJsonSerializationSchema serializationSchema = new DebeziumJsonSerializationSchema( - SCHEMA, + (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), TimestampFormat.SQL); serializationSchema.open(null); actual = new ArrayList<>(); @@ -210,6 +279,36 @@ private void testSerializationDeserialization(String resourceFile, boolean schem assertEquals(expected, actual); } + private void testDeserializationWithMetadata( + String resourceFile, + boolean schemaInclude, + Consumer testConsumer) throws Exception { + // we only read the first line for keeping the test simple + final String firstLine = readLines(resourceFile).get(0); + + final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); + + final DataType producedDataType = DataTypeUtils.appendRowFields( + PHYSICAL_DATA_TYPE, + requestedMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList())); + + final DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + requestedMetadata, + InternalTypeInfo.of(producedDataType.getLogicalType()), + schemaInclude, + false, + TimestampFormat.ISO_8601); + + final SimpleCollector collector = new SimpleCollector(); + deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); + + assertEquals(1, collector.list.size()); + testConsumer.accept(collector.list.get(0)); + } + // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- From 2f6933db5a1500fbb7d205b141d93aa8a67db768 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Sat, 7 Nov 2020 19:18:19 +0800 Subject: [PATCH 107/322] [FLINK-19823][table][fs-connector] Filesystem connector supports de/serialization schema This closes #13957 --- .../json/JsonFileSystemFormatFactory.java | 271 ------------------ .../org.apache.flink.table.factories.Factory | 1 - .../json/JsonBatchFileSystemITCase.java | 2 +- .../json/JsonFileCompactionITCase.java | 32 +++ .../DebeziumJsonFileSystemITCase.java | 152 ++++++++++ 5 files changed, 185 insertions(+), 273 deletions(-) delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java deleted file mode 100644 index 831b8ca5d..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFileSystemFormatFactory.java +++ /dev/null @@ -1,271 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.api.common.io.DelimitedInputFormat; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.serialization.BulkWriter; -import org.apache.flink.api.common.serialization.Encoder; -import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.core.fs.Path; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.FileSystemFormatFactory; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.utils.PartitionPathUtils; - -import java.io.IOException; -import java.io.OutputStream; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; - -import static org.apache.flink.formats.json.JsonFormatFactory.validateFormatOptions; -import static org.apache.flink.formats.json.JsonOptions.FAIL_ON_MISSING_FIELD; -import static org.apache.flink.formats.json.JsonOptions.IGNORE_PARSE_ERRORS; -import static org.apache.flink.formats.json.JsonOptions.TIMESTAMP_FORMAT; - -/** - * Factory to build reader/writer to read/write json format file. - */ -public class JsonFileSystemFormatFactory implements FileSystemFormatFactory { - - public static final String IDENTIFIER = "json"; - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return new HashSet<>(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(FAIL_ON_MISSING_FIELD); - options.add(IGNORE_PARSE_ERRORS); - options.add(TIMESTAMP_FORMAT); - return options; - } - - @Override - public InputFormat createReader(ReaderContext context) { - ReadableConfig options = context.getFormatOptions(); - validateFormatOptions(options); - boolean failOnMissingField = options.get(FAIL_ON_MISSING_FIELD); - boolean ignoreParseErrors = options.get(IGNORE_PARSE_ERRORS); - TimestampFormat timestampOption = JsonOptions.getTimestampFormat(options); - - RowType formatRowType = context.getFormatRowType(); - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - formatRowType, - new GenericTypeInfo(GenericRowData.class), - failOnMissingField, - ignoreParseErrors, - timestampOption); - - String[] fieldNames = context.getSchema().getFieldNames(); - List projectFields = Arrays.stream(context.getProjectFields()) - .mapToObj(idx -> fieldNames[idx]) - .collect(Collectors.toList()); - List jsonFields = Arrays.stream(fieldNames) - .filter(field -> !context.getPartitionKeys().contains(field)) - .collect(Collectors.toList()); - - int[] jsonSelectFieldToProjectFieldMapping = context.getFormatProjectFields().stream() - .mapToInt(projectFields::indexOf) - .toArray(); - int[] jsonSelectFieldToJsonFieldMapping = context.getFormatProjectFields().stream() - .mapToInt(jsonFields::indexOf) - .toArray(); - - return new JsonInputFormat( - context.getPaths(), - context.getSchema().getFieldDataTypes(), - context.getSchema().getFieldNames(), - context.getProjectFields(), - context.getPartitionKeys(), - context.getDefaultPartName(), - context.getPushedDownLimit(), - jsonSelectFieldToProjectFieldMapping, - jsonSelectFieldToJsonFieldMapping, - deserializationSchema); - } - - @Override - public Optional> createEncoder(WriterContext context) { - return Optional.of(new JsonRowDataEncoder(new JsonRowDataSerializationSchema(context.getFormatRowType(), - JsonOptions.getTimestampFormat(context.getFormatOptions())))); - } - - @Override - public Optional> createBulkWriterFactory(WriterContext context) { - return Optional.empty(); - } - - /** - * A {@link JsonInputFormat} is responsible to read {@link RowData} records - * from json format files. - */ - public static class JsonInputFormat extends DelimitedInputFormat { - - private static final long serialVersionUID = 1L; - - /** - * Code of \r, used to remove \r from a line when the line ends with \r\n. - */ - private static final byte CARRIAGE_RETURN = (byte) '\r'; - - /** - * Code of \n, used to identify if \n is used as delimiter. - */ - private static final byte NEW_LINE = (byte) '\n'; - - private final DataType[] fieldTypes; - private final String[] fieldNames; - private final int[] selectFields; - private final List partitionKeys; - private final String defaultPartValue; - private final long limit; - private final int[] jsonSelectFieldToProjectFieldMapping; - private final int[] jsonSelectFieldToJsonFieldMapping; - private final JsonRowDataDeserializationSchema deserializationSchema; - - private transient boolean end; - private transient long emitted; - // reuse object for per record - private transient GenericRowData rowData; - - public JsonInputFormat( - Path[] filePaths, - DataType[] fieldTypes, - String[] fieldNames, - int[] selectFields, - List partitionKeys, - String defaultPartValue, - long limit, - int[] jsonSelectFieldToProjectFieldMapping, - int[] jsonSelectFieldToJsonFieldMapping, - JsonRowDataDeserializationSchema deserializationSchema) { - super.setFilePaths(filePaths); - this.fieldTypes = fieldTypes; - this.fieldNames = fieldNames; - this.selectFields = selectFields; - this.partitionKeys = partitionKeys; - this.defaultPartValue = defaultPartValue; - this.limit = limit; - this.jsonSelectFieldToProjectFieldMapping = jsonSelectFieldToProjectFieldMapping; - this.jsonSelectFieldToJsonFieldMapping = jsonSelectFieldToJsonFieldMapping; - this.deserializationSchema = deserializationSchema; - } - - @Override - public boolean supportsMultiPaths() { - return true; - } - - @Override - public void open(FileInputSplit split) throws IOException { - super.open(split); - this.end = false; - this.emitted = 0L; - this.rowData = PartitionPathUtils.fillPartitionValueForRecord(fieldNames, fieldTypes, selectFields, - partitionKeys, currentSplit.getPath(), defaultPartValue); - } - - @Override - public boolean reachedEnd() { - return emitted >= limit || end; - } - - @Override - public RowData readRecord(RowData reuse, byte[] bytes, int offset, int numBytes) throws IOException { - // remove \r from a line when the line ends with \r\n - if (this.getDelimiter() != null && this.getDelimiter().length == 1 - && this.getDelimiter()[0] == NEW_LINE && offset + numBytes >= 1 - && bytes[offset + numBytes - 1] == CARRIAGE_RETURN) { - numBytes -= 1; - } - byte[] trimBytes = Arrays.copyOfRange(bytes, offset, offset + numBytes); - GenericRowData jsonRow = (GenericRowData) deserializationSchema.deserialize(trimBytes); - - if (jsonRow == null) { - return null; - } - - GenericRowData returnRecord = rowData; - for (int i = 0; i < jsonSelectFieldToJsonFieldMapping.length; i++) { - returnRecord.setField(jsonSelectFieldToProjectFieldMapping[i], - jsonRow.getField(jsonSelectFieldToJsonFieldMapping[i])); - } - - emitted++; - return returnRecord; - } - - @Override - public RowData nextRecord(RowData record) throws IOException { - while (true) { - if (readLine()) { - RowData row = readRecord(record, this.currBuffer, this.currOffset, this.currLen); - if (row == null) { - continue; - } else { - return row; - } - } else { - this.end = true; - return null; - } - } - } - } - - /** - * A {@link JsonRowDataEncoder} is responsible to encode a {@link RowData} to {@link java.io.OutputStream} - * with json format. - */ - public static class JsonRowDataEncoder implements Encoder { - - private static final long serialVersionUID = 1L; - private static final String DEFAULT_LINE_DELIMITER = "\n"; - private final JsonRowDataSerializationSchema serializationSchema; - - public JsonRowDataEncoder(JsonRowDataSerializationSchema serializationSchema) { - this.serializationSchema = serializationSchema; - } - - @Override - public void encode(RowData element, OutputStream stream) throws IOException { - stream.write(serializationSchema.serialize(element)); - stream.write(DEFAULT_LINE_DELIMITER.getBytes(StandardCharsets.UTF_8)); - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 942b3dc43..08f4657d4 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,7 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.formats.json.JsonFileSystemFormatFactory org.apache.flink.formats.json.JsonFormatFactory org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory org.apache.flink.formats.json.canal.CanalJsonFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java index f1ee52b33..7d17dbd1d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java @@ -31,7 +31,7 @@ import java.util.List; /** - * ITCase to test json format for {@link JsonFileSystemFormatFactory}. + * ITCase to test json format for {@link JsonFormatFactory}. */ public class JsonBatchFileSystemITCase extends BatchFileSystemITCaseBase { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java new file mode 100644 index 000000000..dd1a8cc76 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.table.planner.runtime.stream.sql.FileCompactionITCaseBase; + +/** + * Compaction it case for json. + */ +public class JsonFileCompactionITCase extends FileCompactionITCaseBase { + + @Override + protected String format() { + return "json"; + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java new file mode 100644 index 000000000..cf3e12c0a --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.debezium; + +import org.apache.flink.table.planner.runtime.utils.StreamingTestBase; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.CollectionUtil; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static java.lang.String.format; + +/** + * Test Filesystem connector with DebeziumJson. + */ +public class DebeziumJsonFileSystemITCase extends StreamingTestBase { + + private static final List EXPECTED = Arrays.asList( + "+I(101,SCOOTER,Small 2-wheel scooter,3.14)", + "+I(102,CAR BATTERY,12V car battery,8.1)", + "+I(103,12-PACK DRILL BITS,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", + "+I(104,HAMMER,12oz carpenter's hammer,0.75)", + "+I(105,HAMMER,14oz carpenter's hammer,0.875)", + "+I(106,HAMMER,16oz carpenter's hammer,1.0)", + "+I(107,ROCKS,box of assorted rocks,5.3)", + "+I(108,JACKET,water resistent black wind breaker,0.1)", + "+I(109,SPARE TIRE,24 inch spare tire,22.2)", + "-D(106,HAMMER,16oz carpenter's hammer,1.0)", // -U + "+I(106,HAMMER,18oz carpenter hammer,1.0)", // +U + "-D(107,ROCKS,box of assorted rocks,5.3)", // -U + "+I(107,ROCKS,box of assorted rocks,5.1)", // +U + "+I(110,JACKET,water resistent white wind breaker,0.2)", + "+I(111,SCOOTER,Big 2-wheel scooter ,5.18)", + "-D(110,JACKET,water resistent white wind breaker,0.2)", // -U + "+I(110,JACKET,new water resistent white wind breaker,0.5)", // +U + "-D(111,SCOOTER,Big 2-wheel scooter ,5.18)", // -U + "+I(111,SCOOTER,Big 2-wheel scooter ,5.17)", // +U + "-D(111,SCOOTER,Big 2-wheel scooter ,5.17)" + ); + + private File source; + private File sink; + + private void prepareTables(boolean isPartition) throws IOException { + byte[] bytes = readBytes("debezium-data-schema-exclude.txt"); + source = TEMPORARY_FOLDER.newFolder(); + File file; + if (isPartition) { + File partition = new File(source, "p=1"); + partition.mkdirs(); + file = new File(partition, "my_file"); + } else { + file = new File(source, "my_file"); + } + file.createNewFile(); + Files.write(file.toPath(), bytes); + + sink = TEMPORARY_FOLDER.newFolder(); + + env().setParallelism(1); + } + + private void createTable(boolean isSink, String path, boolean isPartition) { + tEnv().executeSql(format("create table %s (", isSink ? "sink" : "source") + + "id int, name string," + + (isSink ? "upper_name string," : "") + + " description string, weight float" + + (isPartition ? ", p int) partitioned by (p) " : ")") + + " with (" + + "'connector'='filesystem'," + + "'format'='debezium-json'," + + format("'path'='%s'", path) + + ")"); + } + + @Test + public void testNonPartition() throws Exception { + prepareTables(false); + createTable(false, source.toURI().toString(), false); + createTable(true, sink.toURI().toString(), false); + + tEnv().executeSql("insert into sink select id,name,UPPER(name),description,weight from source").await(); + CloseableIterator iter = tEnv() + .executeSql("select id,upper_name,description,weight from sink").collect(); + + List results = CollectionUtil.iteratorToList(iter).stream() + .map(row -> row.getKind().shortString() + "(" + row.toString() + ")") + .collect(Collectors.toList()); + iter.close(); + + Assert.assertEquals(EXPECTED, results); + } + + @Test + public void testPartition() throws Exception { + prepareTables(true); + createTable(false, source.toURI().toString(), true); + createTable(true, sink.toURI().toString(), true); + + tEnv().executeSql("insert into sink select id,name,UPPER(name),description,weight,p from source").await(); + CloseableIterator iter = tEnv() + .executeSql("select id,upper_name,description,weight,p from sink").collect(); + List list = CollectionUtil.iteratorToList(iter); + iter.close(); + + List results = list.stream() + .map(row -> Row.project(row, new int[] {0, 1, 2, 3})) + .map(row -> row.getKind().shortString() + "(" + row.toString() + ")") + .collect(Collectors.toList()); + + Assert.assertEquals(EXPECTED, results); + + // check partition value + for (Row row : list) { + Assert.assertEquals(1, row.getField(4)); + } + } + + private static byte[] readBytes(String resource) throws IOException { + final URL url = DebeziumJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllBytes(path); + } +} From 1dd7a3a70b116627d4e4c96c94b156b905f912b0 Mon Sep 17 00:00:00 2001 From: caozhen1937 Date: Tue, 1 Sep 2020 16:33:27 +0800 Subject: [PATCH 108/322] [FLINK-18774][debezium-avro] Support debezium avro format This closes #13296 --- .../DebeziumAvroDeserializationSchema.java | 200 ++++++++++++++++++ .../debezium/DebeziumAvroFormatFactory.java | 142 +++++++++++++ .../DebeziumAvroSerializationSchema.java | 137 ++++++++++++ .../org.apache.flink.table.factories.Factory | 1 + .../DebeziumAvroFormatFactoryTest.java | 132 ++++++++++++ .../debezium/DebeziumAvroSerDeSchemaTest.java | 193 +++++++++++++++++ .../test/resources/debezium-avro-delete.txt | Bin 0 -> 147 bytes .../test/resources/debezium-avro-insert.txt | Bin 0 -> 143 bytes .../test/resources/debezium-avro-update.txt | Bin 0 -> 188 bytes 9 files changed, 805 insertions(+) create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-delete.txt create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-insert.txt create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-update.txt diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java new file mode 100644 index 000000000..5c21a62a5 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent.debezium; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; +import org.apache.flink.formats.avro.AvroToRowDataConverters; +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.Objects; + +import static java.lang.String.format; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Deserialization schema from Debezium Avro to Flink Table/SQL internal data structure {@link RowData}. + * The deserialization schema knows Debezium's schema definition and can extract the database data + * and convert into {@link RowData} with {@link RowKind}. + * Deserializes a byte[] message as a JSON object and reads + * the specified fields. + * Failures during deserialization are forwarded as wrapped IOExceptions. + * + * @see Debezium + */ +@Internal +public final class DebeziumAvroDeserializationSchema implements DeserializationSchema { + private static final long serialVersionUID = 1L; + + /** + * snapshot read. + */ + private static final String OP_READ = "r"; + /** + * insert operation. + */ + private static final String OP_CREATE = "c"; + /** + * update operation. + */ + private static final String OP_UPDATE = "u"; + /** + * delete operation. + */ + private static final String OP_DELETE = "d"; + + private static final String REPLICA_IDENTITY_EXCEPTION = "The \"before\" field of %s message is null, " + + "if you are using Debezium Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; + + /** + * The deserializer to deserialize Debezium Avro data. + */ + private final AvroRowDataDeserializationSchema avroDeserializer; + + /** + * TypeInformation of the produced {@link RowData}. + **/ + private final TypeInformation resultTypeInfo; + + /** + * Debezium Avro data rowType. + */ + private final RowType rowType; + + public DebeziumAvroDeserializationSchema( + RowType rowType, + TypeInformation resultTypeInfo, + String schemaRegistryUrl) { + this.resultTypeInfo = resultTypeInfo; + this.rowType = rowType; + RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + + this.avroDeserializer = new AvroRowDataDeserializationSchema( + ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(debeziumAvroRowType), + schemaRegistryUrl), + AvroToRowDataConverters.createRowConverter(debeziumAvroRowType), + resultTypeInfo); + } + + public DebeziumAvroDeserializationSchema( + RowType rowType, + TypeInformation resultTypeInfo, + AvroRowDataDeserializationSchema avroDeserializer) { + this.rowType = rowType; + this.resultTypeInfo = resultTypeInfo; + this.avroDeserializer = avroDeserializer; + } + + @Override + public RowData deserialize(byte[] message) throws IOException { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + + @Override + public void deserialize(byte[] message, Collector out) throws IOException { + + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } + try { + GenericRowData row = (GenericRowData) avroDeserializer.deserialize(message); + + GenericRowData before = (GenericRowData) row.getField(0); + GenericRowData after = (GenericRowData) row.getField(1); + String op = row.getField(2).toString(); + if (OP_CREATE.equals(op) || OP_READ.equals(op)) { + after.setRowKind(RowKind.INSERT); + out.collect(after); + } else if (OP_UPDATE.equals(op)) { + if (before == null) { + throw new IllegalStateException(String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + before.setRowKind(RowKind.UPDATE_BEFORE); + after.setRowKind(RowKind.UPDATE_AFTER); + out.collect(before); + out.collect(after); + } else if (OP_DELETE.equals(op)) { + if (before == null) { + throw new IllegalStateException(String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); + } + before.setRowKind(RowKind.DELETE); + out.collect(before); + } else { + throw new IOException(format( + "Unknown \"op\" value \"%s\". The Debezium Avro message is '%s'", op, new String(message))); + } + } catch (Throwable t) { + // a big try catch to protect the processing. + throw new IOException(format( + "Corrupt Debezium Avro message '%s'.", new String(message)), t); + } + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return resultTypeInfo; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DebeziumAvroDeserializationSchema that = (DebeziumAvroDeserializationSchema) o; + return Objects.equals(avroDeserializer, that.avroDeserializer) && + Objects.equals(resultTypeInfo, that.resultTypeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(avroDeserializer, resultTypeInfo); + } + + public static RowType createDebeziumAvroRowType(DataType dataType) { + // Debezium Avro contains other information, e.g. "source", "ts_ms" + // but we don't need them + return (RowType) DataTypes.ROW( + DataTypes.FIELD("before", dataType.nullable()), + DataTypes.FIELD("after", dataType.nullable()), + DataTypes.FIELD("op", DataTypes.STRING())).getLogicalType(); + } +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java new file mode 100644 index 000000000..d5d522ac0 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent.debezium; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; + +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_SUBJECT; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_URL; + +/** + * Format factory for providing configured instances of Debezium Avro to RowData {@link DeserializationSchema}. + */ +public class DebeziumAvroFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "debezium-avro-confluent"; + + @SuppressWarnings("unchecked") + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + + FactoryUtil.validateFactoryOptions(this, formatOptions); + String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); + + return new DecodingFormat>() { + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, + DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + context.createTypeInformation(producedDataType); + return new DebeziumAvroDeserializationSchema( + rowType, + rowDataTypeInfo, + schemaRegistryURL); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + }; + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { + + FactoryUtil.validateFactoryOptions(this, formatOptions); + String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); + Optional subject = formatOptions.getOptional(SCHEMA_REGISTRY_SUBJECT); + if (!subject.isPresent()) { + throw new ValidationException(String.format("Option %s.%s is required for serialization", + IDENTIFIER, SCHEMA_REGISTRY_SUBJECT.key())); + } + return new EncodingFormat>() { + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, + DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new DebeziumAvroSerializationSchema(rowType, schemaRegistryURL, subject.get()); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + Set> options = new HashSet<>(); + options.add(SCHEMA_REGISTRY_URL); + return options; + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(SCHEMA_REGISTRY_SUBJECT); + return options; + } + +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java new file mode 100644 index 000000000..6c561c0dc --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent.debezium; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Objects; + +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium Avro. + */ +public class DebeziumAvroSerializationSchema implements SerializationSchema { + private static final long serialVersionUID = 1L; + + /** + * insert operation. + */ + private static final StringData OP_CREATE = StringData.fromString("c"); + /** + * delete operation. + */ + private static final StringData OP_DELETE = StringData.fromString("d"); + + /** + * The deserializer to deserialize Debezium Avro data. + */ + private final AvroRowDataSerializationSchema avroSerializer; + + public DebeziumAvroSerializationSchema( + RowType rowType, + String schemaRegistryUrl, + String schemaRegistrySubject) { + RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + + this.avroSerializer = new AvroRowDataSerializationSchema( + debeziumAvroRowType, + ConfluentRegistryAvroSerializationSchema.forGeneric( + schemaRegistrySubject, + AvroSchemaConverter.convertToSchema(debeziumAvroRowType), + schemaRegistryUrl), + RowDataToAvroConverters.createRowConverter(debeziumAvroRowType)); + } + + public DebeziumAvroSerializationSchema(AvroRowDataSerializationSchema avroSerializer) { + this.avroSerializer = avroSerializer; + } + + @Override + public void open(InitializationContext context) throws Exception { + } + + @Override + public byte[] serialize(RowData element) { + GenericRowData reuse = new GenericRowData(3); + switch (element.getRowKind()) { + case INSERT: + reuse.setField(1, element); + break; + case DELETE: + reuse.setField(0, element); + break; + default: + throw new UnsupportedOperationException("Unsupported operation '" + element.getRowKind() + "' for row kind."); + } + reuse.setField(2, rowKind2String(element.getRowKind())); + return avroSerializer.serialize(reuse); + } + + private StringData rowKind2String(RowKind rowKind) { + switch (rowKind) { + case INSERT: + case UPDATE_AFTER: + return OP_CREATE; + case UPDATE_BEFORE: + case DELETE: + return OP_DELETE; + default: + throw new UnsupportedOperationException("Unsupported operation '" + rowKind + "' for row kind."); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DebeziumAvroSerializationSchema that = (DebeziumAvroSerializationSchema) o; + return Objects.equals(avroSerializer, that.avroSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(avroSerializer); + } + + public static RowType createDebeziumAvroRowType(DataType dataType) { + // Debezium Avro contains other information, e.g. "source", "ts_ms" + // but we don't need them + return (RowType) DataTypes.ROW( + DataTypes.FIELD("before", dataType.nullable()), + DataTypes.FIELD("after", dataType.nullable()), + DataTypes.FIELD("op", DataTypes.STRING())).getLogicalType(); + } + +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 57dd1ba79..e1b78a27c 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -14,3 +14,4 @@ # limitations under the License. org.apache.flink.formats.avro.registry.confluent.RegistryAvroFormatFactory +org.apache.flink.formats.avro.registry.confluent.debezium.DebeziumAvroFormatFactory diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java new file mode 100644 index 000000000..ab280ffb6 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent.debezium; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.HashMap; +import java.util.Map; + +import static junit.framework.TestCase.assertEquals; + +/** + * Tests for {@link DebeziumAvroFormatFactory}. + */ +public class DebeziumAvroFormatFactoryTest extends TestLogger { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + private static final String SUBJECT = "test-debezium-avro"; + private static final String REGISTRY_URL = "http://localhost:8081"; + + @Test + public void testSeDeSchema() { + final Map options = getAllOptions(); + + final DebeziumAvroDeserializationSchema expectedDeser = new DebeziumAvroDeserializationSchema( + ROW_TYPE, + InternalTypeInfo.of(ROW_TYPE), + REGISTRY_URL); + + final DynamicTableSource actualSource = createTableSource(options); + assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = scanSourceMock.valueFormat + .createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toRowDataType()); + + assertEquals(expectedDeser, actualDeser); + final DebeziumAvroSerializationSchema expectedSer = new DebeziumAvroSerializationSchema( + ROW_TYPE, + REGISTRY_URL, + SUBJECT + ); + final DynamicTableSink actualSink = createTableSink(options); + assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = sinkMock.valueFormat + .createRuntimeEncoder( + new SinkRuntimeProviderContext(false), + SCHEMA.toRowDataType()); + + Assert.assertEquals(expectedSer, actualSer); + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", DebeziumAvroFormatFactory.IDENTIFIER); + options.put("debezium-avro-confluent.schema-registry.url", REGISTRY_URL); + options.put("debezium-avro-confluent.schema-registry.subject", SUBJECT); + return options; + } + + private static DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + DebeziumAvroFormatFactoryTest.class.getClassLoader(), false); + } + + private static DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + DebeziumAvroFormatFactoryTest.class.getClassLoader(), false); + } +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java new file mode 100644 index 000000000..d87796ae9 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java @@ -0,0 +1,193 @@ +package org.apache.flink.formats.avro.registry.confluent.debezium; + +import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; +import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; +import org.apache.flink.formats.avro.AvroToRowDataConverters; +import org.apache.flink.formats.avro.RegistryAvroDeserializationSchema; +import org.apache.flink.formats.avro.RegistryAvroSerializationSchema; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.registry.confluent.ConfluentSchemaRegistryCoder; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryStringData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; +import org.apache.flink.util.FileUtils; + +import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.junit.Test; + +import java.io.File; +import java.net.URL; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.table.api.DataTypes.BIGINT; +import static org.apache.flink.table.api.DataTypes.DOUBLE; +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link DebeziumAvroDeserializationSchema}. + */ +public class DebeziumAvroSerDeSchemaTest { + + private static final String SUBJECT = "testDebeziumAvro"; + + private static SchemaRegistryClient client = new MockSchemaRegistryClient(); + + private static final RowType rowType = (RowType) ROW( + FIELD("id", BIGINT()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", DOUBLE()) + ).getLogicalType(); + + private static final Schema DEBEZIUM_SCHEMA_COMPATIBLE_TEST = new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"Envelope\",\"namespace\":\"fullfillment.test1.person\",\"fields\":[{\"name\":\"before\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"Value\",\"fields\":[{\"name\":\"id\",\"type\":\"long\"},{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"description\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"weight\",\"type\":[\"null\",\"double\"],\"default\":null}],\"connect.name\":\"fullfillment.test1.person.Value\"}],\"default\":null},{\"name\":\"after\",\"type\":[\"null\",\"Value\"],\"default\":null},{\"name\":\"source\",\"type\":{\"type\":\"record\",\"name\":\"Source\",\"namespace\":\"io.debezium.connector.mysql\",\"fields\":[{\"name\":\"version\",\"type\":\"string\"},{\"name\":\"connector\",\"type\":\"string\"},{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"ts_ms\",\"type\":\"long\"},{\"name\":\"snapshot\",\"type\":[{\"type\":\"string\",\"connect.version\":1,\"connect.parameters\":{\"allowed\":\"true,last,false\"},\"connect.default\":\"false\",\"connect.name\":\"io.debezium.data.Enum\"},\"null\"],\"default\":\"false\"},{\"name\":\"db\",\"type\":\"string\"},{\"name\":\"table\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"server_id\",\"type\":\"long\"},{\"name\":\"gtid\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"file\",\"type\":\"string\"},{\"name\":\"pos\",\"type\":\"long\"},{\"name\":\"row\",\"type\":\"int\"},{\"name\":\"thread\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"query\",\"type\":[\"null\",\"string\"],\"default\":null}],\"connect.name\":\"io.debezium.connector.mysql.Source\"}},{\"name\":\"op\",\"type\":\"string\"},{\"name\":\"ts_ms\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"transaction\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"ConnectDefault\",\"namespace\":\"io.confluent.connect.avro\",\"fields\":[{\"name\":\"id\",\"type\":\"string\"},{\"name\":\"total_order\",\"type\":\"long\"},{\"name\":\"data_collection_order\",\"type\":\"long\"}]}],\"default\":null}],\"connect.name\":\"fullfillment.test1.person.Envelope\"}"); + + @Test + public void testSerializationDeserialization() throws Exception { + + RowType rowTypeDe = DebeziumAvroDeserializationSchema.createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + RowType rowTypeSe = DebeziumAvroSerializationSchema.createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + + AvroRowDataSerializationSchema serializer = getSerializationSchema(rowTypeSe, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); + DebeziumAvroSerializationSchema debeziumAvroSerializationSchema = new DebeziumAvroSerializationSchema(serializer); + serializer.open(null); + + byte[] serialize = debeziumAvroSerializationSchema.serialize(debeziumRow2RowData()); + + client.register(SUBJECT, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); + AvroRowDataDeserializationSchema deserializer = getDeserializationSchema(rowTypeDe, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); + DebeziumAvroDeserializationSchema debeziumAvroDeserializationSchema = new DebeziumAvroDeserializationSchema(rowTypeDe, InternalTypeInfo.of(rowType), deserializer); + + deserializer.open(null); + SimpleCollector collector = new SimpleCollector(); + + debeziumAvroDeserializationSchema.deserialize(serialize, collector); + + List actual = collector.list.stream() + .map(Object::toString) + .collect(Collectors.toList()); + + List expected = Arrays.asList( + "+I(107,rocks,box of assorted rocks,5.3)"); + assertEquals(expected, actual); + } + + @Test + public void testInsertDataDeserialization() throws Exception { + List actual = testDeserialization(getPath("debezium-avro-insert.txt")); + + List expected = Arrays.asList( + "+I(1,lisi,test debezium avro data,21.799999237060547)"); + assertEquals(expected, actual); + } + + @Test + public void testUpdateDataDeserialization() throws Exception { + List actual = testDeserialization(getPath("debezium-avro-update.txt")); + + List expected = Arrays.asList( + "-U(1,lisi,test debezium avro data,21.799999237060547)", + "+U(1,zhangsan,test debezium avro data,21.799999237060547)"); + assertEquals(expected, actual); + } + + @Test + public void testDeleteDataDeserialization() throws Exception { + List actual = testDeserialization(getPath("debezium-avro-delete.txt")); + + List expected = Arrays.asList( + "-D(1,zhangsan,test debezium avro data,21.799999237060547)"); + assertEquals(expected, actual); + } + + public List testDeserialization(Path path) throws Exception { + RowType rowTypeDe = DebeziumAvroDeserializationSchema + .createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + + client.register(SUBJECT, DEBEZIUM_SCHEMA_COMPATIBLE_TEST, 1, 81); + + AvroRowDataDeserializationSchema deserializer = getDeserializationSchema(rowTypeDe, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); + DebeziumAvroDeserializationSchema debeziumAvroDeserializationSchema = new DebeziumAvroDeserializationSchema(rowTypeDe, InternalTypeInfo.of(rowType), deserializer); + + deserializer.open(null); + SimpleCollector collector = new SimpleCollector(); + byte[] bytes = FileUtils.readAllBytes(path); + + debeziumAvroDeserializationSchema.deserialize(bytes, collector); + + List actual = collector.list.stream() + .map(Object::toString) + .collect(Collectors.toList()); + + return actual; + } + + private Path getPath(String filePath) { + URL url = DebeziumAvroSerDeSchemaTest.class.getClassLoader().getResource(filePath); + return new File(url.getFile()).toPath(); + } + + private static AvroRowDataDeserializationSchema getDeserializationSchema(RowType rowType, Schema schema) { + + ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); + + return new AvroRowDataDeserializationSchema( + new RegistryAvroDeserializationSchema<>( + GenericRecord.class, + AvroSchemaConverter.convertToSchema(rowType), + () -> registryCoder + ), + AvroToRowDataConverters.createRowConverter(rowType), + InternalTypeInfo.of(rowType)); + } + + private static AvroRowDataSerializationSchema getSerializationSchema(RowType rowType, Schema schema) { + + ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); + return new AvroRowDataSerializationSchema( + rowType, + new RegistryAvroSerializationSchema<>( + GenericRecord.class, + AvroSchemaConverter.convertToSchema(rowType), + () -> registryCoder), + RowDataToAvroConverters.createRowConverter(rowType)); + } + + private static RowData debeziumRow2RowData() { + GenericRowData rowData = new GenericRowData(4); + rowData.setField(0, 107L); + rowData.setField(1, new BinaryStringData("rocks")); + rowData.setField(2, new BinaryStringData("box of assorted rocks")); + rowData.setField(3, 5.3D); + return rowData; + } + + private static class SimpleCollector implements Collector { + + private List list = new ArrayList<>(); + + @Override + public void collect(RowData record) { + list.add(record); + } + + @Override + public void close() { + // do nothing + } + } +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-delete.txt b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-delete.txt new file mode 100644 index 0000000000000000000000000000000000000000..84e3ba6586792feb46d12704b1e3104e7043f5c3 GIT binary patch literal 147 zcmZQzU|TEHO-ilGEX`F&EGx=aNJ%V7WCE%@aORAu z1A~~Mo)HkbW#%R3aOGAO7v@N$mFDE6W#;7MrskC_SpM~XUn~PxT4GLdDi_E!LnfYr l)S}}2JSGMOuzKC3%sf2<5HLM6O^Sgjfq^N7Y1c-GJ^*2#FjD{k literal 0 HcmV?d00001 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-insert.txt b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-insert.txt new file mode 100644 index 0000000000000000000000000000000000000000..342f2da150dec5b31738a04707775a58c0ae30ca GIT binary patch literal 143 zcmZQzU|lJ8TEHO-ilGEX`F&EGx=aNJ%V7WCE%>aORAu15l+v zRYqc7dU0YNVbx-WdPYF#mYJ8B! Date: Sat, 7 Nov 2020 14:28:54 +0800 Subject: [PATCH 109/322] [FLINK-18774][debezium-avro] Improve debezium-avro format implementation --- .../confluent/RegistryAvroFormatFactory.java | 2 +- .../DebeziumAvroDeserializationSchema.java | 47 +++-- .../debezium/DebeziumAvroFormatFactory.java | 21 +- .../DebeziumAvroSerializationSchema.java | 60 +++--- .../RegistryAvroFormatFactoryTest.java | 2 +- .../RegistryAvroRowDataSeDeSchemaTest.java | 2 +- .../DebeziumAvroFormatFactoryTest.java | 52 +++-- .../debezium/DebeziumAvroSerDeSchemaTest.java | 107 ++++++---- ...o-delete.txt => debezium-avro-delete.avro} | Bin ...o-insert.txt => debezium-avro-insert.avro} | Bin ...o-update.txt => debezium-avro-update.avro} | Bin .../test/resources/debezium-test-schema.json | 191 ++++++++++++++++++ .../debezium/DebeziumJsonFormatFactory.java | 9 + .../DebeziumJsonFormatFactoryTest.java | 41 +++- 14 files changed, 406 insertions(+), 128 deletions(-) rename flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/{debezium-avro-delete.txt => debezium-avro-delete.avro} (100%) rename flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/{debezium-avro-insert.txt => debezium-avro-insert.avro} (100%) rename flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/{debezium-avro-update.txt => debezium-avro-update.avro} (100%) create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-test-schema.json diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index 8b3c2bcaf..2701c0049 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -114,7 +114,7 @@ public SerializationSchema createRuntimeEncoder( subject.get(), AvroSchemaConverter.convertToSchema(rowType), schemaRegistryURL), - RowDataToAvroConverters.createRowConverter(rowType)); + RowDataToAvroConverters.createConverter(rowType)); } @Override diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java index 5c21a62a5..a493b7983 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.avro.registry.confluent.debezium; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; @@ -82,38 +83,37 @@ public final class DebeziumAvroDeserializationSchema implements DeserializationS /** * TypeInformation of the produced {@link RowData}. **/ - private final TypeInformation resultTypeInfo; - - /** - * Debezium Avro data rowType. - */ - private final RowType rowType; + private final TypeInformation producedTypeInfo; public DebeziumAvroDeserializationSchema( RowType rowType, - TypeInformation resultTypeInfo, + TypeInformation producedTypeInfo, String schemaRegistryUrl) { - this.resultTypeInfo = resultTypeInfo; - this.rowType = rowType; - RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + this.producedTypeInfo = producedTypeInfo; + RowType debeziumAvroRowType = createDebeziumAvroRowType( + fromLogicalToDataType(rowType)); this.avroDeserializer = new AvroRowDataDeserializationSchema( ConfluentRegistryAvroDeserializationSchema.forGeneric( AvroSchemaConverter.convertToSchema(debeziumAvroRowType), schemaRegistryUrl), AvroToRowDataConverters.createRowConverter(debeziumAvroRowType), - resultTypeInfo); + producedTypeInfo); } - public DebeziumAvroDeserializationSchema( - RowType rowType, - TypeInformation resultTypeInfo, + @VisibleForTesting + DebeziumAvroDeserializationSchema( + TypeInformation producedTypeInfo, AvroRowDataDeserializationSchema avroDeserializer) { - this.rowType = rowType; - this.resultTypeInfo = resultTypeInfo; + this.producedTypeInfo = producedTypeInfo; this.avroDeserializer = avroDeserializer; } + @Override + public void open(InitializationContext context) throws Exception { + avroDeserializer.open(context); + } + @Override public RowData deserialize(byte[] message) throws IOException { throw new RuntimeException( @@ -156,8 +156,7 @@ public void deserialize(byte[] message, Collector out) throws IOExcepti } } catch (Throwable t) { // a big try catch to protect the processing. - throw new IOException(format( - "Corrupt Debezium Avro message '%s'.", new String(message)), t); + throw new IOException("Can't deserialize Debezium Avro message.", t); } } @@ -168,7 +167,7 @@ public boolean isEndOfStream(RowData nextElement) { @Override public TypeInformation getProducedType() { - return resultTypeInfo; + return producedTypeInfo; } @Override @@ -181,20 +180,20 @@ public boolean equals(Object o) { } DebeziumAvroDeserializationSchema that = (DebeziumAvroDeserializationSchema) o; return Objects.equals(avroDeserializer, that.avroDeserializer) && - Objects.equals(resultTypeInfo, that.resultTypeInfo); + Objects.equals(producedTypeInfo, that.producedTypeInfo); } @Override public int hashCode() { - return Objects.hash(avroDeserializer, resultTypeInfo); + return Objects.hash(avroDeserializer, producedTypeInfo); } - public static RowType createDebeziumAvroRowType(DataType dataType) { + public static RowType createDebeziumAvroRowType(DataType databaseSchema) { // Debezium Avro contains other information, e.g. "source", "ts_ms" // but we don't need them return (RowType) DataTypes.ROW( - DataTypes.FIELD("before", dataType.nullable()), - DataTypes.FIELD("after", dataType.nullable()), + DataTypes.FIELD("before", databaseSchema.nullable()), + DataTypes.FIELD("after", databaseSchema.nullable()), DataTypes.FIELD("op", DataTypes.STRING())).getLogicalType(); } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java index d5d522ac0..494108c18 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java @@ -52,7 +52,6 @@ public class DebeziumAvroFormatFactory implements DeserializationFormatFactory, public static final String IDENTIFIER = "debezium-avro-confluent"; - @SuppressWarnings("unchecked") @Override public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, @@ -67,11 +66,11 @@ public DeserializationSchema createRuntimeDecoder( DynamicTableSource.Context context, DataType producedDataType) { final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation rowDataTypeInfo = + final TypeInformation producedTypeInfo = context.createTypeInformation(producedDataType); return new DebeziumAvroDeserializationSchema( rowType, - rowDataTypeInfo, + producedTypeInfo, schemaRegistryURL); } @@ -89,16 +88,19 @@ public ChangelogMode getChangelogMode() { @Override public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { + DynamicTableFactory.Context context, + ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); Optional subject = formatOptions.getOptional(SCHEMA_REGISTRY_SUBJECT); if (!subject.isPresent()) { - throw new ValidationException(String.format("Option %s.%s is required for serialization", - IDENTIFIER, SCHEMA_REGISTRY_SUBJECT.key())); + throw new ValidationException(String.format( + "Option '%s.%s' is required for serialization", + IDENTIFIER, + SCHEMA_REGISTRY_SUBJECT.key())); } + return new EncodingFormat>() { @Override public ChangelogMode getChangelogMode() { @@ -115,7 +117,10 @@ public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new DebeziumAvroSerializationSchema(rowType, schemaRegistryURL, subject.get()); + return new DebeziumAvroSerializationSchema( + rowType, + schemaRegistryURL, + subject.get()); } }; } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java index 6c561c0dc..baf8efa6c 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.avro.registry.confluent.debezium; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; import org.apache.flink.formats.avro.RowDataToAvroConverters; @@ -29,10 +30,10 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; import java.util.Objects; +import static java.lang.String.format; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** @@ -44,7 +45,7 @@ public class DebeziumAvroSerializationSchema implements SerializationSchema registryCoder), - RowDataToAvroConverters.createRowConverter(rowType)); + RowDataToAvroConverters.createConverter(rowType)); } private static AvroRowDataDeserializationSchema getDeserializationSchema( diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java index ab280ffb6..fc4440c86 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java @@ -45,6 +45,8 @@ import java.util.Map; import static junit.framework.TestCase.assertEquals; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.junit.Assert.assertThat; /** * Tests for {@link DebeziumAvroFormatFactory}. @@ -68,35 +70,19 @@ public class DebeziumAvroFormatFactoryTest extends TestLogger { public void testSeDeSchema() { final Map options = getAllOptions(); - final DebeziumAvroDeserializationSchema expectedDeser = new DebeziumAvroDeserializationSchema( + DebeziumAvroDeserializationSchema expectedDeser = new DebeziumAvroDeserializationSchema( ROW_TYPE, InternalTypeInfo.of(ROW_TYPE), REGISTRY_URL); - - final DynamicTableSource actualSource = createTableSource(options); - assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - DeserializationSchema actualDeser = scanSourceMock.valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, - SCHEMA.toRowDataType()); - + DeserializationSchema actualDeser = createDeserializationSchema(options); assertEquals(expectedDeser, actualDeser); - final DebeziumAvroSerializationSchema expectedSer = new DebeziumAvroSerializationSchema( + + DebeziumAvroSerializationSchema expectedSer = new DebeziumAvroSerializationSchema( ROW_TYPE, REGISTRY_URL, SUBJECT ); - final DynamicTableSink actualSink = createTableSink(options); - assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = sinkMock.valueFormat - .createRuntimeEncoder( - new SinkRuntimeProviderContext(false), - SCHEMA.toRowDataType()); - + SerializationSchema actualSer = createSerializationSchema(options); Assert.assertEquals(expectedSer, actualSer); } @@ -112,6 +98,30 @@ private Map getAllOptions() { return options; } + private static DeserializationSchema createDeserializationSchema(Map options) { + final DynamicTableSource actualSource = createTableSource(options); + assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + return scanSourceMock.valueFormat + .createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toRowDataType()); + } + + private static SerializationSchema createSerializationSchema(Map options) { + final DynamicTableSink actualSink = createTableSink(options); + assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + return sinkMock.valueFormat + .createRuntimeEncoder( + new SinkRuntimeProviderContext(false), + SCHEMA.toRowDataType()); + } + private static DynamicTableSource createTableSource(Map options) { return FactoryUtil.createTableSource( null, diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java index d87796ae9..f3cc96083 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java @@ -1,5 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.flink.formats.avro.registry.confluent.debezium; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; import org.apache.flink.formats.avro.AvroToRowDataConverters; @@ -10,7 +30,7 @@ import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.binary.BinaryStringData; +import org.apache.flink.table.data.StringData; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; @@ -23,10 +43,12 @@ import org.junit.Test; import java.io.File; +import java.io.IOException; import java.net.URL; import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -37,6 +59,7 @@ import static org.apache.flink.table.api.DataTypes.STRING; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; /** * Tests for {@link DebeziumAvroDeserializationSchema}. @@ -45,8 +68,6 @@ public class DebeziumAvroSerDeSchemaTest { private static final String SUBJECT = "testDebeziumAvro"; - private static SchemaRegistryClient client = new MockSchemaRegistryClient(); - private static final RowType rowType = (RowType) ROW( FIELD("id", BIGINT()), FIELD("name", STRING()), @@ -54,7 +75,11 @@ public class DebeziumAvroSerDeSchemaTest { FIELD("weight", DOUBLE()) ).getLogicalType(); - private static final Schema DEBEZIUM_SCHEMA_COMPATIBLE_TEST = new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"Envelope\",\"namespace\":\"fullfillment.test1.person\",\"fields\":[{\"name\":\"before\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"Value\",\"fields\":[{\"name\":\"id\",\"type\":\"long\"},{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"description\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"weight\",\"type\":[\"null\",\"double\"],\"default\":null}],\"connect.name\":\"fullfillment.test1.person.Value\"}],\"default\":null},{\"name\":\"after\",\"type\":[\"null\",\"Value\"],\"default\":null},{\"name\":\"source\",\"type\":{\"type\":\"record\",\"name\":\"Source\",\"namespace\":\"io.debezium.connector.mysql\",\"fields\":[{\"name\":\"version\",\"type\":\"string\"},{\"name\":\"connector\",\"type\":\"string\"},{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"ts_ms\",\"type\":\"long\"},{\"name\":\"snapshot\",\"type\":[{\"type\":\"string\",\"connect.version\":1,\"connect.parameters\":{\"allowed\":\"true,last,false\"},\"connect.default\":\"false\",\"connect.name\":\"io.debezium.data.Enum\"},\"null\"],\"default\":\"false\"},{\"name\":\"db\",\"type\":\"string\"},{\"name\":\"table\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"server_id\",\"type\":\"long\"},{\"name\":\"gtid\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"file\",\"type\":\"string\"},{\"name\":\"pos\",\"type\":\"long\"},{\"name\":\"row\",\"type\":\"int\"},{\"name\":\"thread\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"query\",\"type\":[\"null\",\"string\"],\"default\":null}],\"connect.name\":\"io.debezium.connector.mysql.Source\"}},{\"name\":\"op\",\"type\":\"string\"},{\"name\":\"ts_ms\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"transaction\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"ConnectDefault\",\"namespace\":\"io.confluent.connect.avro\",\"fields\":[{\"name\":\"id\",\"type\":\"string\"},{\"name\":\"total_order\",\"type\":\"long\"},{\"name\":\"data_collection_order\",\"type\":\"long\"}]}],\"default\":null}],\"connect.name\":\"fullfillment.test1.person.Envelope\"}"); + private static final Schema DEBEZIUM_SCHEMA_COMPATIBLE_TEST = new Schema.Parser().parse( + new String(readBytesFromFile("debezium-test-schema.json")) + ); + + private SchemaRegistryClient client = new MockSchemaRegistryClient(); @Test public void testSerializationDeserialization() throws Exception { @@ -62,42 +87,41 @@ public void testSerializationDeserialization() throws Exception { RowType rowTypeDe = DebeziumAvroDeserializationSchema.createDebeziumAvroRowType(fromLogicalToDataType(rowType)); RowType rowTypeSe = DebeziumAvroSerializationSchema.createDebeziumAvroRowType(fromLogicalToDataType(rowType)); - AvroRowDataSerializationSchema serializer = getSerializationSchema(rowTypeSe, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); - DebeziumAvroSerializationSchema debeziumAvroSerializationSchema = new DebeziumAvroSerializationSchema(serializer); - serializer.open(null); + DebeziumAvroSerializationSchema dbzSerializer = new DebeziumAvroSerializationSchema(getSerializationSchema(rowTypeSe)); + dbzSerializer.open(mock(SerializationSchema.InitializationContext.class)); - byte[] serialize = debeziumAvroSerializationSchema.serialize(debeziumRow2RowData()); + byte[] serialize = dbzSerializer.serialize(debeziumRow2RowData()); client.register(SUBJECT, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); - AvroRowDataDeserializationSchema deserializer = getDeserializationSchema(rowTypeDe, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); - DebeziumAvroDeserializationSchema debeziumAvroDeserializationSchema = new DebeziumAvroDeserializationSchema(rowTypeDe, InternalTypeInfo.of(rowType), deserializer); + DebeziumAvroDeserializationSchema dbzDeserializer = new DebeziumAvroDeserializationSchema( + InternalTypeInfo.of(rowType), + getDeserializationSchema(rowTypeDe)); + dbzDeserializer.open(mock(DeserializationSchema.InitializationContext.class)); - deserializer.open(null); SimpleCollector collector = new SimpleCollector(); - - debeziumAvroDeserializationSchema.deserialize(serialize, collector); + dbzDeserializer.deserialize(serialize, collector); List actual = collector.list.stream() .map(Object::toString) .collect(Collectors.toList()); - List expected = Arrays.asList( + List expected = Collections.singletonList( "+I(107,rocks,box of assorted rocks,5.3)"); assertEquals(expected, actual); } @Test public void testInsertDataDeserialization() throws Exception { - List actual = testDeserialization(getPath("debezium-avro-insert.txt")); + List actual = testDeserialization("debezium-avro-insert.avro"); - List expected = Arrays.asList( + List expected = Collections.singletonList( "+I(1,lisi,test debezium avro data,21.799999237060547)"); assertEquals(expected, actual); } @Test public void testUpdateDataDeserialization() throws Exception { - List actual = testDeserialization(getPath("debezium-avro-update.txt")); + List actual = testDeserialization("debezium-avro-update.avro"); List expected = Arrays.asList( "-U(1,lisi,test debezium avro data,21.799999237060547)", @@ -107,43 +131,35 @@ public void testUpdateDataDeserialization() throws Exception { @Test public void testDeleteDataDeserialization() throws Exception { - List actual = testDeserialization(getPath("debezium-avro-delete.txt")); + List actual = testDeserialization("debezium-avro-delete.avro"); - List expected = Arrays.asList( + List expected = Collections.singletonList( "-D(1,zhangsan,test debezium avro data,21.799999237060547)"); assertEquals(expected, actual); } - public List testDeserialization(Path path) throws Exception { + public List testDeserialization(String dataPath) throws Exception { RowType rowTypeDe = DebeziumAvroDeserializationSchema .createDebeziumAvroRowType(fromLogicalToDataType(rowType)); client.register(SUBJECT, DEBEZIUM_SCHEMA_COMPATIBLE_TEST, 1, 81); - AvroRowDataDeserializationSchema deserializer = getDeserializationSchema(rowTypeDe, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); - DebeziumAvroDeserializationSchema debeziumAvroDeserializationSchema = new DebeziumAvroDeserializationSchema(rowTypeDe, InternalTypeInfo.of(rowType), deserializer); + DebeziumAvroDeserializationSchema dbzDeserializer = new DebeziumAvroDeserializationSchema( + InternalTypeInfo.of(rowType), + getDeserializationSchema(rowTypeDe)); + dbzDeserializer.open(mock(DeserializationSchema.InitializationContext.class)); - deserializer.open(null); SimpleCollector collector = new SimpleCollector(); - byte[] bytes = FileUtils.readAllBytes(path); - - debeziumAvroDeserializationSchema.deserialize(bytes, collector); + dbzDeserializer.deserialize(readBytesFromFile(dataPath), collector); - List actual = collector.list.stream() + return collector.list.stream() .map(Object::toString) .collect(Collectors.toList()); - - return actual; - } - - private Path getPath(String filePath) { - URL url = DebeziumAvroSerDeSchemaTest.class.getClassLoader().getResource(filePath); - return new File(url.getFile()).toPath(); } - private static AvroRowDataDeserializationSchema getDeserializationSchema(RowType rowType, Schema schema) { + private AvroRowDataDeserializationSchema getDeserializationSchema(RowType rowType) { - ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); + final ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); return new AvroRowDataDeserializationSchema( new RegistryAvroDeserializationSchema<>( @@ -155,7 +171,7 @@ private static AvroRowDataDeserializationSchema getDeserializationSchema(RowType InternalTypeInfo.of(rowType)); } - private static AvroRowDataSerializationSchema getSerializationSchema(RowType rowType, Schema schema) { + private AvroRowDataSerializationSchema getSerializationSchema(RowType rowType) { ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); return new AvroRowDataSerializationSchema( @@ -164,18 +180,29 @@ private static AvroRowDataSerializationSchema getSerializationSchema(RowType row GenericRecord.class, AvroSchemaConverter.convertToSchema(rowType), () -> registryCoder), - RowDataToAvroConverters.createRowConverter(rowType)); + RowDataToAvroConverters.createConverter(rowType)); } private static RowData debeziumRow2RowData() { GenericRowData rowData = new GenericRowData(4); rowData.setField(0, 107L); - rowData.setField(1, new BinaryStringData("rocks")); - rowData.setField(2, new BinaryStringData("box of assorted rocks")); + rowData.setField(1, StringData.fromString("rocks")); + rowData.setField(2, StringData.fromString("box of assorted rocks")); rowData.setField(3, 5.3D); return rowData; } + private static byte[] readBytesFromFile(String filePath) { + try { + URL url = DebeziumAvroSerDeSchemaTest.class.getClassLoader().getResource(filePath); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return FileUtils.readAllBytes(path); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + private static class SimpleCollector implements Collector { private List list = new ArrayList<>(); diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-delete.txt b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-delete.avro similarity index 100% rename from flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-delete.txt rename to flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-delete.avro diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-insert.txt b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-insert.avro similarity index 100% rename from flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-insert.txt rename to flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-insert.avro diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-update.txt b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-update.avro similarity index 100% rename from flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-update.txt rename to flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-update.avro diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-test-schema.json b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-test-schema.json new file mode 100644 index 000000000..9b1ba86d0 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-test-schema.json @@ -0,0 +1,191 @@ +{ + "type": "record", + "name": "Envelope", + "namespace": "fullfillment.test1.person", + "fields": [ + { + "name": "before", + "type": [ + "null", + { + "type": "record", + "name": "Value", + "fields": [ + { + "name": "id", + "type": "long" + }, + { + "name": "name", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "description", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "weight", + "type": [ + "null", + "double" + ], + "default": null + } + ], + "connect.name": "fullfillment.test1.person.Value" + } + ], + "default": null + }, + { + "name": "after", + "type": [ + "null", + "Value" + ], + "default": null + }, + { + "name": "source", + "type": { + "type": "record", + "name": "Source", + "namespace": "io.debezium.connector.mysql", + "fields": [ + { + "name": "version", + "type": "string" + }, + { + "name": "connector", + "type": "string" + }, + { + "name": "name", + "type": "string" + }, + { + "name": "ts_ms", + "type": "long" + }, + { + "name": "snapshot", + "type": [ + { + "type": "string", + "connect.version": 1, + "connect.parameters": { + "allowed": "true,last,false" + }, + "connect.default": "false", + "connect.name": "io.debezium.data.Enum" + }, + "null" + ], + "default": "false" + }, + { + "name": "db", + "type": "string" + }, + { + "name": "table", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "server_id", + "type": "long" + }, + { + "name": "gtid", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "file", + "type": "string" + }, + { + "name": "pos", + "type": "long" + }, + { + "name": "row", + "type": "int" + }, + { + "name": "thread", + "type": [ + "null", + "long" + ], + "default": null + }, + { + "name": "query", + "type": [ + "null", + "string" + ], + "default": null + } + ], + "connect.name": "io.debezium.connector.mysql.Source" + } + }, + { + "name": "op", + "type": "string" + }, + { + "name": "ts_ms", + "type": [ + "null", + "long" + ], + "default": null + }, + { + "name": "transaction", + "type": [ + "null", + { + "type": "record", + "name": "ConnectDefault", + "namespace": "io.confluent.connect.avro", + "fields": [ + { + "name": "id", + "type": "string" + }, + { + "name": "total_order", + "type": "long" + }, + { + "name": "data_collection_order", + "type": "long" + } + ] + } + ], + "default": null + } + ], + "connect.name": "fullfillment.test1.person.Envelope" +} \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java index 63171ad66..4c1bab592 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; @@ -82,8 +83,16 @@ public DecodingFormat> createDecodingFormat( public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + if (formatOptions.get(SCHEMA_INCLUDE)) { + throw new ValidationException(String.format( + "Debezium JSON serialization doesn't support '%s.%s' option been set to true.", + IDENTIFIER, + SCHEMA_INCLUDE.key() + )); + } return new EncodingFormat>() { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index ae343c195..459332a1a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -49,6 +49,7 @@ import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; /** * Tests for {@link DebeziumJsonFormatFactory}. @@ -71,7 +72,7 @@ public void testSeDeSchema() { PHYSICAL_DATA_TYPE, Collections.emptyList(), InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - true, + false, true, TimestampFormat.ISO_8601); @@ -116,6 +117,43 @@ public void testInvalidIgnoreParseError() { createTableSource(options); } + @Test + public void testSchemaIncludeOption() { + Map options = getAllOptions(); + options.put("debezium-json.schema-include", "true"); + + final DebeziumJsonDeserializationSchema expectedDeser = new DebeziumJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), + true, + true, + TimestampFormat.ISO_8601); + final DynamicTableSource actualSource = createTableSource(options); + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + DeserializationSchema actualDeser = scanSourceMock.valueFormat + .createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); + assertEquals(expectedDeser, actualDeser); + + try { + final DynamicTableSink actualSink = createTableSink(options); + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + // should fail + sinkMock.valueFormat.createRuntimeEncoder( + new SinkRuntimeProviderContext(false), + PHYSICAL_DATA_TYPE); + fail(); + } catch (Exception e) { + assertEquals( + e.getCause().getCause().getMessage(), + "Debezium JSON serialization doesn't support " + + "'debezium-json.schema-include' option been set to true."); + } + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -139,7 +177,6 @@ private Map getAllOptions() { options.put("format", "debezium-json"); options.put("debezium-json.ignore-parse-errors", "true"); - options.put("debezium-json.schema-include", "true"); options.put("debezium-json.timestamp-format.standard", "ISO-8601"); return options; } From aa4a291c306a06f3f9b7865fd57b4b3807d40b61 Mon Sep 17 00:00:00 2001 From: wangxlong Date: Wed, 11 Nov 2020 10:58:04 +0800 Subject: [PATCH 110/322] [FLINK-19912][json] Fix JSON format fails to serialize map value with null keys This closes #13972 --- .../flink/formats/json/JsonFormatFactory.java | 39 +++---- .../flink/formats/json/JsonOptions.java | 106 +++++++++++++++++- .../json/JsonRowDataSerializationSchema.java | 27 ++++- .../formats/json/RowDataToJsonConverters.java | 36 +++++- .../json/canal/CanalJsonFormatFactory.java | 41 +++---- .../formats/json/canal/CanalJsonOptions.java | 69 ++++++++++++ .../canal/CanalJsonSerializationSchema.java | 11 +- .../debezium/DebeziumJsonFormatFactory.java | 39 +++---- .../json/debezium/DebeziumJsonOptions.java | 77 +++++++++++++ .../DebeziumJsonSerializationSchema.java | 13 ++- .../maxwell/MaxwellJsonFormatFactory.java | 26 ++++- .../json/maxwell/MaxwellJsonOptions.java | 55 +++++++++ .../MaxwellJsonSerializationSchema.java | 13 ++- .../formats/json/JsonFormatFactoryTest.java | 28 ++++- .../json/JsonRowDataSerDeSchemaTest.java | 83 +++++++++++++- .../canal/CanalJsonFormatFactoryTest.java | 32 +++++- .../json/canal/CanalJsonSerDeSchemaTest.java | 5 +- .../DebeziumJsonFormatFactoryTest.java | 34 +++++- .../debezium/DebeziumJsonSerDeSchemaTest.java | 6 +- .../maxwell/MaxwellJsonFormatFactoryTest.java | 28 ++++- .../json/maxwell/MaxwellJsonSerDerTest.java | 5 +- 21 files changed, 666 insertions(+), 107 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index 1fe5da128..17fb69e53 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; @@ -43,8 +42,11 @@ import static org.apache.flink.formats.json.JsonOptions.FAIL_ON_MISSING_FIELD; import static org.apache.flink.formats.json.JsonOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.json.JsonOptions.MAP_NULL_KEY_LITERAL; +import static org.apache.flink.formats.json.JsonOptions.MAP_NULL_KEY_MODE; import static org.apache.flink.formats.json.JsonOptions.TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.JsonOptions.TIMESTAMP_FORMAT_ENUM; +import static org.apache.flink.formats.json.JsonOptions.validateDecodingFormatOptions; +import static org.apache.flink.formats.json.JsonOptions.validateEncodingFormatOptions; /** * Table format factory for providing configured instances of JSON to RowData @@ -61,7 +63,7 @@ public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - validateFormatOptions(formatOptions); + validateDecodingFormatOptions(formatOptions); final boolean failOnMissingField = formatOptions.get(FAIL_ON_MISSING_FIELD); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); @@ -96,8 +98,11 @@ public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); + validateEncodingFormatOptions(formatOptions); TimestampFormat timestampOption = JsonOptions.getTimestampFormat(formatOptions); + JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + String mapNullKeyLiteral = formatOptions.get(MAP_NULL_KEY_LITERAL); return new EncodingFormat>() { @Override @@ -105,7 +110,11 @@ public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new JsonRowDataSerializationSchema(rowType, timestampOption); + return new JsonRowDataSerializationSchema( + rowType, + timestampOption, + mapNullKeyMode, + mapNullKeyLiteral); } @Override @@ -131,26 +140,8 @@ public Set> optionalOptions() { options.add(FAIL_ON_MISSING_FIELD); options.add(IGNORE_PARSE_ERRORS); options.add(TIMESTAMP_FORMAT); + options.add(MAP_NULL_KEY_MODE); + options.add(MAP_NULL_KEY_LITERAL); return options; } - - // ------------------------------------------------------------------------ - // Validation - // ------------------------------------------------------------------------ - - static void validateFormatOptions(ReadableConfig tableOptions) { - boolean failOnMissingField = tableOptions.get(FAIL_ON_MISSING_FIELD); - boolean ignoreParseErrors = tableOptions.get(IGNORE_PARSE_ERRORS); - String timestampFormat = tableOptions.get(TIMESTAMP_FORMAT); - if (ignoreParseErrors && failOnMissingField) { - throw new ValidationException(FAIL_ON_MISSING_FIELD.key() - + " and " - + IGNORE_PARSE_ERRORS.key() - + " shouldn't both be true."); - } - if (!TIMESTAMP_FORMAT_ENUM.contains(timestampFormat)){ - throw new ValidationException(String.format("Unsupported value '%s' for %s. Supported values are [SQL, ISO-8601].", - timestampFormat, TIMESTAMP_FORMAT.key())); - } - } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java index 6fc726baf..037027a58 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java @@ -22,10 +22,13 @@ import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.ValidationException; import java.util.Arrays; import java.util.HashSet; +import java.util.Objects; import java.util.Set; +import java.util.stream.Collectors; /** * This class holds configuration constants used by json format. @@ -36,14 +39,28 @@ public class JsonOptions { .key("fail-on-missing-field") .booleanType() .defaultValue(false) - .withDescription("Optional flag to specify whether to fail if a field is missing or not, false by default"); + .withDescription("Optional flag to specify whether to fail if a field is missing or not, false by default."); public static final ConfigOption IGNORE_PARSE_ERRORS = ConfigOptions .key("ignore-parse-errors") .booleanType() .defaultValue(false) .withDescription("Optional flag to skip fields and rows with parse errors instead of failing;\n" - + "fields are set to null in case of errors, false by default"); + + "fields are set to null in case of errors, false by default."); + + public static final ConfigOption MAP_NULL_KEY_MODE = ConfigOptions + .key("map-null-key.mode") + .stringType() + .defaultValue("FAIL") + .withDescription("Optional flag to control the handling mode when serializing null key for map data, FAIL by default." + + " Option DROP will drop null key entries for map data." + + " Option LITERAL will use 'map-null-key.literal' as key literal."); + + public static final ConfigOption MAP_NULL_KEY_LITERAL = ConfigOptions + .key("map-null-key.literal") + .stringType() + .defaultValue("null") + .withDescription("Optional flag to specify string literal for null keys when 'map-null-key.mode' is LITERAL, \"null\" by default."); public static final ConfigOption TIMESTAMP_FORMAT = ConfigOptions .key("timestamp-format.standard") @@ -65,6 +82,11 @@ public class JsonOptions { ISO_8601 )); + // The handling mode of null key for map data + public static final String JSON_MAP_NULL_KEY_MODE_FAIL = "FAIL"; + public static final String JSON_MAP_NULL_KEY_MODE_DROP = "DROP"; + public static final String JSON_MAP_NULL_KEY_MODE_LITERAL = "LITERAL"; + // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- @@ -81,4 +103,84 @@ public static TimestampFormat getTimestampFormat(ReadableConfig config){ String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); } } + + /** + * Creates handling mode for null key map data. + * + *

See {@link #JSON_MAP_NULL_KEY_MODE_FAIL}, {@link #JSON_MAP_NULL_KEY_MODE_DROP}, + * and {@link #JSON_MAP_NULL_KEY_MODE_LITERAL} for more information. + */ + public static MapNullKeyMode getMapNullKeyMode(ReadableConfig config){ + String mapNullKeyMode = config.get(MAP_NULL_KEY_MODE); + switch (mapNullKeyMode.toUpperCase()){ + case JSON_MAP_NULL_KEY_MODE_FAIL: + return MapNullKeyMode.FAIL; + case JSON_MAP_NULL_KEY_MODE_DROP: + return MapNullKeyMode.DROP; + case JSON_MAP_NULL_KEY_MODE_LITERAL: + return MapNullKeyMode.LITERAL; + default: + throw new TableException( + String.format("Unsupported map null key handling mode '%s'. Validator should have checked that.", mapNullKeyMode)); + } + } + + // -------------------------------------------------------------------------------------------- + // Inner classes + // -------------------------------------------------------------------------------------------- + + /** Handling mode for map data with null key. */ + public enum MapNullKeyMode { + FAIL, + DROP, + LITERAL + } + + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- + + /** + * Validator for json decoding format. + */ + public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + boolean failOnMissingField = tableOptions.get(FAIL_ON_MISSING_FIELD); + boolean ignoreParseErrors = tableOptions.get(IGNORE_PARSE_ERRORS); + if (ignoreParseErrors && failOnMissingField) { + throw new ValidationException(FAIL_ON_MISSING_FIELD.key() + + " and " + + IGNORE_PARSE_ERRORS.key() + + " shouldn't both be true."); + } + validateTimestampFormat(tableOptions); + } + + /** + * Validator for json encoding format. + */ + public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + // validator for {@link MAP_NULL_KEY_MODE} + Set nullKeyModes = Arrays.stream(MapNullKeyMode.values()) + .map(Objects::toString) + .collect(Collectors.toSet()); + if (!nullKeyModes.contains(tableOptions.get(MAP_NULL_KEY_MODE).toUpperCase())){ + throw new ValidationException(String.format( + "Unsupported value '%s' for option %s. Supported values are %s.", + tableOptions.get(MAP_NULL_KEY_MODE), + MAP_NULL_KEY_MODE.key(), + nullKeyModes)); + } + validateTimestampFormat(tableOptions); + } + + /** + * Validates timestamp format which value should be SQL or ISO-8601. + */ + static void validateTimestampFormat(ReadableConfig tableOptions) { + String timestampFormat = tableOptions.get(TIMESTAMP_FORMAT); + if (!TIMESTAMP_FORMAT_ENUM.contains(timestampFormat)){ + throw new ValidationException(String.format("Unsupported value '%s' for %s. Supported values are [SQL, ISO-8601].", + timestampFormat, TIMESTAMP_FORMAT.key())); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index 19bf510c8..6b1bae632 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -55,10 +55,23 @@ public class JsonRowDataSerializationSchema implements SerializationSchema IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; - - public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; - - public static final ConfigOption DATABASE_INCLUDE = ConfigOptions - .key("database.include") - .stringType() - .noDefaultValue() - .withDescription("Only read changelog rows which match the specific database (by comparing the \"database\" meta field in the record)."); - - public static final ConfigOption TABLE_INCLUDE = ConfigOptions - .key("table.include") - .stringType() - .noDefaultValue() - .withDescription("Only read changelog rows which match the specific table (by comparing the \"table\" meta field in the record)."); - @Override public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); + validateDecodingFormatOptions(formatOptions); + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); String database = formatOptions.getOptional(DATABASE_INCLUDE).orElse(null); @@ -111,7 +105,11 @@ public EncodingFormat> createEncodingFormat( ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); + validateEncodingFormatOptions(formatOptions); + TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); return new EncodingFormat>() { @Override @@ -128,8 +126,11 @@ public ChangelogMode getChangelogMode() { public SerializationSchema createRuntimeEncoder(DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); return new CanalJsonSerializationSchema( - rowType, - timestampFormat); + rowType, + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral + ); } }; @@ -152,6 +153,8 @@ public Set> optionalOptions() { options.add(TIMESTAMP_FORMAT); options.add(DATABASE_INCLUDE); options.add(TABLE_INCLUDE); + options.add(JSON_MAP_NULL_KEY_MODE); + options.add(JSON_MAP_NULL_KEY_LITERAL); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java new file mode 100644 index 000000000..a4ce7a586 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.canal; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.json.JsonOptions; + +/** + * Option utils for canal-json format. + */ +public class CanalJsonOptions { + + public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + + public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + + public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; + + public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = JsonOptions.MAP_NULL_KEY_LITERAL; + + public static final ConfigOption DATABASE_INCLUDE = ConfigOptions + .key("database.include") + .stringType() + .noDefaultValue() + .withDescription("Only read changelog rows which match the specific database (by comparing the \"database\" meta field in the record)."); + + public static final ConfigOption TABLE_INCLUDE = ConfigOptions + .key("table.include") + .stringType() + .noDefaultValue() + .withDescription("Only read changelog rows which match the specific table (by comparing the \"table\" meta field in the record)."); + + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- + + /** + * Validator for canal decoding format. + */ + public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateDecodingFormatOptions(tableOptions); + } + + /** + * Validator for canal encoding format. + */ + public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateEncodingFormatOptions(tableOptions); + } + +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java index ea5d4e141..1db9660e3 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.json.canal; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.JsonRowDataSerializationSchema; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; @@ -54,10 +55,16 @@ public class CanalJsonSerializationSchema implements SerializationSchema SCHEMA_INCLUDE = ConfigOptions - .key("schema-include") - .booleanType() - .defaultValue(false) - .withDescription("When setting up a Debezium Kafka Connect, users can enable " + - "a Kafka configuration 'value.converter.schemas.enable' to include schema in the message. " + - "This option indicates the Debezium JSON data include the schema in the message or not. " + - "Default is false."); - - public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; - - public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; - @Override public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); + validateDecodingFormatOptions(formatOptions); final boolean schemaInclude = formatOptions.get(SCHEMA_INCLUDE); @@ -85,14 +79,11 @@ public EncodingFormat> createEncodingFormat( ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); + validateEncodingFormatOptions(formatOptions); + TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - if (formatOptions.get(SCHEMA_INCLUDE)) { - throw new ValidationException(String.format( - "Debezium JSON serialization doesn't support '%s.%s' option been set to true.", - IDENTIFIER, - SCHEMA_INCLUDE.key() - )); - } + JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); return new EncodingFormat>() { @@ -109,7 +100,7 @@ public ChangelogMode getChangelogMode() { @Override public SerializationSchema createRuntimeEncoder(DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new DebeziumJsonSerializationSchema(rowType, timestampFormat); + return new DebeziumJsonSerializationSchema(rowType, timestampFormat, mapNullKeyMode, mapNullKeyLiteral); } }; } @@ -130,6 +121,8 @@ public Set> optionalOptions() { options.add(SCHEMA_INCLUDE); options.add(IGNORE_PARSE_ERRORS); options.add(TIMESTAMP_FORMAT); + options.add(JSON_MAP_NULL_KEY_MODE); + options.add(JSON_MAP_NULL_KEY_LITERAL); return options; } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java new file mode 100644 index 000000000..7cafd69cf --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.debezium; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.table.api.ValidationException; + +import static org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory.IDENTIFIER; + +/** + * Option utils for debezium-json format. + */ +public class DebeziumJsonOptions { + + public static final ConfigOption SCHEMA_INCLUDE = ConfigOptions + .key("schema-include") + .booleanType() + .defaultValue(false) + .withDescription("When setting up a Debezium Kafka Connect, users can enable " + + "a Kafka configuration 'value.converter.schemas.enable' to include schema in the message. " + + "This option indicates the Debezium JSON data include the schema in the message or not. " + + "Default is false."); + + public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + + public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + + public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; + + public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = JsonOptions.MAP_NULL_KEY_LITERAL; + + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- + + /** + * Validator for debezium decoding format. + */ + public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateDecodingFormatOptions(tableOptions); + } + + /** + * Validator for debezium encoding format. + */ + public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateEncodingFormatOptions(tableOptions); + + // validator for {@link SCHEMA_INCLUDE} + if (tableOptions.get(SCHEMA_INCLUDE)) { + throw new ValidationException(String.format( + "Debezium JSON serialization doesn't support '%s.%s' option been set to true.", + IDENTIFIER, + SCHEMA_INCLUDE.key() + )); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java index f94c81371..d3defc008 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.json.debezium; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.JsonRowDataSerializationSchema; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; @@ -49,10 +50,16 @@ public class DebeziumJsonSerializationSchema implements SerializationSchema IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; - - public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; - @Override public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); + validateDecodingFormatOptions(formatOptions); + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); @@ -93,7 +98,11 @@ public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); + validateEncodingFormatOptions(formatOptions); + TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); return new EncodingFormat>() { @@ -112,8 +121,11 @@ public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); return new MaxwellJsonSerializationSchema( - rowType, - timestampFormat); + rowType, + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral + ); } }; } @@ -133,6 +145,8 @@ public Set> optionalOptions() { Set> options = new HashSet<>(); options.add(IGNORE_PARSE_ERRORS); options.add(TIMESTAMP_FORMAT); + options.add(JSON_MAP_NULL_KEY_MODE); + options.add(JSON_MAP_NULL_KEY_LITERAL); return options; } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java new file mode 100644 index 000000000..1e68c5479 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.maxwell; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.json.JsonOptions; + +/** + * Option utils for maxwell-json format. + */ +public class MaxwellJsonOptions { + + public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + + public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + + public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; + + public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = JsonOptions.MAP_NULL_KEY_LITERAL; + + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- + + /** + * Validator for maxwell decoding format. + */ + public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateDecodingFormatOptions(tableOptions); + } + + /** + * Validator for maxwell encoding format. + */ + public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateEncodingFormatOptions(tableOptions); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java index be9df5edc..2774f106a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.json.maxwell; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.JsonRowDataSerializationSchema; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; @@ -51,10 +52,16 @@ public class MaxwellJsonSerializationSchema implements SerializationSchema tableOptions = getModifyOptions( + options -> options.put("json.map-null-key.mode", "invalid")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new ValidationException("Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); + testSchemaSerializationSchema(tableOptions); + } + + @Test + public void testLowerCaseOptionForMapNullKeyMode() { + final Map tableOptions = getModifyOptions( + options -> options.put("json.map-null-key.mode", "fail")); + + testSchemaDeserializationSchema(tableOptions); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -137,8 +156,11 @@ private void testSchemaDeserializationSchema(Map options) { } private void testSchemaSerializationSchema(Map options) { - final JsonRowDataSerializationSchema expectedSer = new JsonRowDataSerializationSchema(ROW_TYPE, - TimestampFormat.ISO_8601); + final JsonRowDataSerializationSchema expectedSer = new JsonRowDataSerializationSchema( + ROW_TYPE, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); final DynamicTableSink actualSink = createTableSink(options); assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; @@ -174,6 +196,8 @@ private Map getAllOptions() { options.put("json.fail-on-missing-field", "false"); options.put("json.ignore-parse-errors", "true"); options.put("json.timestamp-format.standard", "ISO-8601"); + options.put("json.map-null-key.mode", "LITERAL"); + options.put("json.map-null-key.literal", "null"); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index bba9cc1d2..0008187df 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -19,7 +19,10 @@ package org.apache.flink.formats.json; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.util.DataFormatConverters; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.DataType; @@ -42,6 +45,7 @@ import java.time.ZoneOffset; import java.util.Arrays; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Random; @@ -182,7 +186,12 @@ public void testSerDe() throws Exception { assertEquals(expected, actual); // test serialization - JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(schema, TimestampFormat.ISO_8601); + JsonRowDataSerializationSchema serializationSchema = + new JsonRowDataSerializationSchema( + schema, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); byte[] actualBytes = serializationSchema.serialize(rowData); assertEquals(new String(serializedJson), new String(actualBytes)); @@ -256,7 +265,9 @@ public void testSerDeMultiRows() throws Exception { JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.ISO_8601); - JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.ISO_8601); + JsonRowDataSerializationSchema serializationSchema = + new JsonRowDataSerializationSchema( + rowType, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, "null"); ObjectMapper objectMapper = new ObjectMapper(); @@ -326,7 +337,9 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( rowType, InternalTypeInfo.of(rowType), false, true, TimestampFormat.ISO_8601); - JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.ISO_8601); + JsonRowDataSerializationSchema serializationSchema = + new JsonRowDataSerializationSchema( + rowType, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, "null"); for (int i = 0; i < jsons.length; i++) { String json = jsons[i]; @@ -386,7 +399,7 @@ public void testDeserializationMissingNode() throws Exception { } @Test - public void testSerDeSQLTimestampFormat() throws Exception{ + public void testSerDeSQLTimestampFormat() throws Exception { RowType rowType = (RowType) ROW( FIELD("timestamp3", TIMESTAMP(3)), FIELD("timestamp9", TIMESTAMP(9)), @@ -396,7 +409,9 @@ public void testSerDeSQLTimestampFormat() throws Exception{ JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); - JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema(rowType, TimestampFormat.SQL); + JsonRowDataSerializationSchema serializationSchema = + new JsonRowDataSerializationSchema( + rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, "null"); ObjectMapper objectMapper = new ObjectMapper(); @@ -411,6 +426,64 @@ public void testSerDeSQLTimestampFormat() throws Exception{ assertEquals(new String(serializedJson), new String(actual)); } + @Test + public void testSerializationMapNullKey() throws Exception { + RowType rowType = (RowType) ROW( + FIELD("nestedMap", MAP(STRING(), MAP(STRING(), INT()))) + ).getLogicalType(); + + // test data + // use LinkedHashMap to make sure entries order + Map map = new LinkedHashMap<>(); + map.put(StringData.fromString("no-null key"), 1); + map.put(StringData.fromString(null), 2); + GenericMapData mapData = new GenericMapData(map); + + Map nestedMap = new LinkedHashMap<>(); + nestedMap.put(StringData.fromString("no-null key"), mapData); + nestedMap.put(StringData.fromString(null), mapData); + + GenericMapData nestedMapData = new GenericMapData(nestedMap); + GenericRowData rowData = new GenericRowData(1); + rowData.setField(0, nestedMapData); + + JsonRowDataSerializationSchema serializationSchema1 = + new JsonRowDataSerializationSchema( + rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.FAIL, "null"); + // expect message for serializationSchema1 + String errorMessage1 = "JSON format doesn't support to serialize map data with null keys." + + " You can drop null key entries or encode null in literals by specifying map-null-key.mode option."; + + JsonRowDataSerializationSchema serializationSchema2 = + new JsonRowDataSerializationSchema( + rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.DROP, "null"); + // expect result for serializationSchema2 + String expectResult2 = "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1}}}"; + + JsonRowDataSerializationSchema serializationSchema3 = + new JsonRowDataSerializationSchema( + rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, "nullKey"); + // expect result for serializationSchema3 + String expectResult3 = + "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1,\"nullKey\":2},\"nullKey\":{\"no-null key\":1,\"nullKey\":2}}}"; + + try { + // throw exception when mapNullKey Mode is fail + serializationSchema1.serialize(rowData); + Assert.fail("expecting exception message: " + errorMessage1); + } catch (Throwable t) { + assertEquals(errorMessage1, t.getCause().getMessage()); + } + + // mapNullKey Mode is drop + byte[] actual2 = serializationSchema2.serialize(rowData); + assertEquals(expectResult2, new String(actual2)); + + // mapNullKey Mode is literal + byte[] actual3 = serializationSchema3.serialize(rowData); + assertEquals(expectResult3, new String(actual3)); + } + @Test public void testJsonParse() throws Exception { for (TestSpec spec : testData) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 8beb8eabc..a914098d9 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -21,9 +21,11 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -81,7 +83,9 @@ public void testDefaultOptions() { // test Ser CanalJsonSerializationSchema expectedSer = new CanalJsonSerializationSchema( ROW_TYPE, - TimestampFormat.SQL); + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.FAIL, + "null"); SerializationSchema actualSer = createSerializationSchema(options); assertEquals(expectedSer, actualSer); } @@ -93,6 +97,8 @@ public void testUserDefinedOptions() { options.put("canal-json.timestamp-format.standard", "ISO-8601"); options.put("canal-json.database.include", "mydb"); options.put("canal-json.table.include", "mytable"); + options.put("canal-json.map-null-key.mode", "LITERAL"); + options.put("canal-json.map-null-key.literal", "nullKey"); // test Deser CanalJsonDeserializationSchema expectedDeser = CanalJsonDeserializationSchema @@ -108,7 +114,9 @@ public void testUserDefinedOptions() { // test Ser CanalJsonSerializationSchema expectedSer = new CanalJsonSerializationSchema( ROW_TYPE, - TimestampFormat.ISO_8601); + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "nullKey"); SerializationSchema actualSer = createSerializationSchema(options); assertEquals(expectedSer, actualSer); } @@ -124,6 +132,26 @@ public void testInvalidIgnoreParseError() { createDeserializationSchema(options); } + @Test + public void testInvalidOptionForTimestampFormat() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("canal-json.timestamp-format.standard", "test")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new ValidationException("Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + createDeserializationSchema(tableOptions); + } + + @Test + public void testInvalidOptionForMapNullKeyMode() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("canal-json.map-null-key.mode", "invalid")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new ValidationException("Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); + createSerializationSchema(tableOptions); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index e8f3401af..6d3b76cda 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json.canal; +import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; @@ -154,7 +155,9 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali // test Serialization CanalJsonSerializationSchema serializationSchema = new CanalJsonSerializationSchema( SCHEMA, - TimestampFormat.ISO_8601); + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); serializationSchema.open(null); List result = new ArrayList<>(); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 459332a1a..48081bafb 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -21,9 +21,11 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -90,8 +92,10 @@ public void testSeDeSchema() { assertEquals(expectedDeser, actualDeser); final DebeziumJsonSerializationSchema expectedSer = new DebeziumJsonSerializationSchema( - (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), - TimestampFormat.ISO_8601); + (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); final DynamicTableSink actualSink = createTableSink(options); assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; @@ -143,8 +147,8 @@ public void testSchemaIncludeOption() { (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; // should fail sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), - PHYSICAL_DATA_TYPE); + new SinkRuntimeProviderContext(false), + PHYSICAL_DATA_TYPE); fail(); } catch (Exception e) { assertEquals( @@ -154,6 +158,26 @@ public void testSchemaIncludeOption() { } } + @Test + public void testInvalidOptionForTimestampFormat() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("debezium-json.timestamp-format.standard", "test")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new ValidationException("Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + createTableSource(tableOptions); + } + + @Test + public void testInvalidOptionForMapNullKeyMode() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("debezium-json.map-null-key.mode", "invalid")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new ValidationException("Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); + createTableSink(tableOptions); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -178,6 +202,8 @@ private Map getAllOptions() { options.put("format", "debezium-json"); options.put("debezium-json.ignore-parse-errors", "true"); options.put("debezium-json.timestamp-format.standard", "ISO-8601"); + options.put("debezium-json.map-null-key.mode", "LITERAL"); + options.put("debezium-json.map-null-key.literal", "null"); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java index 00e6ffbcd..c8c1ef2f8 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json.debezium; +import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.formats.json.debezium.DebeziumJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; @@ -247,7 +248,10 @@ private void testSerializationDeserialization(String resourceFile, boolean schem DebeziumJsonSerializationSchema serializationSchema = new DebeziumJsonSerializationSchema( (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), - TimestampFormat.SQL); + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); + serializationSchema.open(null); actual = new ArrayList<>(); for (RowData rowData : collector.list) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java index 569954a00..395d5248e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java @@ -21,9 +21,11 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -73,7 +75,9 @@ public void testSeDeSchema() { final MaxwellJsonSerializationSchema expectedSer = new MaxwellJsonSerializationSchema( ROW_TYPE, - TimestampFormat.ISO_8601); + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); final Map options = getAllOptions(); @@ -112,6 +116,26 @@ public void testInvalidIgnoreParseError() { createTableSource(options); } + @Test + public void testInvalidOptionForTimestampFormat() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("maxwell-json.timestamp-format.standard", "test")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new ValidationException("Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + createTableSource(tableOptions); + } + + @Test + public void testInvalidOptionForMapNullKeyMode() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("maxwell-json.map-null-key.mode", "invalid")); + + thrown.expect(ValidationException.class); + thrown.expect(containsCause(new ValidationException("Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); + createTableSink(tableOptions); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -136,6 +160,8 @@ private Map getAllOptions() { options.put("format", "maxwell-json"); options.put("maxwell-json.ignore-parse-errors", "true"); options.put("maxwell-json.timestamp-format.standard", "ISO-8601"); + options.put("maxwell-json.map-null-key.mode", "LITERAL"); + options.put("maxwell-json.map-null-key.literal", "null"); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java index cab2b61bd..fef5ede23 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json.maxwell; +import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; @@ -133,7 +134,9 @@ public void testSerializationDeserialization() throws Exception { MaxwellJsonSerializationSchema serializationSchema = new MaxwellJsonSerializationSchema( SCHEMA, - TimestampFormat.SQL); + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); serializationSchema.open(null); List result = new ArrayList<>(); for (RowData rowData : collector.list) { From 48f6d292b735a6b382390b1cbd6ebd38223a6e01 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 16 Nov 2020 11:07:15 +0100 Subject: [PATCH 111/322] [FLINK-20147][json] Replace lambdas with classes to prevent serialization issues This closes #14081. --- .../debezium/DebeziumJsonDecodingFormat.java | 104 ++++++++++++------ .../DebeziumJsonDeserializationSchema.java | 18 ++- 2 files changed, 82 insertions(+), 40 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java index 9590e22d5..25623f118 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java @@ -135,74 +135,106 @@ public ChangelogMode getChangelogMode() { */ enum ReadableMetadata { SCHEMA( - "schema", - DataTypes.STRING().nullable(), - false, - DataTypes.FIELD("schema", DataTypes.STRING()), - GenericRowData::getString + "schema", + DataTypes.STRING().nullable(), + false, + DataTypes.FIELD("schema", DataTypes.STRING()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + @Override + public Object convert(GenericRowData row, int pos) { + return row.getString(pos); + } + } ), INGESTION_TIMESTAMP( - "ingestion-timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), - true, - DataTypes.FIELD("ts_ms", DataTypes.BIGINT()), - (row, pos) -> { + "ingestion-timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + true, + DataTypes.FIELD("ts_ms", DataTypes.BIGINT()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + @Override + public Object convert(GenericRowData row, int pos) { return TimestampData.fromEpochMillis(row.getLong(pos)); } + } ), SOURCE_TIMESTAMP( - "source.timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - (row, pos) -> { + "source.timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + @Override + public Object convert(GenericRowData row, int pos) { final StringData timestamp = (StringData) readProperty(row, pos, KEY_SOURCE_TIMESTAMP); if (timestamp == null) { return null; } return TimestampData.fromEpochMillis(Long.parseLong(timestamp.toString())); } + } ), SOURCE_DATABASE( - "source.database", - DataTypes.STRING().nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - (row, pos) -> { + "source.database", + DataTypes.STRING().nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + @Override + public Object convert(GenericRowData row, int pos) { return readProperty(row, pos, KEY_SOURCE_DATABASE); } + } ), SOURCE_SCHEMA( - "source.schema", - DataTypes.STRING().nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - (row, pos) -> { + "source.schema", + DataTypes.STRING().nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + @Override + public Object convert(GenericRowData row, int pos) { return readProperty(row, pos, KEY_SOURCE_SCHEMA); } + } ), SOURCE_TABLE( - "source.table", - DataTypes.STRING().nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - (row, pos) -> { + "source.table", + DataTypes.STRING().nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + @Override + public Object convert(GenericRowData row, int pos) { return readProperty(row, pos, KEY_SOURCE_TABLE); } + } ), SOURCE_PROPERTIES( - "source.properties", - // key and value of the map are nullable to make handling easier in queries - DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()).nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - GenericRowData::getMap + "source.properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()).nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + @Override + public Object convert(GenericRowData row, int pos) { + return row.getMap(pos); + } + } ); final String key; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java index 43bd94efa..2fe5ee49d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java @@ -284,7 +284,13 @@ private static MetadataConverter convertInRoot( RowType jsonRowType, ReadableMetadata metadata) { final int pos = findFieldPos(metadata, jsonRowType); - return (root, unused) -> metadata.converter.convert(root, pos); + return new MetadataConverter() { + private static final long serialVersionUID = 1L; + @Override + public Object convert(GenericRowData root, int unused) { + return metadata.converter.convert(root, pos); + } + }; } private static MetadataConverter convertInPayload( @@ -293,9 +299,13 @@ private static MetadataConverter convertInPayload( boolean schemaInclude) { if (schemaInclude) { final int pos = findFieldPos(metadata, (RowType) jsonRowType.getChildren().get(0)); - return (root, unused) -> { - final GenericRowData payload = (GenericRowData) root.getField(0); - return metadata.converter.convert(payload, pos); + return new MetadataConverter() { + private static final long serialVersionUID = 1L; + @Override + public Object convert(GenericRowData root, int unused) { + final GenericRowData payload = (GenericRowData) root.getField(0); + return metadata.converter.convert(payload, pos); + } }; } return convertInRoot(jsonRowType, metadata); From ea1af30d7a091a17256d104f0c1b8653ba7e088a Mon Sep 17 00:00:00 2001 From: caozhen Date: Fri, 20 Nov 2020 17:17:13 +0800 Subject: [PATCH 112/322] [FLINK-20170][json] Fix JSON format loses precision when deserializing decimals This closes #14134 --- .../formats/json/JsonRowDataDeserializationSchema.java | 7 +++++++ .../formats/json/JsonRowDeserializationSchema.java | 10 ++++++++++ .../flink/formats/json/JsonRowDataSerDeSchemaTest.java | 7 ++++++- .../formats/json/JsonRowDeserializationSchemaTest.java | 7 +++++-- 4 files changed, 28 insertions(+), 3 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index b7c5d0732..78a324716 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -22,8 +22,11 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -88,6 +91,10 @@ public JsonRowDataDeserializationSchema( this.runtimeConverter = new JsonToRowDataConverters(failOnMissingField, ignoreParseErrors, timestampFormat) .createRowConverter(checkNotNull(rowType)); this.timestampFormat = timestampFormat; + boolean hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t instanceof DecimalType); + if (hasDecimalType) { + objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); + } } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index f9a6395df..acb96a514 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -27,9 +27,12 @@ import org.apache.flink.api.java.typeutils.MapTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; import org.apache.flink.types.Row; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; @@ -63,6 +66,8 @@ import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL; +import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -106,6 +111,11 @@ private JsonRowDeserializationSchema( this.failOnMissingField = failOnMissingField; this.runtimeConverter = createConverter(this.typeInfo); this.ignoreParseErrors = ignoreParseErrors; + RowType rowType = (RowType) fromLegacyInfoToDataType(this.typeInfo).getLogicalType(); + boolean hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t.getTypeRoot().equals(DECIMAL)); + if (hasDecimalType) { + objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); + } } /** diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 0008187df..d324e8500 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -664,7 +664,12 @@ private void testParseErrors(TestSpec spec) throws Exception { TestSpec .json("{\"id\":\"2019-11-12T18:00:12+0800\"}") .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0)))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12+0800\"}'.") + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12+0800\"}'."), + + TestSpec + .json("{\"id\":1,\"factor\":799.929496989092949698}") + .rowType(ROW(FIELD("id", INT()), FIELD("factor", DECIMAL(38, 18)))) + .expect(Row.of(1, new BigDecimal("799.929496989092949698"))) ); private static Map createHashMap(String k1, Integer v1, String k2, Integer v2) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index 438a18c20..88512906c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -394,9 +394,12 @@ private void testParseErrors(TestSpec spec) { .json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") .typeInfo(Types.ROW_NAMED(new String[]{"map"}, Types.MAP(Types.STRING, Types.INT))) .expect(Row.of(createHashMap("key1", 123, "key2", null))) - .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'") - + .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'"), + TestSpec + .json("{\"id\":1,\"factor\":799.929496989092949698}") + .typeInfo(Types.ROW_NAMED(new String[]{"id", "factor"}, Types.INT, Types.BIG_DEC)) + .expect(Row.of(1, new BigDecimal("799.929496989092949698"))) ); private static Map createHashMap(String k1, Integer v1, String k2, Integer v2) { From 7113361d4128c8265af4bf96db7bd7ef36d367b8 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 20 Nov 2020 15:47:52 +0100 Subject: [PATCH 113/322] [hotfix][json] Consider nullability for Debezium ingestion timestamp --- .../formats/json/debezium/DebeziumJsonDecodingFormat.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java index 25623f118..ab3203f5c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java @@ -150,13 +150,16 @@ public Object convert(GenericRowData row, int pos) { INGESTION_TIMESTAMP( "ingestion-timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), true, DataTypes.FIELD("ts_ms", DataTypes.BIGINT()), new MetadataConverter() { private static final long serialVersionUID = 1L; @Override public Object convert(GenericRowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } return TimestampData.fromEpochMillis(row.getLong(pos)); } } From fb84e6226025c69acc20c3f95fecd077a3d199d3 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 25 Nov 2020 14:37:10 +0800 Subject: [PATCH 114/322] [FLINK-20295][table][fs-connector] Table File Source lost data when reading from directories with JSON format This closes #14192 --- .../json/JsonBatchFileSystemITCase.java | 52 +++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java index 7d17dbd1d..7ba344c02 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java @@ -18,13 +18,17 @@ package org.apache.flink.formats.json; +import org.apache.flink.table.api.TableResult; import org.apache.flink.table.planner.runtime.batch.sql.BatchFileSystemITCaseBase; import org.apache.flink.types.Row; import org.apache.flink.util.FileUtils; +import org.junit.Assert; import org.junit.Test; import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; import java.net.URI; import java.util.ArrayList; import java.util.Arrays; @@ -59,4 +63,52 @@ public void testParseError() throws Exception { Row.of("x5,5,1,1"), Row.of("x5,5,1,1"))); } + + @Test + public void bigDataTest() throws IOException { + int numRecords = 1000; + File dir = generateTestData(numRecords); + + env().setParallelism(1); + + String sql = String.format( + "CREATE TABLE bigdata_source ( " + + " id INT, " + + " content STRING" + + ") PARTITIONED by (id) WITH (" + + " 'connector' = 'filesystem'," + + " 'path' = '%s'," + + " 'format' = 'json'" + + ")", dir); + tEnv().executeSql(sql); + TableResult result = tEnv().executeSql("select * from bigdata_source"); + List elements = new ArrayList<>(); + result.collect().forEachRemaining(r -> elements.add((String) r.getField(1))); + Assert.assertEquals(numRecords, elements.size()); + elements.sort(String::compareTo); + + List expected = new ArrayList<>(); + for (int i = 0; i < numRecords; i++) { + expected.add(String.valueOf(i)); + } + expected.sort(String::compareTo); + + Assert.assertEquals(expected, elements); + } + + private static File generateTestData(int numRecords) throws IOException { + File tempDir = TEMPORARY_FOLDER.newFolder(); + + File root = new File(tempDir, "id=0"); + root.mkdir(); + + File dataFile = new File(root, "testdata"); + try (PrintWriter writer = new PrintWriter(dataFile)) { + for (int i = 0; i < numRecords; ++i) { + writer.println(String.format("{\"content\":\"%s\"}", i)); + } + } + + return tempDir; + } } From 45fd0b4047aca645d8b715f54aef8c36f6eb2fa6 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 20 Nov 2020 13:11:28 +0100 Subject: [PATCH 115/322] [hotfix] Shade avro in sql-avro jars --- .../flink-sql-avro-confluent-registry/pom.xml | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index aac58240f..b31988269 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -75,17 +75,27 @@ under the License. + + org.apache.kafka + org.apache.flink.avro.registry.confluent.shaded.org.apache.kafka + + + io.confluent + org.apache.flink.avro.registry.confluent.shaded.io.confluent + + com.fasterxml.jackson - org.apache.flink.formats.avro.registry.confluent.shaded.com.fasterxml.jackson + org.apache.flink.avro.shaded.com.fasterxml.jackson - org.apache.commons.compress - org.apache.flink.formats.avro.registry.confluent.shaded.org.apache.commons.compress + org.apache.avro + org.apache.flink.avro.shaded.org.apache.avro - org.apache.kafka - org.apache.flink.formats.avro.registry.confluent.shaded.org.apache.kafka + org.apache.commons.compress + org.apache.flink.avro.shaded.org.apache.commons.compress From 3b61f33544e845ba0e958ad867ad5b69895fbec7 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 26 Nov 2020 14:20:51 +0800 Subject: [PATCH 116/322] Update version to 1.13-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 1b43ecd96..9b706c61d 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.12-SNAPSHOT + 1.13-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 3dac36deb..fb1fdeb85 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.12-SNAPSHOT + 1.13-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 81424eb57..f7c653f14 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.12-SNAPSHOT + 1.13-SNAPSHOT .. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index b31988269..3e718b7e7 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.12-SNAPSHOT + 1.13-SNAPSHOT .. From 93ce72932cea0baf4afd13fc8f4289dba65b1030 Mon Sep 17 00:00:00 2001 From: zhuxiaoshang <31235759+zhuxiaoshang@users.noreply.github.com> Date: Wed, 9 Dec 2020 13:35:48 +0800 Subject: [PATCH 117/322] [FLINK-20470][json] MissingNode can't be casted to ObjectNode when deserializing JSON This closes #14316 --- .../json/JsonRowDataDeserializationSchema.java | 2 +- .../flink/formats/json/JsonToRowDataConverters.java | 4 ++-- .../formats/json/JsonRowDataSerDeSchemaTest.java | 11 +++++++++++ 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index 78a324716..0a2fb5ef1 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -89,7 +89,7 @@ public JsonRowDataDeserializationSchema( this.failOnMissingField = failOnMissingField; this.ignoreParseErrors = ignoreParseErrors; this.runtimeConverter = new JsonToRowDataConverters(failOnMissingField, ignoreParseErrors, timestampFormat) - .createRowConverter(checkNotNull(rowType)); + .createConverter(checkNotNull(rowType)); this.timestampFormat = timestampFormat; boolean hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t instanceof DecimalType); if (hasDecimalType) { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java index 25df3df80..65feec353 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java @@ -106,7 +106,7 @@ public interface JsonToRowDataConverter extends Serializable { /** * Creates a runtime converter which is null safe. */ - private JsonToRowDataConverter createConverter(LogicalType type) { + public JsonToRowDataConverter createConverter(LogicalType type) { return wrapIntoNullableConverter(createNotNullConverter(type)); } @@ -368,7 +368,7 @@ private Object convertField( private JsonToRowDataConverter wrapIntoNullableConverter( JsonToRowDataConverter converter) { return jsonNode -> { - if (jsonNode == null || jsonNode.isNull()) { + if (jsonNode == null || jsonNode.isNull() || jsonNode.isMissingNode()) { return null; } try { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index d324e8500..b49326b2d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -351,6 +351,17 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { @Test public void testDeserializationMissingNode() throws Exception { + DataType dataType = ROW(FIELD("name", STRING())); + RowType schema = (RowType) dataType.getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( + schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); + RowData rowData = deserializationSchema.deserialize("".getBytes()); + assertEquals(null, rowData); + } + + @Test + public void testDeserializationMissingField() throws Exception { ObjectMapper objectMapper = new ObjectMapper(); // Root From 954ef1be5cfd2f72aab5004d2d02efafc9670f0c Mon Sep 17 00:00:00 2001 From: cxiiiiiii Date: Fri, 18 Dec 2020 22:35:51 +0800 Subject: [PATCH 118/322] [FLINK-19880][json] Fix ignore-parse-errors option not work for the legacy JSON format This closes #14415 Co-authored-by: Xiao Cai --- .../flink/formats/json/JsonRowFormatFactory.java | 1 + .../formats/json/JsonRowFormatFactoryTest.java | 16 ++++++++++++++++ 2 files changed, 17 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java index b855d94e7..356594a27 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -50,6 +50,7 @@ protected List supportedFormatProperties() { properties.add(JsonValidator.FORMAT_JSON_SCHEMA); properties.add(JsonValidator.FORMAT_SCHEMA); properties.add(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD); + properties.add(JsonValidator.FORMAT_IGNORE_PARSE_ERRORS); return properties; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java index 6f94094e8..28beacbb8 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -79,6 +79,22 @@ public void testSchema() { testSchemaDeserializationSchema(properties); } + @Test + public void testSchemaIgnoreParseErrors() { + final Map properties = toMap( + new Json() + .schema(SCHEMA) + .ignoreParseErrors(true)); + + testSchemaSerializationSchema(properties); + + final DeserializationSchema actual2 = TableFactoryService + .find(DeserializationSchemaFactory.class, properties) + .createDeserializationSchema(properties); + final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema.Builder(SCHEMA).ignoreParseErrors().build(); + assertEquals(expected2, actual2); + } + @Test public void testJsonSchema() { final Map properties = toMap( From fbf6bf1518b785119cbf30fdf1bfb8620b775d0a Mon Sep 17 00:00:00 2001 From: Rufus Refactor Date: Mon, 28 Dec 2020 14:30:59 +0100 Subject: [PATCH 119/322] [FLINK-20651] Format code with Spotless/google-java-format --- .../test/TestAvroConsumerConfluent.java | 86 +- .../confluent/CachedSchemaCoderProvider.java | 75 +- ...uentRegistryAvroDeserializationSchema.java | 153 +- ...fluentRegistryAvroSerializationSchema.java | 112 +- .../ConfluentSchemaRegistryCoder.java | 102 +- .../confluent/RegistryAvroFormatFactory.java | 178 ++- .../confluent/RegistryAvroOptions.java | 23 +- .../DebeziumAvroDeserializationSchema.java | 292 ++-- .../debezium/DebeziumAvroFormatFactory.java | 187 ++- .../DebeziumAvroSerializationSchema.java | 188 ++- .../ConfluentSchemaRegistryCoderTest.java | 69 +- .../RegistryAvroFormatFactoryTest.java | 239 ++- .../RegistryAvroRowDataSeDeSchemaTest.java | 278 ++-- .../DebeziumAvroFormatFactoryTest.java | 174 ++- .../debezium/DebeziumAvroSerDeSchemaTest.java | 309 ++-- .../flink/formats/json/JsonFormatFactory.java | 178 ++- .../json/JsonNodeDeserializationSchema.java | 12 +- .../flink/formats/json/JsonOptions.java | 306 ++-- .../JsonRowDataDeserializationSchema.java | 186 ++- .../json/JsonRowDataSerializationSchema.java | 148 +- .../json/JsonRowDeserializationSchema.java | 981 ++++++------ .../formats/json/JsonRowFormatFactory.java | 121 +- .../formats/json/JsonRowSchemaConverter.java | 647 ++++---- .../json/JsonRowSerializationSchema.java | 692 ++++----- .../formats/json/JsonToRowDataConverters.java | 657 ++++---- .../formats/json/RowDataToJsonConverters.java | 566 +++---- .../flink/formats/json/TimeFormats.java | 93 +- .../flink/formats/json/TimestampFormat.java | 22 +- .../canal/CanalJsonDeserializationSchema.java | 462 +++--- .../json/canal/CanalJsonFormatFactory.java | 202 ++- .../formats/json/canal/CanalJsonOptions.java | 62 +- .../canal/CanalJsonSerializationSchema.java | 166 ++- .../debezium/DebeziumJsonDecodingFormat.java | 470 +++--- .../DebeziumJsonDeserializationSchema.java | 566 +++---- .../debezium/DebeziumJsonFormatFactory.java | 150 +- .../json/debezium/DebeziumJsonOptions.java | 71 +- .../DebeziumJsonSerializationSchema.java | 143 +- .../MaxwellJsonDeserializationSchema.java | 271 ++-- .../maxwell/MaxwellJsonFormatFactory.java | 187 ++- .../json/maxwell/MaxwellJsonOptions.java | 39 +- .../MaxwellJsonSerializationSchema.java | 157 +- .../apache/flink/table/descriptors/Json.java | 246 ++- .../table/descriptors/JsonValidator.java | 76 +- .../json/JsonBatchFileSystemITCase.java | 150 +- .../json/JsonFileCompactionITCase.java | 12 +- .../formats/json/JsonFormatFactoryTest.java | 355 ++--- .../formats/json/JsonFsStreamSinkITCase.java | 20 +- .../JsonNodeDeserializationSchemaTest.java | 26 +- .../json/JsonRowDataSerDeSchemaTest.java | 1322 +++++++++-------- .../JsonRowDeserializationSchemaTest.java | 768 +++++----- .../json/JsonRowFormatFactoryTest.java | 277 ++-- .../json/JsonRowSchemaConverterTest.java | 190 ++- .../json/JsonRowSerializationSchemaTest.java | 389 ++--- .../canal/CanalJsonFormatFactoryTest.java | 328 ++-- .../json/canal/CanalJsonSerDeSchemaTest.java | 346 ++--- .../DebeziumJsonFileSystemITCase.java | 225 +-- .../DebeziumJsonFormatFactoryTest.java | 352 ++--- .../debezium/DebeziumJsonSerDeSchemaTest.java | 558 +++---- .../maxwell/MaxwellJsonFormatFactoryTest.java | 271 ++-- .../json/maxwell/MaxwellJsonSerDerTest.java | 297 ++-- .../utils/DeserializationSchemaMatcher.java | 235 ++- .../utils/SerializationSchemaMatcher.java | 290 ++-- .../flink/table/descriptors/JsonTest.java | 225 +-- 63 files changed, 8559 insertions(+), 8419 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java index 459a16c9a..840aa9e52 100644 --- a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java +++ b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java @@ -35,52 +35,62 @@ /** * A simple example that shows how to read from and write to Kafka with Confluent Schema Registry. - * This will read AVRO messages from the input topic, parse them into a POJO type via checking the Schema by calling Schema registry. - * Then this example publish the POJO type to kafka by converting the POJO to AVRO and verifying the schema. - * --input-topic test-input --output-string-topic test-output --output-avro-topic test-avro-output --output-subject --bootstrap.servers localhost:9092 --schema-registry-url http://localhost:8081 --group.id myconsumer + * This will read AVRO messages from the input topic, parse them into a POJO type via checking the + * Schema by calling Schema registry. Then this example publish the POJO type to kafka by converting + * the POJO to AVRO and verifying the schema. --input-topic test-input --output-string-topic + * test-output --output-avro-topic test-avro-output --output-subject --bootstrap.servers + * localhost:9092 --schema-registry-url http://localhost:8081 --group.id myconsumer */ public class TestAvroConsumerConfluent { - public static void main(String[] args) throws Exception { - // parse input arguments - final ParameterTool parameterTool = ParameterTool.fromArgs(args); + public static void main(String[] args) throws Exception { + // parse input arguments + final ParameterTool parameterTool = ParameterTool.fromArgs(args); - if (parameterTool.getNumberOfParameters() < 6) { - System.out.println("Missing parameters!\n" + - "Usage: Kafka --input-topic --output-string-topic --output-avro-topic " + - "--bootstrap.servers " + - "--schema-registry-url --group.id "); - return; - } - Properties config = new Properties(); - config.setProperty("bootstrap.servers", parameterTool.getRequired("bootstrap.servers")); - config.setProperty("group.id", parameterTool.getRequired("group.id")); - String schemaRegistryUrl = parameterTool.getRequired("schema-registry-url"); + if (parameterTool.getNumberOfParameters() < 6) { + System.out.println( + "Missing parameters!\n" + + "Usage: Kafka --input-topic --output-string-topic --output-avro-topic " + + "--bootstrap.servers " + + "--schema-registry-url --group.id "); + return; + } + Properties config = new Properties(); + config.setProperty("bootstrap.servers", parameterTool.getRequired("bootstrap.servers")); + config.setProperty("group.id", parameterTool.getRequired("group.id")); + String schemaRegistryUrl = parameterTool.getRequired("schema-registry-url"); - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataStreamSource input = env - .addSource( - new FlinkKafkaConsumer<>( - parameterTool.getRequired("input-topic"), - ConfluentRegistryAvroDeserializationSchema.forSpecific(User.class, schemaRegistryUrl), - config).setStartFromEarliest()); + DataStreamSource input = + env.addSource( + new FlinkKafkaConsumer<>( + parameterTool.getRequired("input-topic"), + ConfluentRegistryAvroDeserializationSchema.forSpecific( + User.class, schemaRegistryUrl), + config) + .setStartFromEarliest()); - SingleOutputStreamOperator mapToString = input - .map((MapFunction) SpecificRecordBase::toString); + SingleOutputStreamOperator mapToString = + input.map((MapFunction) SpecificRecordBase::toString); - FlinkKafkaProducer stringFlinkKafkaProducer = new FlinkKafkaProducer<>( - parameterTool.getRequired("output-string-topic"), - new SimpleStringSchema(), - config); - mapToString.addSink(stringFlinkKafkaProducer); + FlinkKafkaProducer stringFlinkKafkaProducer = + new FlinkKafkaProducer<>( + parameterTool.getRequired("output-string-topic"), + new SimpleStringSchema(), + config); + mapToString.addSink(stringFlinkKafkaProducer); - FlinkKafkaProducer avroFlinkKafkaProducer = new FlinkKafkaProducer<>( - parameterTool.getRequired("output-avro-topic"), - ConfluentRegistryAvroSerializationSchema.forSpecific(User.class, parameterTool.getRequired("output-subject"), schemaRegistryUrl), - config); - input.addSink(avroFlinkKafkaProducer); + FlinkKafkaProducer avroFlinkKafkaProducer = + new FlinkKafkaProducer<>( + parameterTool.getRequired("output-avro-topic"), + ConfluentRegistryAvroSerializationSchema.forSpecific( + User.class, + parameterTool.getRequired("output-subject"), + schemaRegistryUrl), + config); + input.addSink(avroFlinkKafkaProducer); - env.execute("Kafka Confluent Schema Registry AVRO Example"); - } + env.execute("Kafka Confluent Schema Registry AVRO Example"); + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java index a739ae17a..b5f32008d 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java @@ -27,50 +27,49 @@ import java.util.Objects; -/** A {@link SchemaCoder.SchemaCoderProvider} that uses a cached schema registry - * client underlying. **/ +/** + * A {@link SchemaCoder.SchemaCoderProvider} that uses a cached schema registry client underlying. * + */ @Internal class CachedSchemaCoderProvider implements SchemaCoder.SchemaCoderProvider { - private static final long serialVersionUID = 8610401613495438381L; - private final String subject; - private final String url; - private final int identityMapCapacity; + private static final long serialVersionUID = 8610401613495438381L; + private final String subject; + private final String url; + private final int identityMapCapacity; - CachedSchemaCoderProvider(String url, int identityMapCapacity) { - this(null, url, identityMapCapacity); - } + CachedSchemaCoderProvider(String url, int identityMapCapacity) { + this(null, url, identityMapCapacity); + } - CachedSchemaCoderProvider(@Nullable String subject, String url, int identityMapCapacity) { - this.subject = subject; - this.url = Objects.requireNonNull(url); - this.identityMapCapacity = identityMapCapacity; - } + CachedSchemaCoderProvider(@Nullable String subject, String url, int identityMapCapacity) { + this.subject = subject; + this.url = Objects.requireNonNull(url); + this.identityMapCapacity = identityMapCapacity; + } - @Override - public SchemaCoder get() { - return new ConfluentSchemaRegistryCoder(this.subject, - new CachedSchemaRegistryClient( - url, - identityMapCapacity)); - } + @Override + public SchemaCoder get() { + return new ConfluentSchemaRegistryCoder( + this.subject, new CachedSchemaRegistryClient(url, identityMapCapacity)); + } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - CachedSchemaCoderProvider that = (CachedSchemaCoderProvider) o; - return identityMapCapacity == that.identityMapCapacity && - Objects.equals(subject, that.subject) && - url.equals(that.url); - } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CachedSchemaCoderProvider that = (CachedSchemaCoderProvider) o; + return identityMapCapacity == that.identityMapCapacity + && Objects.equals(subject, that.subject) + && url.equals(that.url); + } - @Override - public int hashCode() { - return Objects.hash(subject, url, identityMapCapacity); - } + @Override + public int hashCode() { + return Objects.hash(subject, url, identityMapCapacity); + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java index c5c421e80..7becf5013 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java @@ -29,88 +29,93 @@ import javax.annotation.Nullable; /** - * Deserialization schema that deserializes from Avro binary format using {@link SchemaCoder} that uses - * Confluent Schema Registry. + * Deserialization schema that deserializes from Avro binary format using {@link SchemaCoder} that + * uses Confluent Schema Registry. * * @param type of record it produces */ -public class ConfluentRegistryAvroDeserializationSchema extends RegistryAvroDeserializationSchema { +public class ConfluentRegistryAvroDeserializationSchema + extends RegistryAvroDeserializationSchema { - private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; + private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; - private static final long serialVersionUID = -1671641202177852775L; + private static final long serialVersionUID = -1671641202177852775L; - /** - * Creates a Avro deserialization schema. - * - * @param recordClazz class to which deserialize. Should be either - * {@link SpecificRecord} or {@link GenericRecord}. - * @param reader reader's Avro schema. Should be provided if recordClazz is - * {@link GenericRecord} - * @param schemaCoderProvider provider for schema coder that reads writer schema from Confluent Schema Registry - */ - private ConfluentRegistryAvroDeserializationSchema(Class recordClazz, @Nullable Schema reader, - SchemaCoder.SchemaCoderProvider schemaCoderProvider) { - super(recordClazz, reader, schemaCoderProvider); - } + /** + * Creates a Avro deserialization schema. + * + * @param recordClazz class to which deserialize. Should be either {@link SpecificRecord} or + * {@link GenericRecord}. + * @param reader reader's Avro schema. Should be provided if recordClazz is {@link + * GenericRecord} + * @param schemaCoderProvider provider for schema coder that reads writer schema from Confluent + * Schema Registry + */ + private ConfluentRegistryAvroDeserializationSchema( + Class recordClazz, + @Nullable Schema reader, + SchemaCoder.SchemaCoderProvider schemaCoderProvider) { + super(recordClazz, reader, schemaCoderProvider); + } - /** - * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link GenericRecord} - * using provided reader schema and looks up writer schema in Confluent Schema Registry. - * - * @param schema schema of produced records - * @param url url of schema registry to connect - * @return deserialized record in form of {@link GenericRecord} - */ - public static ConfluentRegistryAvroDeserializationSchema forGeneric(Schema schema, String url) { - return forGeneric(schema, url, DEFAULT_IDENTITY_MAP_CAPACITY); - } + /** + * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link + * GenericRecord} using provided reader schema and looks up writer schema in Confluent Schema + * Registry. + * + * @param schema schema of produced records + * @param url url of schema registry to connect + * @return deserialized record in form of {@link GenericRecord} + */ + public static ConfluentRegistryAvroDeserializationSchema forGeneric( + Schema schema, String url) { + return forGeneric(schema, url, DEFAULT_IDENTITY_MAP_CAPACITY); + } - /** - * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link GenericRecord} - * using provided reader schema and looks up writer schema in Confluent Schema Registry. - * - * @param schema schema of produced records - * @param url url of schema registry to connect - * @param identityMapCapacity maximum number of cached schema versions (default: 1000) - * @return deserialized record in form of {@link GenericRecord} - */ - public static ConfluentRegistryAvroDeserializationSchema forGeneric(Schema schema, String url, - int identityMapCapacity) { - return new ConfluentRegistryAvroDeserializationSchema<>( - GenericRecord.class, - schema, - new CachedSchemaCoderProvider(url, identityMapCapacity)); - } + /** + * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link + * GenericRecord} using provided reader schema and looks up writer schema in Confluent Schema + * Registry. + * + * @param schema schema of produced records + * @param url url of schema registry to connect + * @param identityMapCapacity maximum number of cached schema versions (default: 1000) + * @return deserialized record in form of {@link GenericRecord} + */ + public static ConfluentRegistryAvroDeserializationSchema forGeneric( + Schema schema, String url, int identityMapCapacity) { + return new ConfluentRegistryAvroDeserializationSchema<>( + GenericRecord.class, + schema, + new CachedSchemaCoderProvider(url, identityMapCapacity)); + } - /** - * Creates {@link AvroDeserializationSchema} that produces classes that were generated from avro - * schema and looks up writer schema in Confluent Schema Registry. - * - * @param tClass class of record to be produced - * @param url url of schema registry to connect - * @return deserialized record - */ - public static ConfluentRegistryAvroDeserializationSchema forSpecific(Class tClass, - String url) { - return forSpecific(tClass, url, DEFAULT_IDENTITY_MAP_CAPACITY); - } + /** + * Creates {@link AvroDeserializationSchema} that produces classes that were generated from avro + * schema and looks up writer schema in Confluent Schema Registry. + * + * @param tClass class of record to be produced + * @param url url of schema registry to connect + * @return deserialized record + */ + public static + ConfluentRegistryAvroDeserializationSchema forSpecific(Class tClass, String url) { + return forSpecific(tClass, url, DEFAULT_IDENTITY_MAP_CAPACITY); + } - /** - * Creates {@link AvroDeserializationSchema} that produces classes that were generated from avro - * schema and looks up writer schema in Confluent Schema Registry. - * - * @param tClass class of record to be produced - * @param url url of schema registry to connect - * @param identityMapCapacity maximum number of cached schema versions (default: 1000) - * @return deserialized record - */ - public static ConfluentRegistryAvroDeserializationSchema forSpecific(Class tClass, - String url, int identityMapCapacity) { - return new ConfluentRegistryAvroDeserializationSchema<>( - tClass, - null, - new CachedSchemaCoderProvider(url, identityMapCapacity) - ); - } + /** + * Creates {@link AvroDeserializationSchema} that produces classes that were generated from avro + * schema and looks up writer schema in Confluent Schema Registry. + * + * @param tClass class of record to be produced + * @param url url of schema registry to connect + * @param identityMapCapacity maximum number of cached schema versions (default: 1000) + * @return deserialized record + */ + public static + ConfluentRegistryAvroDeserializationSchema forSpecific( + Class tClass, String url, int identityMapCapacity) { + return new ConfluentRegistryAvroDeserializationSchema<>( + tClass, null, new CachedSchemaCoderProvider(url, identityMapCapacity)); + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java index 0afbd885d..f06193fbd 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java @@ -27,68 +27,68 @@ import org.apache.avro.specific.SpecificRecord; /** - * Serialization schema that serializes to Avro binary format that uses - * Confluent Schema Registry. + * Serialization schema that serializes to Avro binary format that uses Confluent Schema Registry. * * @param the type to be serialized */ -public class ConfluentRegistryAvroSerializationSchema extends RegistryAvroSerializationSchema { +public class ConfluentRegistryAvroSerializationSchema + extends RegistryAvroSerializationSchema { - private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; + private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; - private static final long serialVersionUID = -1771641202177852775L; + private static final long serialVersionUID = -1771641202177852775L; - /** - * Creates a Avro serialization schema. - * - * @param recordClazz class to serialize. Should be either - * {@link SpecificRecord} or {@link GenericRecord}. - * @param schema writer's Avro schema. Should be provided if recordClazz is - * {@link GenericRecord} - * @param schemaCoderProvider provider for schema coder that writes the writer schema to Confluent Schema Registry - */ - private ConfluentRegistryAvroSerializationSchema(Class recordClazz, Schema schema, - SchemaCoder.SchemaCoderProvider schemaCoderProvider) { - super(recordClazz, schema, schemaCoderProvider); - } + /** + * Creates a Avro serialization schema. + * + * @param recordClazz class to serialize. Should be either {@link SpecificRecord} or {@link + * GenericRecord}. + * @param schema writer's Avro schema. Should be provided if recordClazz is {@link + * GenericRecord} + * @param schemaCoderProvider provider for schema coder that writes the writer schema to + * Confluent Schema Registry + */ + private ConfluentRegistryAvroSerializationSchema( + Class recordClazz, + Schema schema, + SchemaCoder.SchemaCoderProvider schemaCoderProvider) { + super(recordClazz, schema, schemaCoderProvider); + } - /** - * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from avro - * schema and writes the writer schema to Confluent Schema Registry. - * - * @param tClass the type to be serialized - * @param subject subject of schema registry to produce - * @param schemaRegistryUrl url of schema registry to connect - * - * @return Serialized record - */ - public static ConfluentRegistryAvroSerializationSchema forSpecific(Class tClass, - String subject, - String schemaRegistryUrl) { - return new ConfluentRegistryAvroSerializationSchema<>( - tClass, - null, - new CachedSchemaCoderProvider(subject, schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY) - ); - } + /** + * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from + * avro schema and writes the writer schema to Confluent Schema Registry. + * + * @param tClass the type to be serialized + * @param subject subject of schema registry to produce + * @param schemaRegistryUrl url of schema registry to connect + * @return Serialized record + */ + public static + ConfluentRegistryAvroSerializationSchema forSpecific( + Class tClass, String subject, String schemaRegistryUrl) { + return new ConfluentRegistryAvroSerializationSchema<>( + tClass, + null, + new CachedSchemaCoderProvider( + subject, schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); + } - /** - * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from avro - * schema and writes the writer schema to Confluent Schema Registry. - * - * @param subject subject of schema registry to produce - * @param schema schema that will be used for serialization - * @param schemaRegistryUrl url of schema registry to connect - * - * @return Serialized record in form of byte array - */ - public static ConfluentRegistryAvroSerializationSchema forGeneric(String subject, - Schema schema, - String schemaRegistryUrl) { - return new ConfluentRegistryAvroSerializationSchema<>( - GenericRecord.class, - schema, - new CachedSchemaCoderProvider(subject, schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY) - ); - } + /** + * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from + * avro schema and writes the writer schema to Confluent Schema Registry. + * + * @param subject subject of schema registry to produce + * @param schema schema that will be used for serialization + * @param schemaRegistryUrl url of schema registry to connect + * @return Serialized record in form of byte array + */ + public static ConfluentRegistryAvroSerializationSchema forGeneric( + String subject, Schema schema, String schemaRegistryUrl) { + return new ConfluentRegistryAvroSerializationSchema<>( + GenericRecord.class, + schema, + new CachedSchemaCoderProvider( + subject, schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java index 36b973ea6..7be314531 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java @@ -32,64 +32,62 @@ import static java.lang.String.format; -/** - * Reads and Writes schema using Confluent Schema Registry protocol. - */ +/** Reads and Writes schema using Confluent Schema Registry protocol. */ public class ConfluentSchemaRegistryCoder implements SchemaCoder { - private final SchemaRegistryClient schemaRegistryClient; - private String subject; - private static final int CONFLUENT_MAGIC_BYTE = 0; - - /** - * Creates {@link SchemaCoder} that uses provided {@link SchemaRegistryClient} to connect to - * schema registry. - * - * @param schemaRegistryClient client to connect schema registry - * @param subject subject of schema registry to produce - */ - public ConfluentSchemaRegistryCoder(String subject, SchemaRegistryClient schemaRegistryClient) { - this.schemaRegistryClient = schemaRegistryClient; - this.subject = subject; - } + private final SchemaRegistryClient schemaRegistryClient; + private String subject; + private static final int CONFLUENT_MAGIC_BYTE = 0; - /** - * Creates {@link SchemaCoder} that uses provided {@link SchemaRegistryClient} to connect to - * schema registry. - * - * @param schemaRegistryClient client to connect schema registry - */ - public ConfluentSchemaRegistryCoder(SchemaRegistryClient schemaRegistryClient) { - this.schemaRegistryClient = schemaRegistryClient; - } + /** + * Creates {@link SchemaCoder} that uses provided {@link SchemaRegistryClient} to connect to + * schema registry. + * + * @param schemaRegistryClient client to connect schema registry + * @param subject subject of schema registry to produce + */ + public ConfluentSchemaRegistryCoder(String subject, SchemaRegistryClient schemaRegistryClient) { + this.schemaRegistryClient = schemaRegistryClient; + this.subject = subject; + } - @Override - public Schema readSchema(InputStream in) throws IOException { - DataInputStream dataInputStream = new DataInputStream(in); + /** + * Creates {@link SchemaCoder} that uses provided {@link SchemaRegistryClient} to connect to + * schema registry. + * + * @param schemaRegistryClient client to connect schema registry + */ + public ConfluentSchemaRegistryCoder(SchemaRegistryClient schemaRegistryClient) { + this.schemaRegistryClient = schemaRegistryClient; + } - if (dataInputStream.readByte() != 0) { - throw new IOException("Unknown data format. Magic number does not match"); - } else { - int schemaId = dataInputStream.readInt(); + @Override + public Schema readSchema(InputStream in) throws IOException { + DataInputStream dataInputStream = new DataInputStream(in); - try { - return schemaRegistryClient.getById(schemaId); - } catch (RestClientException e) { - throw new IOException(format("Could not find schema with id %s in registry", schemaId), e); - } - } - } + if (dataInputStream.readByte() != 0) { + throw new IOException("Unknown data format. Magic number does not match"); + } else { + int schemaId = dataInputStream.readInt(); - @Override - public void writeSchema(Schema schema, OutputStream out) throws IOException { - try { - int registeredId = schemaRegistryClient.register(subject, schema); - out.write(CONFLUENT_MAGIC_BYTE); - byte[] schemaIdBytes = ByteBuffer.allocate(4).putInt(registeredId).array(); - out.write(schemaIdBytes); - } catch (RestClientException e) { - throw new IOException("Could not register schema in registry", e); - } - } + try { + return schemaRegistryClient.getById(schemaId); + } catch (RestClientException e) { + throw new IOException( + format("Could not find schema with id %s in registry", schemaId), e); + } + } + } + @Override + public void writeSchema(Schema schema, OutputStream out) throws IOException { + try { + int registeredId = schemaRegistryClient.register(subject, schema); + out.write(CONFLUENT_MAGIC_BYTE); + byte[] schemaIdBytes = ByteBuffer.allocate(4).putInt(registeredId).array(); + out.write(schemaIdBytes); + } catch (RestClientException e) { + throw new IOException("Could not register schema in registry", e); + } + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index 2701c0049..cff1870c6 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -50,96 +50,92 @@ import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_URL; /** - * Table format factory for providing configured instances of Schema Registry Avro to RowData - * {@link SerializationSchema} and {@link DeserializationSchema}. + * Table format factory for providing configured instances of Schema Registry Avro to RowData {@link + * SerializationSchema} and {@link DeserializationSchema}. */ -public class RegistryAvroFormatFactory implements - DeserializationFormatFactory, - SerializationFormatFactory { - - public static final String IDENTIFIER = "avro-confluent"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - - String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); - return new DecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, - DataType producedDataType) { - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation rowDataTypeInfo = - context.createTypeInformation(producedDataType); - return new AvroRowDataDeserializationSchema( - ConfluentRegistryAvroDeserializationSchema.forGeneric( - AvroSchemaConverter.convertToSchema(rowType), - schemaRegistryURL), - AvroToRowDataConverters.createRowConverter(rowType), - rowDataTypeInfo); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - }; - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - - String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); - Optional subject = formatOptions.getOptional(SCHEMA_REGISTRY_SUBJECT); - if (!subject.isPresent()) { - throw new ValidationException(String.format("Option %s.%s is required for serialization", - IDENTIFIER, SCHEMA_REGISTRY_SUBJECT.key())); - } - - return new EncodingFormat>() { - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, - DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new AvroRowDataSerializationSchema( - rowType, - ConfluentRegistryAvroSerializationSchema.forGeneric( - subject.get(), - AvroSchemaConverter.convertToSchema(rowType), - schemaRegistryURL), - RowDataToAvroConverters.createConverter(rowType)); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - Set> options = new HashSet<>(); - options.add(SCHEMA_REGISTRY_URL); - return options; - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(SCHEMA_REGISTRY_SUBJECT); - return options; - } +public class RegistryAvroFormatFactory + implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "avro-confluent"; + + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + + String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); + return new DecodingFormat>() { + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + context.createTypeInformation(producedDataType); + return new AvroRowDataDeserializationSchema( + ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(rowType), schemaRegistryURL), + AvroToRowDataConverters.createRowConverter(rowType), + rowDataTypeInfo); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + }; + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + + String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); + Optional subject = formatOptions.getOptional(SCHEMA_REGISTRY_SUBJECT); + if (!subject.isPresent()) { + throw new ValidationException( + String.format( + "Option %s.%s is required for serialization", + IDENTIFIER, SCHEMA_REGISTRY_SUBJECT.key())); + } + + return new EncodingFormat>() { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new AvroRowDataSerializationSchema( + rowType, + ConfluentRegistryAvroSerializationSchema.forGeneric( + subject.get(), + AvroSchemaConverter.convertToSchema(rowType), + schemaRegistryURL), + RowDataToAvroConverters.createConverter(rowType)); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + Set> options = new HashSet<>(); + options.add(SCHEMA_REGISTRY_URL); + return options; + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(SCHEMA_REGISTRY_SUBJECT); + return options; + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java index a3739c144..c94ebc19e 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java @@ -23,17 +23,18 @@ /** Options for Schema Registry Avro format. */ public class RegistryAvroOptions { - private RegistryAvroOptions () {} + private RegistryAvroOptions() {} - public static final ConfigOption SCHEMA_REGISTRY_URL = ConfigOptions - .key("schema-registry.url") - .stringType() - .noDefaultValue() - .withDescription("Required URL to connect to schema registry service"); + public static final ConfigOption SCHEMA_REGISTRY_URL = + ConfigOptions.key("schema-registry.url") + .stringType() + .noDefaultValue() + .withDescription("Required URL to connect to schema registry service"); - public static final ConfigOption SCHEMA_REGISTRY_SUBJECT = ConfigOptions - .key("schema-registry.subject") - .stringType() - .noDefaultValue() - .withDescription("Subject name to write to the Schema Registry service, required for sink"); + public static final ConfigOption SCHEMA_REGISTRY_SUBJECT = + ConfigOptions.key("schema-registry.subject") + .stringType() + .noDefaultValue() + .withDescription( + "Subject name to write to the Schema Registry service, required for sink"); } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java index a493b7983..28d253196 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java @@ -41,159 +41,151 @@ import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** - * Deserialization schema from Debezium Avro to Flink Table/SQL internal data structure {@link RowData}. - * The deserialization schema knows Debezium's schema definition and can extract the database data - * and convert into {@link RowData} with {@link RowKind}. - * Deserializes a byte[] message as a JSON object and reads - * the specified fields. - * Failures during deserialization are forwarded as wrapped IOExceptions. + * Deserialization schema from Debezium Avro to Flink Table/SQL internal data structure {@link + * RowData}. The deserialization schema knows Debezium's schema definition and can extract the + * database data and convert into {@link RowData} with {@link RowKind}. Deserializes a byte[] + * message as a JSON object and reads the specified fields. Failures during deserialization + * are forwarded as wrapped IOExceptions. * * @see Debezium */ @Internal public final class DebeziumAvroDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - - /** - * snapshot read. - */ - private static final String OP_READ = "r"; - /** - * insert operation. - */ - private static final String OP_CREATE = "c"; - /** - * update operation. - */ - private static final String OP_UPDATE = "u"; - /** - * delete operation. - */ - private static final String OP_DELETE = "d"; - - private static final String REPLICA_IDENTITY_EXCEPTION = "The \"before\" field of %s message is null, " + - "if you are using Debezium Postgres Connector, " + - "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; - - /** - * The deserializer to deserialize Debezium Avro data. - */ - private final AvroRowDataDeserializationSchema avroDeserializer; - - /** - * TypeInformation of the produced {@link RowData}. - **/ - private final TypeInformation producedTypeInfo; - - public DebeziumAvroDeserializationSchema( - RowType rowType, - TypeInformation producedTypeInfo, - String schemaRegistryUrl) { - this.producedTypeInfo = producedTypeInfo; - RowType debeziumAvroRowType = createDebeziumAvroRowType( - fromLogicalToDataType(rowType)); - - this.avroDeserializer = new AvroRowDataDeserializationSchema( - ConfluentRegistryAvroDeserializationSchema.forGeneric( - AvroSchemaConverter.convertToSchema(debeziumAvroRowType), - schemaRegistryUrl), - AvroToRowDataConverters.createRowConverter(debeziumAvroRowType), - producedTypeInfo); - } - - @VisibleForTesting - DebeziumAvroDeserializationSchema( - TypeInformation producedTypeInfo, - AvroRowDataDeserializationSchema avroDeserializer) { - this.producedTypeInfo = producedTypeInfo; - this.avroDeserializer = avroDeserializer; - } - - @Override - public void open(InitializationContext context) throws Exception { - avroDeserializer.open(context); - } - - @Override - public RowData deserialize(byte[] message) throws IOException { - throw new RuntimeException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); - } - - @Override - public void deserialize(byte[] message, Collector out) throws IOException { - - if (message == null || message.length == 0) { - // skip tombstone messages - return; - } - try { - GenericRowData row = (GenericRowData) avroDeserializer.deserialize(message); - - GenericRowData before = (GenericRowData) row.getField(0); - GenericRowData after = (GenericRowData) row.getField(1); - String op = row.getField(2).toString(); - if (OP_CREATE.equals(op) || OP_READ.equals(op)) { - after.setRowKind(RowKind.INSERT); - out.collect(after); - } else if (OP_UPDATE.equals(op)) { - if (before == null) { - throw new IllegalStateException(String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - before.setRowKind(RowKind.UPDATE_BEFORE); - after.setRowKind(RowKind.UPDATE_AFTER); - out.collect(before); - out.collect(after); - } else if (OP_DELETE.equals(op)) { - if (before == null) { - throw new IllegalStateException(String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); - } - before.setRowKind(RowKind.DELETE); - out.collect(before); - } else { - throw new IOException(format( - "Unknown \"op\" value \"%s\". The Debezium Avro message is '%s'", op, new String(message))); - } - } catch (Throwable t) { - // a big try catch to protect the processing. - throw new IOException("Can't deserialize Debezium Avro message.", t); - } - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return producedTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DebeziumAvroDeserializationSchema that = (DebeziumAvroDeserializationSchema) o; - return Objects.equals(avroDeserializer, that.avroDeserializer) && - Objects.equals(producedTypeInfo, that.producedTypeInfo); - } - - @Override - public int hashCode() { - return Objects.hash(avroDeserializer, producedTypeInfo); - } - - public static RowType createDebeziumAvroRowType(DataType databaseSchema) { - // Debezium Avro contains other information, e.g. "source", "ts_ms" - // but we don't need them - return (RowType) DataTypes.ROW( - DataTypes.FIELD("before", databaseSchema.nullable()), - DataTypes.FIELD("after", databaseSchema.nullable()), - DataTypes.FIELD("op", DataTypes.STRING())).getLogicalType(); - } + private static final long serialVersionUID = 1L; + + /** snapshot read. */ + private static final String OP_READ = "r"; + /** insert operation. */ + private static final String OP_CREATE = "c"; + /** update operation. */ + private static final String OP_UPDATE = "u"; + /** delete operation. */ + private static final String OP_DELETE = "d"; + + private static final String REPLICA_IDENTITY_EXCEPTION = + "The \"before\" field of %s message is null, " + + "if you are using Debezium Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; + + /** The deserializer to deserialize Debezium Avro data. */ + private final AvroRowDataDeserializationSchema avroDeserializer; + + /** TypeInformation of the produced {@link RowData}. */ + private final TypeInformation producedTypeInfo; + + public DebeziumAvroDeserializationSchema( + RowType rowType, TypeInformation producedTypeInfo, String schemaRegistryUrl) { + this.producedTypeInfo = producedTypeInfo; + RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + + this.avroDeserializer = + new AvroRowDataDeserializationSchema( + ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(debeziumAvroRowType), + schemaRegistryUrl), + AvroToRowDataConverters.createRowConverter(debeziumAvroRowType), + producedTypeInfo); + } + + @VisibleForTesting + DebeziumAvroDeserializationSchema( + TypeInformation producedTypeInfo, + AvroRowDataDeserializationSchema avroDeserializer) { + this.producedTypeInfo = producedTypeInfo; + this.avroDeserializer = avroDeserializer; + } + + @Override + public void open(InitializationContext context) throws Exception { + avroDeserializer.open(context); + } + + @Override + public RowData deserialize(byte[] message) throws IOException { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + + @Override + public void deserialize(byte[] message, Collector out) throws IOException { + + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } + try { + GenericRowData row = (GenericRowData) avroDeserializer.deserialize(message); + + GenericRowData before = (GenericRowData) row.getField(0); + GenericRowData after = (GenericRowData) row.getField(1); + String op = row.getField(2).toString(); + if (OP_CREATE.equals(op) || OP_READ.equals(op)) { + after.setRowKind(RowKind.INSERT); + out.collect(after); + } else if (OP_UPDATE.equals(op)) { + if (before == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + before.setRowKind(RowKind.UPDATE_BEFORE); + after.setRowKind(RowKind.UPDATE_AFTER); + out.collect(before); + out.collect(after); + } else if (OP_DELETE.equals(op)) { + if (before == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); + } + before.setRowKind(RowKind.DELETE); + out.collect(before); + } else { + throw new IOException( + format( + "Unknown \"op\" value \"%s\". The Debezium Avro message is '%s'", + op, new String(message))); + } + } catch (Throwable t) { + // a big try catch to protect the processing. + throw new IOException("Can't deserialize Debezium Avro message.", t); + } + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return producedTypeInfo; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DebeziumAvroDeserializationSchema that = (DebeziumAvroDeserializationSchema) o; + return Objects.equals(avroDeserializer, that.avroDeserializer) + && Objects.equals(producedTypeInfo, that.producedTypeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(avroDeserializer, producedTypeInfo); + } + + public static RowType createDebeziumAvroRowType(DataType databaseSchema) { + // Debezium Avro contains other information, e.g. "source", "ts_ms" + // but we don't need them + return (RowType) + DataTypes.ROW( + DataTypes.FIELD("before", databaseSchema.nullable()), + DataTypes.FIELD("after", databaseSchema.nullable()), + DataTypes.FIELD("op", DataTypes.STRING())) + .getLogicalType(); + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java index 494108c18..d67994e2a 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java @@ -46,102 +46,95 @@ import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_URL; /** - * Format factory for providing configured instances of Debezium Avro to RowData {@link DeserializationSchema}. + * Format factory for providing configured instances of Debezium Avro to RowData {@link + * DeserializationSchema}. */ -public class DebeziumAvroFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "debezium-avro-confluent"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); - - return new DecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, - DataType producedDataType) { - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation producedTypeInfo = - context.createTypeInformation(producedDataType); - return new DebeziumAvroDeserializationSchema( - rowType, - producedTypeInfo, - schemaRegistryURL); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - }; - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); - Optional subject = formatOptions.getOptional(SCHEMA_REGISTRY_SUBJECT); - if (!subject.isPresent()) { - throw new ValidationException(String.format( - "Option '%s.%s' is required for serialization", - IDENTIFIER, - SCHEMA_REGISTRY_SUBJECT.key())); - } - - return new EncodingFormat>() { - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, - DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new DebeziumAvroSerializationSchema( - rowType, - schemaRegistryURL, - subject.get()); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - Set> options = new HashSet<>(); - options.add(SCHEMA_REGISTRY_URL); - return options; - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(SCHEMA_REGISTRY_SUBJECT); - return options; - } - +public class DebeziumAvroFormatFactory + implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "debezium-avro-confluent"; + + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + + FactoryUtil.validateFactoryOptions(this, formatOptions); + String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); + + return new DecodingFormat>() { + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + return new DebeziumAvroDeserializationSchema( + rowType, producedTypeInfo, schemaRegistryURL); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + }; + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + + FactoryUtil.validateFactoryOptions(this, formatOptions); + String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); + Optional subject = formatOptions.getOptional(SCHEMA_REGISTRY_SUBJECT); + if (!subject.isPresent()) { + throw new ValidationException( + String.format( + "Option '%s.%s' is required for serialization", + IDENTIFIER, SCHEMA_REGISTRY_SUBJECT.key())); + } + + return new EncodingFormat>() { + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new DebeziumAvroSerializationSchema( + rowType, schemaRegistryURL, subject.get()); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + Set> options = new HashSet<>(); + options.add(SCHEMA_REGISTRY_URL); + return options; + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(SCHEMA_REGISTRY_SUBJECT); + return options; + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java index baf8efa6c..07ab0e73e 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java @@ -37,101 +37,99 @@ import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** - * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium Avro. + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium + * Avro. */ public class DebeziumAvroSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = 1L; - - /** - * insert operation. - */ - private static final StringData OP_INSERT = StringData.fromString("c"); - /** - * delete operation. - */ - private static final StringData OP_DELETE = StringData.fromString("d"); - - /** - * The deserializer to deserialize Debezium Avro data. - */ - private final AvroRowDataSerializationSchema avroSerializer; - - private transient GenericRowData outputReuse; - - public DebeziumAvroSerializationSchema( - RowType rowType, - String schemaRegistryUrl, - String schemaRegistrySubject) { - RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); - - this.avroSerializer = new AvroRowDataSerializationSchema( - debeziumAvroRowType, - ConfluentRegistryAvroSerializationSchema.forGeneric( - schemaRegistrySubject, - AvroSchemaConverter.convertToSchema(debeziumAvroRowType), - schemaRegistryUrl), - RowDataToAvroConverters.createConverter(debeziumAvroRowType)); - } - - @VisibleForTesting - DebeziumAvroSerializationSchema(AvroRowDataSerializationSchema avroSerializer) { - this.avroSerializer = avroSerializer; - } - - @Override - public void open(InitializationContext context) throws Exception { - avroSerializer.open(context); - outputReuse = new GenericRowData(3); - } - - @Override - public byte[] serialize(RowData rowData) { - try { - switch (rowData.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - outputReuse.setField(0, null); - outputReuse.setField(1, rowData); - outputReuse.setField(2, OP_INSERT); - return avroSerializer.serialize(outputReuse); - case UPDATE_BEFORE: - case DELETE: - outputReuse.setField(0, rowData); - outputReuse.setField(1, null); - outputReuse.setField(2, OP_DELETE); - return avroSerializer.serialize(outputReuse); - default: - throw new UnsupportedOperationException(format("Unsupported operation '%s' for row kind.", rowData.getRowKind())); - } - } catch (Throwable t) { - throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DebeziumAvroSerializationSchema that = (DebeziumAvroSerializationSchema) o; - return Objects.equals(avroSerializer, that.avroSerializer); - } - - @Override - public int hashCode() { - return Objects.hash(avroSerializer); - } - - public static RowType createDebeziumAvroRowType(DataType dataType) { - // Debezium Avro contains other information, e.g. "source", "ts_ms" - // but we don't need them - return (RowType) DataTypes.ROW( - DataTypes.FIELD("before", dataType.nullable()), - DataTypes.FIELD("after", dataType.nullable()), - DataTypes.FIELD("op", DataTypes.STRING())).getLogicalType(); - } - + private static final long serialVersionUID = 1L; + + /** insert operation. */ + private static final StringData OP_INSERT = StringData.fromString("c"); + /** delete operation. */ + private static final StringData OP_DELETE = StringData.fromString("d"); + + /** The deserializer to deserialize Debezium Avro data. */ + private final AvroRowDataSerializationSchema avroSerializer; + + private transient GenericRowData outputReuse; + + public DebeziumAvroSerializationSchema( + RowType rowType, String schemaRegistryUrl, String schemaRegistrySubject) { + RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + + this.avroSerializer = + new AvroRowDataSerializationSchema( + debeziumAvroRowType, + ConfluentRegistryAvroSerializationSchema.forGeneric( + schemaRegistrySubject, + AvroSchemaConverter.convertToSchema(debeziumAvroRowType), + schemaRegistryUrl), + RowDataToAvroConverters.createConverter(debeziumAvroRowType)); + } + + @VisibleForTesting + DebeziumAvroSerializationSchema(AvroRowDataSerializationSchema avroSerializer) { + this.avroSerializer = avroSerializer; + } + + @Override + public void open(InitializationContext context) throws Exception { + avroSerializer.open(context); + outputReuse = new GenericRowData(3); + } + + @Override + public byte[] serialize(RowData rowData) { + try { + switch (rowData.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + outputReuse.setField(0, null); + outputReuse.setField(1, rowData); + outputReuse.setField(2, OP_INSERT); + return avroSerializer.serialize(outputReuse); + case UPDATE_BEFORE: + case DELETE: + outputReuse.setField(0, rowData); + outputReuse.setField(1, null); + outputReuse.setField(2, OP_DELETE); + return avroSerializer.serialize(outputReuse); + default: + throw new UnsupportedOperationException( + format( + "Unsupported operation '%s' for row kind.", + rowData.getRowKind())); + } + } catch (Throwable t) { + throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DebeziumAvroSerializationSchema that = (DebeziumAvroSerializationSchema) o; + return Objects.equals(avroSerializer, that.avroSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(avroSerializer); + } + + public static RowType createDebeziumAvroRowType(DataType dataType) { + // Debezium Avro contains other information, e.g. "source", "ts_ms" + // but we don't need them + return (RowType) + DataTypes.ROW( + DataTypes.FIELD("before", dataType.nullable()), + DataTypes.FIELD("after", dataType.nullable()), + DataTypes.FIELD("op", DataTypes.STRING())) + .getLogicalType(); + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java index 01e807c7d..009ef029b 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java @@ -30,51 +30,46 @@ import static org.junit.Assert.assertEquals; -/** - * Tests for {@link ConfluentSchemaRegistryCoder}. - */ +/** Tests for {@link ConfluentSchemaRegistryCoder}. */ public class ConfluentSchemaRegistryCoderTest { - @Test - public void testSpecificRecordWithConfluentSchemaRegistry() throws Exception { - MockSchemaRegistryClient client = new MockSchemaRegistryClient(); - - Schema schema = SchemaBuilder.record("testRecord") - .fields() - .optionalString("testField") - .endRecord(); - int schemaId = client.register("testTopic", schema); + @Test + public void testSpecificRecordWithConfluentSchemaRegistry() throws Exception { + MockSchemaRegistryClient client = new MockSchemaRegistryClient(); - ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(client); - ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream(); - DataOutputStream dataOutputStream = new DataOutputStream(byteOutStream); - dataOutputStream.writeByte(0); - dataOutputStream.writeInt(schemaId); - dataOutputStream.flush(); + Schema schema = + SchemaBuilder.record("testRecord").fields().optionalString("testField").endRecord(); + int schemaId = client.register("testTopic", schema); - ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray()); - Schema readSchema = registryCoder.readSchema(byteInStream); + ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(client); + ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream(); + DataOutputStream dataOutputStream = new DataOutputStream(byteOutStream); + dataOutputStream.writeByte(0); + dataOutputStream.writeInt(schemaId); + dataOutputStream.flush(); - assertEquals(schema, readSchema); - assertEquals(0, byteInStream.available()); - } + ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray()); + Schema readSchema = registryCoder.readSchema(byteInStream); - @Test(expected = IOException.class) - public void testMagicByteVerification() throws Exception { - MockSchemaRegistryClient client = new MockSchemaRegistryClient(); - int schemaId = client.register("testTopic", Schema.create(Schema.Type.BOOLEAN)); + assertEquals(schema, readSchema); + assertEquals(0, byteInStream.available()); + } - ConfluentSchemaRegistryCoder coder = new ConfluentSchemaRegistryCoder(client); - ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream(); - DataOutputStream dataOutputStream = new DataOutputStream(byteOutStream); - dataOutputStream.writeByte(5); - dataOutputStream.writeInt(schemaId); - dataOutputStream.flush(); + @Test(expected = IOException.class) + public void testMagicByteVerification() throws Exception { + MockSchemaRegistryClient client = new MockSchemaRegistryClient(); + int schemaId = client.register("testTopic", Schema.create(Schema.Type.BOOLEAN)); - ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray()); - coder.readSchema(byteInStream); + ConfluentSchemaRegistryCoder coder = new ConfluentSchemaRegistryCoder(client); + ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream(); + DataOutputStream dataOutputStream = new DataOutputStream(byteOutStream); + dataOutputStream.writeByte(5); + dataOutputStream.writeInt(schemaId); + dataOutputStream.flush(); - // exception is thrown - } + ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray()); + coder.readSchema(byteInStream); + // exception is thrown + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java index 70005f21e..b1aab7903 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java @@ -53,127 +53,122 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; -/** - * Tests for the {@link RegistryAvroFormatFactory}. - */ +/** Tests for the {@link RegistryAvroFormatFactory}. */ public class RegistryAvroFormatFactoryTest { - private static final TableSchema SCHEMA = TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); - private static final String SUBJECT = "test-subject"; - private static final String REGISTRY_URL = "http://localhost:8081"; - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Test - public void testDeserializationSchema() { - final AvroRowDataDeserializationSchema expectedDeser = - new AvroRowDataDeserializationSchema( - ConfluentRegistryAvroDeserializationSchema.forGeneric( - AvroSchemaConverter.convertToSchema(ROW_TYPE), - REGISTRY_URL), - AvroToRowDataConverters.createRowConverter(ROW_TYPE), - InternalTypeInfo.of(ROW_TYPE)); - - final DynamicTableSource actualSource = createTableSource(getDefaultOptions()); - assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - DeserializationSchema actualDeser = scanSourceMock.valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, - SCHEMA.toRowDataType()); - - assertEquals(expectedDeser, actualDeser); - } - - @Test - public void testSerializationSchema() { - final AvroRowDataSerializationSchema expectedSer = - new AvroRowDataSerializationSchema( - ROW_TYPE, - ConfluentRegistryAvroSerializationSchema.forGeneric( - SUBJECT, - AvroSchemaConverter.convertToSchema(ROW_TYPE), - REGISTRY_URL), - RowDataToAvroConverters.createConverter(ROW_TYPE)); - - final DynamicTableSink actualSink = createTableSink(getDefaultOptions()); - assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = sinkMock.valueFormat - .createRuntimeEncoder( - null, - SCHEMA.toRowDataType()); - - assertEquals(expectedSer, actualSer); - } - - @Test - public void testMissingSubjectForSink() { - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException("Option avro-confluent.schema-registry.subject " - + "is required for serialization"))); - - final Map options = - getModifiedOptions(opts -> opts.remove("avro-confluent.schema-registry.subject")); - - createTableSink(options); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifiedOptions(Consumer> optionModifier) { - Map options = getDefaultOptions(); - optionModifier.accept(options); - return options; - } - - private Map getDefaultOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", RegistryAvroFormatFactory.IDENTIFIER); - options.put("avro-confluent.schema-registry.subject", SUBJECT); - options.put("avro-confluent.schema-registry.url", REGISTRY_URL); - return options; - } - - private DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - RegistryAvroFormatFactoryTest.class.getClassLoader(), - false); - } - - private DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - RegistryAvroFormatFactoryTest.class.getClassLoader(), - false); - } + private static final TableSchema SCHEMA = + TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + private static final String SUBJECT = "test-subject"; + private static final String REGISTRY_URL = "http://localhost:8081"; + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testDeserializationSchema() { + final AvroRowDataDeserializationSchema expectedDeser = + new AvroRowDataDeserializationSchema( + ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(ROW_TYPE), REGISTRY_URL), + AvroToRowDataConverters.createRowConverter(ROW_TYPE), + InternalTypeInfo.of(ROW_TYPE)); + + final DynamicTableSource actualSource = createTableSource(getDefaultOptions()); + assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = + scanSourceMock.valueFormat.createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); + + assertEquals(expectedDeser, actualDeser); + } + + @Test + public void testSerializationSchema() { + final AvroRowDataSerializationSchema expectedSer = + new AvroRowDataSerializationSchema( + ROW_TYPE, + ConfluentRegistryAvroSerializationSchema.forGeneric( + SUBJECT, + AvroSchemaConverter.convertToSchema(ROW_TYPE), + REGISTRY_URL), + RowDataToAvroConverters.createConverter(ROW_TYPE)); + + final DynamicTableSink actualSink = createTableSink(getDefaultOptions()); + assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = + sinkMock.valueFormat.createRuntimeEncoder(null, SCHEMA.toRowDataType()); + + assertEquals(expectedSer, actualSer); + } + + @Test + public void testMissingSubjectForSink() { + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Option avro-confluent.schema-registry.subject " + + "is required for serialization"))); + + final Map options = + getModifiedOptions(opts -> opts.remove("avro-confluent.schema-registry.subject")); + + createTableSink(options); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifiedOptions(Consumer> optionModifier) { + Map options = getDefaultOptions(); + optionModifier.accept(options); + return options; + } + + private Map getDefaultOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", RegistryAvroFormatFactory.IDENTIFIER); + options.put("avro-confluent.schema-registry.subject", SUBJECT); + options.put("avro-confluent.schema-registry.url", REGISTRY_URL); + return options; + } + + private DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + RegistryAvroFormatFactoryTest.class.getClassLoader(), + false); + } + + private DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + RegistryAvroFormatFactoryTest.class.getClassLoader(), + false); + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java index cc16e0822..092bc1d21 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java @@ -57,146 +57,144 @@ import static org.junit.Assert.assertThat; /** - * Tests for {@link AvroRowDataDeserializationSchema} and - * {@link AvroRowDataSerializationSchema} for schema registry avro. + * Tests for {@link AvroRowDataDeserializationSchema} and {@link AvroRowDataSerializationSchema} for + * schema registry avro. */ public class RegistryAvroRowDataSeDeSchemaTest { - private static final Schema ADDRESS_SCHEMA = Address.getClassSchema(); - - private static final Schema ADDRESS_SCHEMA_COMPATIBLE = new Schema.Parser().parse( - "" + - "{\"namespace\": \"org.apache.flink.formats.avro.generated\",\n" + - " \"type\": \"record\",\n" + - " \"name\": \"Address\",\n" + - " \"fields\": [\n" + - " {\"name\": \"num\", \"type\": \"int\"},\n" + - " {\"name\": \"street\", \"type\": \"string\"}\n" + - " ]\n" + - "}"); - - private static final String SUBJECT = "address-value"; - - private static SchemaRegistryClient client; - - private Address address; - - @Rule - public ExpectedException expectedEx = ExpectedException.none(); - - @BeforeClass - public static void beforeClass() { - client = new MockSchemaRegistryClient(); - } - - @Before - public void before() { - this.address = TestDataGenerator.generateRandomAddress(new Random()); - } - - @After - public void after() throws IOException, RestClientException { - client.deleteSubject(SUBJECT); - } - - @Test - public void testRowDataWriteReadWithFullSchema() throws Exception { - testRowDataWriteReadWithSchema(ADDRESS_SCHEMA); - } - - @Test - public void testRowDataWriteReadWithCompatibleSchema() throws Exception { - testRowDataWriteReadWithSchema(ADDRESS_SCHEMA_COMPATIBLE); - // Validates new schema has been registered. - assertThat(client.getAllVersions(SUBJECT).size(), is(1)); - } - - @Test - public void testRowDataWriteReadWithPreRegisteredSchema() throws Exception { - client.register(SUBJECT, ADDRESS_SCHEMA); - testRowDataWriteReadWithSchema(ADDRESS_SCHEMA); - // Validates it does not produce new schema. - assertThat(client.getAllVersions(SUBJECT).size(), is(1)); - } - - @Test - public void testRowDataReadWithNonRegistryAvro() throws Exception { - DataType dataType = AvroSchemaConverter.convertToDataType(ADDRESS_SCHEMA.toString()); - RowType rowType = (RowType) dataType.getLogicalType(); - - AvroRowDataDeserializationSchema deserializer = getDeserializationSchema(rowType, ADDRESS_SCHEMA); - - deserializer.open(null); - - client.register(SUBJECT, ADDRESS_SCHEMA); - byte[] oriBytes = writeRecord(address, ADDRESS_SCHEMA); - expectedEx.expect(IOException.class); - expectedEx.expect(containsCause(new IOException("Unknown data format. Magic number does not match"))); - deserializer.deserialize(oriBytes); - } - - private void testRowDataWriteReadWithSchema(Schema schema) throws Exception { - DataType dataType = AvroSchemaConverter.convertToDataType(schema.toString()); - RowType rowType = (RowType) dataType.getLogicalType(); - - AvroRowDataSerializationSchema serializer = getSerializationSchema(rowType, schema); - Schema writeSchema = AvroSchemaConverter - .convertToSchema(dataType.getLogicalType()); - AvroRowDataDeserializationSchema deserializer = - getDeserializationSchema(rowType, writeSchema); - - serializer.open(null); - deserializer.open(null); - - RowData oriData = address2RowData(address); - byte[] serialized = serializer.serialize(oriData); - RowData rowData = deserializer.deserialize(serialized); - assertThat(rowData.getArity(), equalTo(schema.getFields().size())); - assertEquals(address.getNum(), rowData.getInt(0)); - assertEquals(address.getStreet(), rowData.getString(1).toString()); - if (schema != ADDRESS_SCHEMA_COMPATIBLE) { - assertEquals(address.getCity(), rowData.getString(2).toString()); - assertEquals(address.getState(), rowData.getString(3).toString()); - assertEquals(address.getZip(), rowData.getString(4).toString()); - } - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - private static AvroRowDataSerializationSchema getSerializationSchema( - RowType rowType, - Schema avroSchema) { - ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); - return new AvroRowDataSerializationSchema( - rowType, - new RegistryAvroSerializationSchema( - GenericRecord.class, - avroSchema, - () -> registryCoder), - RowDataToAvroConverters.createConverter(rowType)); - } - - private static AvroRowDataDeserializationSchema getDeserializationSchema( - RowType rowType, - Schema avroSchema) { - ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); - return new AvroRowDataDeserializationSchema( - new RegistryAvroDeserializationSchema( - GenericRecord.class, - avroSchema, - () -> registryCoder), - AvroToRowDataConverters.createRowConverter(rowType), - InternalTypeInfo.of(rowType)); - } - - private static RowData address2RowData(Address address) { - GenericRowData rowData = new GenericRowData(5); - rowData.setField(0, address.getNum()); - rowData.setField(1, new BinaryStringData(address.getStreet().toString())); - rowData.setField(2, new BinaryStringData(address.getCity().toString())); - rowData.setField(3, new BinaryStringData(address.getState().toString())); - rowData.setField(4, new BinaryStringData(address.getZip().toString())); - return rowData; - } + private static final Schema ADDRESS_SCHEMA = Address.getClassSchema(); + + private static final Schema ADDRESS_SCHEMA_COMPATIBLE = + new Schema.Parser() + .parse( + "" + + "{\"namespace\": \"org.apache.flink.formats.avro.generated\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"Address\",\n" + + " \"fields\": [\n" + + " {\"name\": \"num\", \"type\": \"int\"},\n" + + " {\"name\": \"street\", \"type\": \"string\"}\n" + + " ]\n" + + "}"); + + private static final String SUBJECT = "address-value"; + + private static SchemaRegistryClient client; + + private Address address; + + @Rule public ExpectedException expectedEx = ExpectedException.none(); + + @BeforeClass + public static void beforeClass() { + client = new MockSchemaRegistryClient(); + } + + @Before + public void before() { + this.address = TestDataGenerator.generateRandomAddress(new Random()); + } + + @After + public void after() throws IOException, RestClientException { + client.deleteSubject(SUBJECT); + } + + @Test + public void testRowDataWriteReadWithFullSchema() throws Exception { + testRowDataWriteReadWithSchema(ADDRESS_SCHEMA); + } + + @Test + public void testRowDataWriteReadWithCompatibleSchema() throws Exception { + testRowDataWriteReadWithSchema(ADDRESS_SCHEMA_COMPATIBLE); + // Validates new schema has been registered. + assertThat(client.getAllVersions(SUBJECT).size(), is(1)); + } + + @Test + public void testRowDataWriteReadWithPreRegisteredSchema() throws Exception { + client.register(SUBJECT, ADDRESS_SCHEMA); + testRowDataWriteReadWithSchema(ADDRESS_SCHEMA); + // Validates it does not produce new schema. + assertThat(client.getAllVersions(SUBJECT).size(), is(1)); + } + + @Test + public void testRowDataReadWithNonRegistryAvro() throws Exception { + DataType dataType = AvroSchemaConverter.convertToDataType(ADDRESS_SCHEMA.toString()); + RowType rowType = (RowType) dataType.getLogicalType(); + + AvroRowDataDeserializationSchema deserializer = + getDeserializationSchema(rowType, ADDRESS_SCHEMA); + + deserializer.open(null); + + client.register(SUBJECT, ADDRESS_SCHEMA); + byte[] oriBytes = writeRecord(address, ADDRESS_SCHEMA); + expectedEx.expect(IOException.class); + expectedEx.expect( + containsCause(new IOException("Unknown data format. Magic number does not match"))); + deserializer.deserialize(oriBytes); + } + + private void testRowDataWriteReadWithSchema(Schema schema) throws Exception { + DataType dataType = AvroSchemaConverter.convertToDataType(schema.toString()); + RowType rowType = (RowType) dataType.getLogicalType(); + + AvroRowDataSerializationSchema serializer = getSerializationSchema(rowType, schema); + Schema writeSchema = AvroSchemaConverter.convertToSchema(dataType.getLogicalType()); + AvroRowDataDeserializationSchema deserializer = + getDeserializationSchema(rowType, writeSchema); + + serializer.open(null); + deserializer.open(null); + + RowData oriData = address2RowData(address); + byte[] serialized = serializer.serialize(oriData); + RowData rowData = deserializer.deserialize(serialized); + assertThat(rowData.getArity(), equalTo(schema.getFields().size())); + assertEquals(address.getNum(), rowData.getInt(0)); + assertEquals(address.getStreet(), rowData.getString(1).toString()); + if (schema != ADDRESS_SCHEMA_COMPATIBLE) { + assertEquals(address.getCity(), rowData.getString(2).toString()); + assertEquals(address.getState(), rowData.getString(3).toString()); + assertEquals(address.getZip(), rowData.getString(4).toString()); + } + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + private static AvroRowDataSerializationSchema getSerializationSchema( + RowType rowType, Schema avroSchema) { + ConfluentSchemaRegistryCoder registryCoder = + new ConfluentSchemaRegistryCoder(SUBJECT, client); + return new AvroRowDataSerializationSchema( + rowType, + new RegistryAvroSerializationSchema( + GenericRecord.class, avroSchema, () -> registryCoder), + RowDataToAvroConverters.createConverter(rowType)); + } + + private static AvroRowDataDeserializationSchema getDeserializationSchema( + RowType rowType, Schema avroSchema) { + ConfluentSchemaRegistryCoder registryCoder = + new ConfluentSchemaRegistryCoder(SUBJECT, client); + return new AvroRowDataDeserializationSchema( + new RegistryAvroDeserializationSchema( + GenericRecord.class, avroSchema, () -> registryCoder), + AvroToRowDataConverters.createRowConverter(rowType), + InternalTypeInfo.of(rowType)); + } + + private static RowData address2RowData(Address address) { + GenericRowData rowData = new GenericRowData(5); + rowData.setField(0, address.getNum()); + rowData.setField(1, new BinaryStringData(address.getStreet().toString())); + rowData.setField(2, new BinaryStringData(address.getCity().toString())); + rowData.setField(3, new BinaryStringData(address.getState().toString())); + rowData.setField(4, new BinaryStringData(address.getZip().toString())); + return rowData; + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java index fc4440c86..3e442a4fa 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java @@ -48,95 +48,89 @@ import static org.hamcrest.CoreMatchers.instanceOf; import static org.junit.Assert.assertThat; -/** - * Tests for {@link DebeziumAvroFormatFactory}. - */ +/** Tests for {@link DebeziumAvroFormatFactory}. */ public class DebeziumAvroFormatFactoryTest extends TestLogger { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - private static final TableSchema SCHEMA = TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); - - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); - - private static final String SUBJECT = "test-debezium-avro"; - private static final String REGISTRY_URL = "http://localhost:8081"; - - @Test - public void testSeDeSchema() { - final Map options = getAllOptions(); - - DebeziumAvroDeserializationSchema expectedDeser = new DebeziumAvroDeserializationSchema( - ROW_TYPE, - InternalTypeInfo.of(ROW_TYPE), - REGISTRY_URL); - DeserializationSchema actualDeser = createDeserializationSchema(options); - assertEquals(expectedDeser, actualDeser); - - DebeziumAvroSerializationSchema expectedSer = new DebeziumAvroSerializationSchema( - ROW_TYPE, - REGISTRY_URL, - SUBJECT - ); - SerializationSchema actualSer = createSerializationSchema(options); - Assert.assertEquals(expectedSer, actualSer); - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", DebeziumAvroFormatFactory.IDENTIFIER); - options.put("debezium-avro-confluent.schema-registry.url", REGISTRY_URL); - options.put("debezium-avro-confluent.schema-registry.subject", SUBJECT); - return options; - } - - private static DeserializationSchema createDeserializationSchema(Map options) { - final DynamicTableSource actualSource = createTableSource(options); - assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - return scanSourceMock.valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, - SCHEMA.toRowDataType()); - } - - private static SerializationSchema createSerializationSchema(Map options) { - final DynamicTableSink actualSink = createTableSink(options); - assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - return sinkMock.valueFormat - .createRuntimeEncoder( - new SinkRuntimeProviderContext(false), - SCHEMA.toRowDataType()); - } - - private static DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - DebeziumAvroFormatFactoryTest.class.getClassLoader(), false); - } - - private static DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - DebeziumAvroFormatFactoryTest.class.getClassLoader(), false); - } + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = + TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + private static final String SUBJECT = "test-debezium-avro"; + private static final String REGISTRY_URL = "http://localhost:8081"; + + @Test + public void testSeDeSchema() { + final Map options = getAllOptions(); + + DebeziumAvroDeserializationSchema expectedDeser = + new DebeziumAvroDeserializationSchema( + ROW_TYPE, InternalTypeInfo.of(ROW_TYPE), REGISTRY_URL); + DeserializationSchema actualDeser = createDeserializationSchema(options); + assertEquals(expectedDeser, actualDeser); + + DebeziumAvroSerializationSchema expectedSer = + new DebeziumAvroSerializationSchema(ROW_TYPE, REGISTRY_URL, SUBJECT); + SerializationSchema actualSer = createSerializationSchema(options); + Assert.assertEquals(expectedSer, actualSer); + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", DebeziumAvroFormatFactory.IDENTIFIER); + options.put("debezium-avro-confluent.schema-registry.url", REGISTRY_URL); + options.put("debezium-avro-confluent.schema-registry.subject", SUBJECT); + return options; + } + + private static DeserializationSchema createDeserializationSchema( + Map options) { + final DynamicTableSource actualSource = createTableSource(options); + assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + return scanSourceMock.valueFormat.createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); + } + + private static SerializationSchema createSerializationSchema( + Map options) { + final DynamicTableSink actualSink = createTableSink(options); + assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + return sinkMock.valueFormat.createRuntimeEncoder( + new SinkRuntimeProviderContext(false), SCHEMA.toRowDataType()); + } + + private static DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + DebeziumAvroFormatFactoryTest.class.getClassLoader(), + false); + } + + private static DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + DebeziumAvroFormatFactoryTest.class.getClassLoader(), + false); + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java index f3cc96083..a1130386c 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java @@ -61,160 +61,165 @@ import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; -/** - * Tests for {@link DebeziumAvroDeserializationSchema}. - */ +/** Tests for {@link DebeziumAvroDeserializationSchema}. */ public class DebeziumAvroSerDeSchemaTest { - private static final String SUBJECT = "testDebeziumAvro"; - - private static final RowType rowType = (RowType) ROW( - FIELD("id", BIGINT()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", DOUBLE()) - ).getLogicalType(); - - private static final Schema DEBEZIUM_SCHEMA_COMPATIBLE_TEST = new Schema.Parser().parse( - new String(readBytesFromFile("debezium-test-schema.json")) - ); + private static final String SUBJECT = "testDebeziumAvro"; - private SchemaRegistryClient client = new MockSchemaRegistryClient(); - - @Test - public void testSerializationDeserialization() throws Exception { + private static final RowType rowType = + (RowType) + ROW( + FIELD("id", BIGINT()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", DOUBLE())) + .getLogicalType(); + + private static final Schema DEBEZIUM_SCHEMA_COMPATIBLE_TEST = + new Schema.Parser().parse(new String(readBytesFromFile("debezium-test-schema.json"))); - RowType rowTypeDe = DebeziumAvroDeserializationSchema.createDebeziumAvroRowType(fromLogicalToDataType(rowType)); - RowType rowTypeSe = DebeziumAvroSerializationSchema.createDebeziumAvroRowType(fromLogicalToDataType(rowType)); - - DebeziumAvroSerializationSchema dbzSerializer = new DebeziumAvroSerializationSchema(getSerializationSchema(rowTypeSe)); - dbzSerializer.open(mock(SerializationSchema.InitializationContext.class)); - - byte[] serialize = dbzSerializer.serialize(debeziumRow2RowData()); - - client.register(SUBJECT, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); - DebeziumAvroDeserializationSchema dbzDeserializer = new DebeziumAvroDeserializationSchema( - InternalTypeInfo.of(rowType), - getDeserializationSchema(rowTypeDe)); - dbzDeserializer.open(mock(DeserializationSchema.InitializationContext.class)); - - SimpleCollector collector = new SimpleCollector(); - dbzDeserializer.deserialize(serialize, collector); - - List actual = collector.list.stream() - .map(Object::toString) - .collect(Collectors.toList()); - - List expected = Collections.singletonList( - "+I(107,rocks,box of assorted rocks,5.3)"); - assertEquals(expected, actual); - } - - @Test - public void testInsertDataDeserialization() throws Exception { - List actual = testDeserialization("debezium-avro-insert.avro"); - - List expected = Collections.singletonList( - "+I(1,lisi,test debezium avro data,21.799999237060547)"); - assertEquals(expected, actual); - } - - @Test - public void testUpdateDataDeserialization() throws Exception { - List actual = testDeserialization("debezium-avro-update.avro"); - - List expected = Arrays.asList( - "-U(1,lisi,test debezium avro data,21.799999237060547)", - "+U(1,zhangsan,test debezium avro data,21.799999237060547)"); - assertEquals(expected, actual); - } - - @Test - public void testDeleteDataDeserialization() throws Exception { - List actual = testDeserialization("debezium-avro-delete.avro"); - - List expected = Collections.singletonList( - "-D(1,zhangsan,test debezium avro data,21.799999237060547)"); - assertEquals(expected, actual); - } - - public List testDeserialization(String dataPath) throws Exception { - RowType rowTypeDe = DebeziumAvroDeserializationSchema - .createDebeziumAvroRowType(fromLogicalToDataType(rowType)); - - client.register(SUBJECT, DEBEZIUM_SCHEMA_COMPATIBLE_TEST, 1, 81); - - DebeziumAvroDeserializationSchema dbzDeserializer = new DebeziumAvroDeserializationSchema( - InternalTypeInfo.of(rowType), - getDeserializationSchema(rowTypeDe)); - dbzDeserializer.open(mock(DeserializationSchema.InitializationContext.class)); - - SimpleCollector collector = new SimpleCollector(); - dbzDeserializer.deserialize(readBytesFromFile(dataPath), collector); - - return collector.list.stream() - .map(Object::toString) - .collect(Collectors.toList()); - } - - private AvroRowDataDeserializationSchema getDeserializationSchema(RowType rowType) { - - final ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); - - return new AvroRowDataDeserializationSchema( - new RegistryAvroDeserializationSchema<>( - GenericRecord.class, - AvroSchemaConverter.convertToSchema(rowType), - () -> registryCoder - ), - AvroToRowDataConverters.createRowConverter(rowType), - InternalTypeInfo.of(rowType)); - } - - private AvroRowDataSerializationSchema getSerializationSchema(RowType rowType) { - - ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(SUBJECT, client); - return new AvroRowDataSerializationSchema( - rowType, - new RegistryAvroSerializationSchema<>( - GenericRecord.class, - AvroSchemaConverter.convertToSchema(rowType), - () -> registryCoder), - RowDataToAvroConverters.createConverter(rowType)); - } - - private static RowData debeziumRow2RowData() { - GenericRowData rowData = new GenericRowData(4); - rowData.setField(0, 107L); - rowData.setField(1, StringData.fromString("rocks")); - rowData.setField(2, StringData.fromString("box of assorted rocks")); - rowData.setField(3, 5.3D); - return rowData; - } - - private static byte[] readBytesFromFile(String filePath) { - try { - URL url = DebeziumAvroSerDeSchemaTest.class.getClassLoader().getResource(filePath); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return FileUtils.readAllBytes(path); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - private static class SimpleCollector implements Collector { - - private List list = new ArrayList<>(); - - @Override - public void collect(RowData record) { - list.add(record); - } - - @Override - public void close() { - // do nothing - } - } + private SchemaRegistryClient client = new MockSchemaRegistryClient(); + + @Test + public void testSerializationDeserialization() throws Exception { + + RowType rowTypeDe = + DebeziumAvroDeserializationSchema.createDebeziumAvroRowType( + fromLogicalToDataType(rowType)); + RowType rowTypeSe = + DebeziumAvroSerializationSchema.createDebeziumAvroRowType( + fromLogicalToDataType(rowType)); + + DebeziumAvroSerializationSchema dbzSerializer = + new DebeziumAvroSerializationSchema(getSerializationSchema(rowTypeSe)); + dbzSerializer.open(mock(SerializationSchema.InitializationContext.class)); + + byte[] serialize = dbzSerializer.serialize(debeziumRow2RowData()); + + client.register(SUBJECT, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); + DebeziumAvroDeserializationSchema dbzDeserializer = + new DebeziumAvroDeserializationSchema( + InternalTypeInfo.of(rowType), getDeserializationSchema(rowTypeDe)); + dbzDeserializer.open(mock(DeserializationSchema.InitializationContext.class)); + + SimpleCollector collector = new SimpleCollector(); + dbzDeserializer.deserialize(serialize, collector); + + List actual = + collector.list.stream().map(Object::toString).collect(Collectors.toList()); + + List expected = + Collections.singletonList("+I(107,rocks,box of assorted rocks,5.3)"); + assertEquals(expected, actual); + } + + @Test + public void testInsertDataDeserialization() throws Exception { + List actual = testDeserialization("debezium-avro-insert.avro"); + + List expected = + Collections.singletonList("+I(1,lisi,test debezium avro data,21.799999237060547)"); + assertEquals(expected, actual); + } + + @Test + public void testUpdateDataDeserialization() throws Exception { + List actual = testDeserialization("debezium-avro-update.avro"); + + List expected = + Arrays.asList( + "-U(1,lisi,test debezium avro data,21.799999237060547)", + "+U(1,zhangsan,test debezium avro data,21.799999237060547)"); + assertEquals(expected, actual); + } + + @Test + public void testDeleteDataDeserialization() throws Exception { + List actual = testDeserialization("debezium-avro-delete.avro"); + + List expected = + Collections.singletonList( + "-D(1,zhangsan,test debezium avro data,21.799999237060547)"); + assertEquals(expected, actual); + } + + public List testDeserialization(String dataPath) throws Exception { + RowType rowTypeDe = + DebeziumAvroDeserializationSchema.createDebeziumAvroRowType( + fromLogicalToDataType(rowType)); + + client.register(SUBJECT, DEBEZIUM_SCHEMA_COMPATIBLE_TEST, 1, 81); + + DebeziumAvroDeserializationSchema dbzDeserializer = + new DebeziumAvroDeserializationSchema( + InternalTypeInfo.of(rowType), getDeserializationSchema(rowTypeDe)); + dbzDeserializer.open(mock(DeserializationSchema.InitializationContext.class)); + + SimpleCollector collector = new SimpleCollector(); + dbzDeserializer.deserialize(readBytesFromFile(dataPath), collector); + + return collector.list.stream().map(Object::toString).collect(Collectors.toList()); + } + + private AvroRowDataDeserializationSchema getDeserializationSchema(RowType rowType) { + + final ConfluentSchemaRegistryCoder registryCoder = + new ConfluentSchemaRegistryCoder(SUBJECT, client); + + return new AvroRowDataDeserializationSchema( + new RegistryAvroDeserializationSchema<>( + GenericRecord.class, + AvroSchemaConverter.convertToSchema(rowType), + () -> registryCoder), + AvroToRowDataConverters.createRowConverter(rowType), + InternalTypeInfo.of(rowType)); + } + + private AvroRowDataSerializationSchema getSerializationSchema(RowType rowType) { + + ConfluentSchemaRegistryCoder registryCoder = + new ConfluentSchemaRegistryCoder(SUBJECT, client); + return new AvroRowDataSerializationSchema( + rowType, + new RegistryAvroSerializationSchema<>( + GenericRecord.class, + AvroSchemaConverter.convertToSchema(rowType), + () -> registryCoder), + RowDataToAvroConverters.createConverter(rowType)); + } + + private static RowData debeziumRow2RowData() { + GenericRowData rowData = new GenericRowData(4); + rowData.setField(0, 107L); + rowData.setField(1, StringData.fromString("rocks")); + rowData.setField(2, StringData.fromString("box of assorted rocks")); + rowData.setField(3, 5.3D); + return rowData; + } + + private static byte[] readBytesFromFile(String filePath) { + try { + URL url = DebeziumAvroSerDeSchemaTest.class.getClassLoader().getResource(filePath); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return FileUtils.readAllBytes(path); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static class SimpleCollector implements Collector { + + private List list = new ArrayList<>(); + + @Override + public void collect(RowData record) { + list.add(record); + } + + @Override + public void close() { + // do nothing + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index 17fb69e53..386bdd8db 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -49,99 +49,89 @@ import static org.apache.flink.formats.json.JsonOptions.validateEncodingFormatOptions; /** - * Table format factory for providing configured instances of JSON to RowData - * {@link SerializationSchema} and {@link DeserializationSchema}. + * Table format factory for providing configured instances of JSON to RowData {@link + * SerializationSchema} and {@link DeserializationSchema}. */ -public class JsonFormatFactory implements - DeserializationFormatFactory, - SerializationFormatFactory { - - public static final String IDENTIFIER = "json"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateDecodingFormatOptions(formatOptions); - - final boolean failOnMissingField = formatOptions.get(FAIL_ON_MISSING_FIELD); - final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - TimestampFormat timestampOption = JsonOptions.getTimestampFormat(formatOptions); - - return new DecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, - DataType producedDataType) { - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation rowDataTypeInfo = - context.createTypeInformation(producedDataType); - return new JsonRowDataDeserializationSchema( - rowType, - rowDataTypeInfo, - failOnMissingField, - ignoreParseErrors, - timestampOption - ); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - }; - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateEncodingFormatOptions(formatOptions); - - TimestampFormat timestampOption = JsonOptions.getTimestampFormat(formatOptions); - JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); - String mapNullKeyLiteral = formatOptions.get(MAP_NULL_KEY_LITERAL); - - return new EncodingFormat>() { - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, - DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new JsonRowDataSerializationSchema( - rowType, - timestampOption, - mapNullKeyMode, - mapNullKeyLiteral); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return Collections.emptySet(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(FAIL_ON_MISSING_FIELD); - options.add(IGNORE_PARSE_ERRORS); - options.add(TIMESTAMP_FORMAT); - options.add(MAP_NULL_KEY_MODE); - options.add(MAP_NULL_KEY_LITERAL); - return options; - } +public class JsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "json"; + + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateDecodingFormatOptions(formatOptions); + + final boolean failOnMissingField = formatOptions.get(FAIL_ON_MISSING_FIELD); + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + TimestampFormat timestampOption = JsonOptions.getTimestampFormat(formatOptions); + + return new DecodingFormat>() { + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + context.createTypeInformation(producedDataType); + return new JsonRowDataDeserializationSchema( + rowType, + rowDataTypeInfo, + failOnMissingField, + ignoreParseErrors, + timestampOption); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + }; + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateEncodingFormatOptions(formatOptions); + + TimestampFormat timestampOption = JsonOptions.getTimestampFormat(formatOptions); + JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + String mapNullKeyLiteral = formatOptions.get(MAP_NULL_KEY_LITERAL); + + return new EncodingFormat>() { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new JsonRowDataSerializationSchema( + rowType, timestampOption, mapNullKeyMode, mapNullKeyLiteral); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(FAIL_ON_MISSING_FIELD); + options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); + options.add(MAP_NULL_KEY_MODE); + options.add(MAP_NULL_KEY_LITERAL); + return options; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java index e20c83977..55c61e1a6 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java @@ -33,12 +33,12 @@ @PublicEvolving public class JsonNodeDeserializationSchema extends AbstractDeserializationSchema { - private static final long serialVersionUID = -1699854177598621044L; + private static final long serialVersionUID = -1699854177598621044L; - private final ObjectMapper mapper = new ObjectMapper(); + private final ObjectMapper mapper = new ObjectMapper(); - @Override - public ObjectNode deserialize(byte[] message) throws IOException { - return mapper.readValue(message, ObjectNode.class); - } + @Override + public ObjectNode deserialize(byte[] message) throws IOException { + return mapper.readValue(message, ObjectNode.class); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java index 037027a58..ac21981cf 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java @@ -30,157 +30,161 @@ import java.util.Set; import java.util.stream.Collectors; -/** - * This class holds configuration constants used by json format. - */ +/** This class holds configuration constants used by json format. */ public class JsonOptions { - public static final ConfigOption FAIL_ON_MISSING_FIELD = ConfigOptions - .key("fail-on-missing-field") - .booleanType() - .defaultValue(false) - .withDescription("Optional flag to specify whether to fail if a field is missing or not, false by default."); - - public static final ConfigOption IGNORE_PARSE_ERRORS = ConfigOptions - .key("ignore-parse-errors") - .booleanType() - .defaultValue(false) - .withDescription("Optional flag to skip fields and rows with parse errors instead of failing;\n" - + "fields are set to null in case of errors, false by default."); - - public static final ConfigOption MAP_NULL_KEY_MODE = ConfigOptions - .key("map-null-key.mode") - .stringType() - .defaultValue("FAIL") - .withDescription("Optional flag to control the handling mode when serializing null key for map data, FAIL by default." - + " Option DROP will drop null key entries for map data." - + " Option LITERAL will use 'map-null-key.literal' as key literal."); - - public static final ConfigOption MAP_NULL_KEY_LITERAL = ConfigOptions - .key("map-null-key.literal") - .stringType() - .defaultValue("null") - .withDescription("Optional flag to specify string literal for null keys when 'map-null-key.mode' is LITERAL, \"null\" by default."); - - public static final ConfigOption TIMESTAMP_FORMAT = ConfigOptions - .key("timestamp-format.standard") - .stringType() - .defaultValue("SQL") - .withDescription("Optional flag to specify timestamp format, SQL by default." + - " Option ISO-8601 will parse input timestamp in \"yyyy-MM-ddTHH:mm:ss.s{precision}\" format and output timestamp in the same format." + - " Option SQL will parse input timestamp in \"yyyy-MM-dd HH:mm:ss.s{precision}\" format and output timestamp in the same format."); - - // -------------------------------------------------------------------------------------------- - // Option enumerations - // -------------------------------------------------------------------------------------------- - - public static final String SQL = "SQL"; - public static final String ISO_8601 = "ISO-8601"; - - public static final Set TIMESTAMP_FORMAT_ENUM = new HashSet<>(Arrays.asList( - SQL, - ISO_8601 - )); - - // The handling mode of null key for map data - public static final String JSON_MAP_NULL_KEY_MODE_FAIL = "FAIL"; - public static final String JSON_MAP_NULL_KEY_MODE_DROP = "DROP"; - public static final String JSON_MAP_NULL_KEY_MODE_LITERAL = "LITERAL"; - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - public static TimestampFormat getTimestampFormat(ReadableConfig config){ - String timestampFormat = config.get(TIMESTAMP_FORMAT); - switch (timestampFormat){ - case SQL: - return TimestampFormat.SQL; - case ISO_8601: - return TimestampFormat.ISO_8601; - default: - throw new TableException( - String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); - } - } - - /** - * Creates handling mode for null key map data. - * - *

See {@link #JSON_MAP_NULL_KEY_MODE_FAIL}, {@link #JSON_MAP_NULL_KEY_MODE_DROP}, - * and {@link #JSON_MAP_NULL_KEY_MODE_LITERAL} for more information. - */ - public static MapNullKeyMode getMapNullKeyMode(ReadableConfig config){ - String mapNullKeyMode = config.get(MAP_NULL_KEY_MODE); - switch (mapNullKeyMode.toUpperCase()){ - case JSON_MAP_NULL_KEY_MODE_FAIL: - return MapNullKeyMode.FAIL; - case JSON_MAP_NULL_KEY_MODE_DROP: - return MapNullKeyMode.DROP; - case JSON_MAP_NULL_KEY_MODE_LITERAL: - return MapNullKeyMode.LITERAL; - default: - throw new TableException( - String.format("Unsupported map null key handling mode '%s'. Validator should have checked that.", mapNullKeyMode)); - } - } - - // -------------------------------------------------------------------------------------------- - // Inner classes - // -------------------------------------------------------------------------------------------- - - /** Handling mode for map data with null key. */ - public enum MapNullKeyMode { - FAIL, - DROP, - LITERAL - } - - // -------------------------------------------------------------------------------------------- - // Validation - // -------------------------------------------------------------------------------------------- - - /** - * Validator for json decoding format. - */ - public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - boolean failOnMissingField = tableOptions.get(FAIL_ON_MISSING_FIELD); - boolean ignoreParseErrors = tableOptions.get(IGNORE_PARSE_ERRORS); - if (ignoreParseErrors && failOnMissingField) { - throw new ValidationException(FAIL_ON_MISSING_FIELD.key() - + " and " - + IGNORE_PARSE_ERRORS.key() - + " shouldn't both be true."); - } - validateTimestampFormat(tableOptions); - } - - /** - * Validator for json encoding format. - */ - public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - // validator for {@link MAP_NULL_KEY_MODE} - Set nullKeyModes = Arrays.stream(MapNullKeyMode.values()) - .map(Objects::toString) - .collect(Collectors.toSet()); - if (!nullKeyModes.contains(tableOptions.get(MAP_NULL_KEY_MODE).toUpperCase())){ - throw new ValidationException(String.format( - "Unsupported value '%s' for option %s. Supported values are %s.", - tableOptions.get(MAP_NULL_KEY_MODE), - MAP_NULL_KEY_MODE.key(), - nullKeyModes)); - } - validateTimestampFormat(tableOptions); - } - - /** - * Validates timestamp format which value should be SQL or ISO-8601. - */ - static void validateTimestampFormat(ReadableConfig tableOptions) { - String timestampFormat = tableOptions.get(TIMESTAMP_FORMAT); - if (!TIMESTAMP_FORMAT_ENUM.contains(timestampFormat)){ - throw new ValidationException(String.format("Unsupported value '%s' for %s. Supported values are [SQL, ISO-8601].", - timestampFormat, TIMESTAMP_FORMAT.key())); - } - } + public static final ConfigOption FAIL_ON_MISSING_FIELD = + ConfigOptions.key("fail-on-missing-field") + .booleanType() + .defaultValue(false) + .withDescription( + "Optional flag to specify whether to fail if a field is missing or not, false by default."); + + public static final ConfigOption IGNORE_PARSE_ERRORS = + ConfigOptions.key("ignore-parse-errors") + .booleanType() + .defaultValue(false) + .withDescription( + "Optional flag to skip fields and rows with parse errors instead of failing;\n" + + "fields are set to null in case of errors, false by default."); + + public static final ConfigOption MAP_NULL_KEY_MODE = + ConfigOptions.key("map-null-key.mode") + .stringType() + .defaultValue("FAIL") + .withDescription( + "Optional flag to control the handling mode when serializing null key for map data, FAIL by default." + + " Option DROP will drop null key entries for map data." + + " Option LITERAL will use 'map-null-key.literal' as key literal."); + + public static final ConfigOption MAP_NULL_KEY_LITERAL = + ConfigOptions.key("map-null-key.literal") + .stringType() + .defaultValue("null") + .withDescription( + "Optional flag to specify string literal for null keys when 'map-null-key.mode' is LITERAL, \"null\" by default."); + + public static final ConfigOption TIMESTAMP_FORMAT = + ConfigOptions.key("timestamp-format.standard") + .stringType() + .defaultValue("SQL") + .withDescription( + "Optional flag to specify timestamp format, SQL by default." + + " Option ISO-8601 will parse input timestamp in \"yyyy-MM-ddTHH:mm:ss.s{precision}\" format and output timestamp in the same format." + + " Option SQL will parse input timestamp in \"yyyy-MM-dd HH:mm:ss.s{precision}\" format and output timestamp in the same format."); + + // -------------------------------------------------------------------------------------------- + // Option enumerations + // -------------------------------------------------------------------------------------------- + + public static final String SQL = "SQL"; + public static final String ISO_8601 = "ISO-8601"; + + public static final Set TIMESTAMP_FORMAT_ENUM = + new HashSet<>(Arrays.asList(SQL, ISO_8601)); + + // The handling mode of null key for map data + public static final String JSON_MAP_NULL_KEY_MODE_FAIL = "FAIL"; + public static final String JSON_MAP_NULL_KEY_MODE_DROP = "DROP"; + public static final String JSON_MAP_NULL_KEY_MODE_LITERAL = "LITERAL"; + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + public static TimestampFormat getTimestampFormat(ReadableConfig config) { + String timestampFormat = config.get(TIMESTAMP_FORMAT); + switch (timestampFormat) { + case SQL: + return TimestampFormat.SQL; + case ISO_8601: + return TimestampFormat.ISO_8601; + default: + throw new TableException( + String.format( + "Unsupported timestamp format '%s'. Validator should have checked that.", + timestampFormat)); + } + } + + /** + * Creates handling mode for null key map data. + * + *

See {@link #JSON_MAP_NULL_KEY_MODE_FAIL}, {@link #JSON_MAP_NULL_KEY_MODE_DROP}, and {@link + * #JSON_MAP_NULL_KEY_MODE_LITERAL} for more information. + */ + public static MapNullKeyMode getMapNullKeyMode(ReadableConfig config) { + String mapNullKeyMode = config.get(MAP_NULL_KEY_MODE); + switch (mapNullKeyMode.toUpperCase()) { + case JSON_MAP_NULL_KEY_MODE_FAIL: + return MapNullKeyMode.FAIL; + case JSON_MAP_NULL_KEY_MODE_DROP: + return MapNullKeyMode.DROP; + case JSON_MAP_NULL_KEY_MODE_LITERAL: + return MapNullKeyMode.LITERAL; + default: + throw new TableException( + String.format( + "Unsupported map null key handling mode '%s'. Validator should have checked that.", + mapNullKeyMode)); + } + } + + // -------------------------------------------------------------------------------------------- + // Inner classes + // -------------------------------------------------------------------------------------------- + + /** Handling mode for map data with null key. */ + public enum MapNullKeyMode { + FAIL, + DROP, + LITERAL + } + + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- + + /** Validator for json decoding format. */ + public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + boolean failOnMissingField = tableOptions.get(FAIL_ON_MISSING_FIELD); + boolean ignoreParseErrors = tableOptions.get(IGNORE_PARSE_ERRORS); + if (ignoreParseErrors && failOnMissingField) { + throw new ValidationException( + FAIL_ON_MISSING_FIELD.key() + + " and " + + IGNORE_PARSE_ERRORS.key() + + " shouldn't both be true."); + } + validateTimestampFormat(tableOptions); + } + + /** Validator for json encoding format. */ + public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + // validator for {@link MAP_NULL_KEY_MODE} + Set nullKeyModes = + Arrays.stream(MapNullKeyMode.values()) + .map(Objects::toString) + .collect(Collectors.toSet()); + if (!nullKeyModes.contains(tableOptions.get(MAP_NULL_KEY_MODE).toUpperCase())) { + throw new ValidationException( + String.format( + "Unsupported value '%s' for option %s. Supported values are %s.", + tableOptions.get(MAP_NULL_KEY_MODE), + MAP_NULL_KEY_MODE.key(), + nullKeyModes)); + } + validateTimestampFormat(tableOptions); + } + + /** Validates timestamp format which value should be SQL or ISO-8601. */ + static void validateTimestampFormat(ReadableConfig tableOptions) { + String timestampFormat = tableOptions.get(TIMESTAMP_FORMAT); + if (!TIMESTAMP_FORMAT_ENUM.contains(timestampFormat)) { + throw new ValidationException( + String.format( + "Unsupported value '%s' for %s. Supported values are [SQL, ISO-8601].", + timestampFormat, TIMESTAMP_FORMAT.key())); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index 0a2fb5ef1..a0c417563 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -39,104 +39,100 @@ /** * Deserialization schema from JSON to Flink Table/SQL internal data structure {@link RowData}. * - *

Deserializes a byte[] message as a JSON object and reads - * the specified fields. + *

Deserializes a byte[] message as a JSON object and reads the specified fields. * *

Failures during deserialization are forwarded as wrapped IOExceptions. */ @Internal public class JsonRowDataDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - - /** Flag indicating whether to fail if a field is missing. */ - private final boolean failOnMissingField; - - /** - * Flag indicating whether to ignore invalid fields/rows (default: throw an exception). - */ - private final boolean ignoreParseErrors; - - /** - * TypeInformation of the produced {@link RowData}. - **/ - private final TypeInformation resultTypeInfo; - - /** - * Runtime converter that converts {@link JsonNode}s into - * objects of Flink SQL internal data structures. - **/ - private final JsonToRowDataConverters.JsonToRowDataConverter runtimeConverter; - - /** - * Object mapper for parsing the JSON. - */ - private final ObjectMapper objectMapper = new ObjectMapper(); - - /** Timestamp format specification which is used to parse timestamp. */ - private final TimestampFormat timestampFormat; - - public JsonRowDataDeserializationSchema( - RowType rowType, - TypeInformation resultTypeInfo, - boolean failOnMissingField, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - if (ignoreParseErrors && failOnMissingField) { - throw new IllegalArgumentException( - "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."); - } - this.resultTypeInfo = checkNotNull(resultTypeInfo); - this.failOnMissingField = failOnMissingField; - this.ignoreParseErrors = ignoreParseErrors; - this.runtimeConverter = new JsonToRowDataConverters(failOnMissingField, ignoreParseErrors, timestampFormat) - .createConverter(checkNotNull(rowType)); - this.timestampFormat = timestampFormat; - boolean hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t instanceof DecimalType); - if (hasDecimalType) { - objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); - } - } - - @Override - public RowData deserialize(byte[] message) throws IOException { - try { - final JsonNode root = objectMapper.readTree(message); - return (RowData) runtimeConverter.convert(root); - } catch (Throwable t) { - if (ignoreParseErrors) { - return null; - } - throw new IOException(format("Failed to deserialize JSON '%s'.", new String(message)), t); - } - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return resultTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - JsonRowDataDeserializationSchema that = (JsonRowDataDeserializationSchema) o; - return failOnMissingField == that.failOnMissingField && - ignoreParseErrors == that.ignoreParseErrors && - resultTypeInfo.equals(that.resultTypeInfo) && - timestampFormat.equals(that.timestampFormat); - } - - @Override - public int hashCode() { - return Objects.hash(failOnMissingField, ignoreParseErrors, resultTypeInfo, timestampFormat); - } + private static final long serialVersionUID = 1L; + + /** Flag indicating whether to fail if a field is missing. */ + private final boolean failOnMissingField; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + /** TypeInformation of the produced {@link RowData}. */ + private final TypeInformation resultTypeInfo; + + /** + * Runtime converter that converts {@link JsonNode}s into objects of Flink SQL internal data + * structures. + */ + private final JsonToRowDataConverters.JsonToRowDataConverter runtimeConverter; + + /** Object mapper for parsing the JSON. */ + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Timestamp format specification which is used to parse timestamp. */ + private final TimestampFormat timestampFormat; + + public JsonRowDataDeserializationSchema( + RowType rowType, + TypeInformation resultTypeInfo, + boolean failOnMissingField, + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { + if (ignoreParseErrors && failOnMissingField) { + throw new IllegalArgumentException( + "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."); + } + this.resultTypeInfo = checkNotNull(resultTypeInfo); + this.failOnMissingField = failOnMissingField; + this.ignoreParseErrors = ignoreParseErrors; + this.runtimeConverter = + new JsonToRowDataConverters(failOnMissingField, ignoreParseErrors, timestampFormat) + .createConverter(checkNotNull(rowType)); + this.timestampFormat = timestampFormat; + boolean hasDecimalType = + LogicalTypeChecks.hasNested(rowType, t -> t instanceof DecimalType); + if (hasDecimalType) { + objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); + } + } + + @Override + public RowData deserialize(byte[] message) throws IOException { + try { + final JsonNode root = objectMapper.readTree(message); + return (RowData) runtimeConverter.convert(root); + } catch (Throwable t) { + if (ignoreParseErrors) { + return null; + } + throw new IOException( + format("Failed to deserialize JSON '%s'.", new String(message)), t); + } + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return resultTypeInfo; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JsonRowDataDeserializationSchema that = (JsonRowDataDeserializationSchema) o; + return failOnMissingField == that.failOnMissingField + && ignoreParseErrors == that.ignoreParseErrors + && resultTypeInfo.equals(that.resultTypeInfo) + && timestampFormat.equals(that.timestampFormat); + } + + @Override + public int hashCode() { + return Objects.hash(failOnMissingField, ignoreParseErrors, resultTypeInfo, timestampFormat); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index 6b1bae632..a7d3d674c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -29,82 +29,84 @@ import java.util.Objects; /** - * Serialization schema that serializes an object of Flink internal data structure into a JSON bytes. + * Serialization schema that serializes an object of Flink internal data structure into a JSON + * bytes. * - *

Serializes the input Flink object into a JSON string and - * converts it into byte[]. + *

Serializes the input Flink object into a JSON string and converts it into byte[]. * - *

Result byte[] messages can be deserialized using {@link JsonRowDataDeserializationSchema}. + *

Result byte[] messages can be deserialized using {@link + * JsonRowDataDeserializationSchema}. */ @Internal public class JsonRowDataSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = 1L; - - /** RowType to generate the runtime converter. */ - private final RowType rowType; - - /** The converter that converts internal data formats to JsonNode. */ - private final RowDataToJsonConverters.RowDataToJsonConverter runtimeConverter; - - /** Object mapper that is used to create output JSON objects. */ - private final ObjectMapper mapper = new ObjectMapper(); - - /** Reusable object node. */ - private transient ObjectNode node; - - /** Timestamp format specification which is used to parse timestamp. */ - private final TimestampFormat timestampFormat; - - /** The handling mode when serializing null keys for map data. */ - private final JsonOptions.MapNullKeyMode mapNullKeyMode; - - /** The string literal when handling mode for map null key LITERAL. is */ - private final String mapNullKeyLiteral; - - public JsonRowDataSerializationSchema( - RowType rowType, - TimestampFormat timestampFormat, - JsonOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral) { - this.rowType = rowType; - this.timestampFormat = timestampFormat; - this.mapNullKeyMode = mapNullKeyMode; - this.mapNullKeyLiteral = mapNullKeyLiteral; - this.runtimeConverter = new RowDataToJsonConverters( - timestampFormat, mapNullKeyMode, mapNullKeyLiteral).createConverter(rowType); - } - - @Override - public byte[] serialize(RowData row) { - if (node == null) { - node = mapper.createObjectNode(); - } - - try { - runtimeConverter.convert(mapper, node, row); - return mapper.writeValueAsBytes(node); - } catch (Throwable t) { - throw new RuntimeException("Could not serialize row '" + row + "'. ", t); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - JsonRowDataSerializationSchema that = (JsonRowDataSerializationSchema) o; - return rowType.equals(that.rowType) && - timestampFormat.equals(that.timestampFormat) && - mapNullKeyMode.equals(that.mapNullKeyMode) && - mapNullKeyLiteral.equals(that.mapNullKeyLiteral); - } - - @Override - public int hashCode() { - return Objects.hash(rowType, timestampFormat, mapNullKeyMode, mapNullKeyLiteral); - } + private static final long serialVersionUID = 1L; + + /** RowType to generate the runtime converter. */ + private final RowType rowType; + + /** The converter that converts internal data formats to JsonNode. */ + private final RowDataToJsonConverters.RowDataToJsonConverter runtimeConverter; + + /** Object mapper that is used to create output JSON objects. */ + private final ObjectMapper mapper = new ObjectMapper(); + + /** Reusable object node. */ + private transient ObjectNode node; + + /** Timestamp format specification which is used to parse timestamp. */ + private final TimestampFormat timestampFormat; + + /** The handling mode when serializing null keys for map data. */ + private final JsonOptions.MapNullKeyMode mapNullKeyMode; + + /** The string literal when handling mode for map null key LITERAL. is */ + private final String mapNullKeyLiteral; + + public JsonRowDataSerializationSchema( + RowType rowType, + TimestampFormat timestampFormat, + JsonOptions.MapNullKeyMode mapNullKeyMode, + String mapNullKeyLiteral) { + this.rowType = rowType; + this.timestampFormat = timestampFormat; + this.mapNullKeyMode = mapNullKeyMode; + this.mapNullKeyLiteral = mapNullKeyLiteral; + this.runtimeConverter = + new RowDataToJsonConverters(timestampFormat, mapNullKeyMode, mapNullKeyLiteral) + .createConverter(rowType); + } + + @Override + public byte[] serialize(RowData row) { + if (node == null) { + node = mapper.createObjectNode(); + } + + try { + runtimeConverter.convert(mapper, node, row); + return mapper.writeValueAsBytes(node); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'. ", t); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JsonRowDataSerializationSchema that = (JsonRowDataSerializationSchema) o; + return rowType.equals(that.rowType) + && timestampFormat.equals(that.timestampFormat) + && mapNullKeyMode.equals(that.mapNullKeyMode) + && mapNullKeyLiteral.equals(that.mapNullKeyLiteral); + } + + @Override + public int hashCode() { + return Objects.hash(rowType, timestampFormat, mapNullKeyMode, mapNullKeyLiteral); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index acb96a514..f6aa03bd1 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -74,499 +74,502 @@ /** * Deserialization schema from JSON to Flink types. * - *

Deserializes a byte[] message as a JSON object and reads - * the specified fields. + *

Deserializes a byte[] message as a JSON object and reads the specified fields. * *

Failures during deserialization are forwarded as wrapped IOExceptions. */ @PublicEvolving public class JsonRowDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = -228294330688809195L; - - /** Type information describing the result type. */ - private final RowTypeInfo typeInfo; - - private boolean failOnMissingField; - - /** Object mapper for parsing the JSON. */ - private final ObjectMapper objectMapper = new ObjectMapper(); - - private DeserializationRuntimeConverter runtimeConverter; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - private JsonRowDeserializationSchema( - TypeInformation typeInfo, - boolean failOnMissingField, - boolean ignoreParseErrors) { - checkNotNull(typeInfo, "Type information"); - checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); - if (ignoreParseErrors && failOnMissingField) { - throw new IllegalArgumentException( - "JSON format doesn't support failOnMissingField and ignoreParseErrors are both true."); - } - this.typeInfo = (RowTypeInfo) typeInfo; - this.failOnMissingField = failOnMissingField; - this.runtimeConverter = createConverter(this.typeInfo); - this.ignoreParseErrors = ignoreParseErrors; - RowType rowType = (RowType) fromLegacyInfoToDataType(this.typeInfo).getLogicalType(); - boolean hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t.getTypeRoot().equals(DECIMAL)); - if (hasDecimalType) { - objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); - } - } - - /** - * @deprecated Use the provided {@link Builder} instead. - */ - @Deprecated - public JsonRowDeserializationSchema(TypeInformation typeInfo) { - this(typeInfo, false, false); - } - - /** - * @deprecated Use the provided {@link Builder} instead. - */ - @Deprecated - public JsonRowDeserializationSchema(String jsonSchema) { - this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)), false, false); - } - - /** - * @deprecated Use the provided {@link Builder} instead. - */ - @Deprecated - public void setFailOnMissingField(boolean failOnMissingField) { - // TODO make this class immutable once we drop this method - this.failOnMissingField = failOnMissingField; - this.runtimeConverter = createConverter(this.typeInfo); - } - - @Override - public Row deserialize(byte[] message) throws IOException { - try { - final JsonNode root = objectMapper.readTree(message); - return (Row) runtimeConverter.convert(objectMapper, root); - } catch (Throwable t) { - if (ignoreParseErrors) { - return null; - } - throw new IOException(format("Failed to deserialize JSON '%s'.", new String(message)), t); - } - } - - @Override - public boolean isEndOfStream(Row nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return typeInfo; - } - - /** - * Builder for {@link JsonRowDeserializationSchema}. - */ - public static class Builder { - - private final RowTypeInfo typeInfo; - private boolean failOnMissingField = false; - private boolean ignoreParseErrors = false; - - /** - * Creates a JSON deserialization schema for the given type information. - * - * @param typeInfo Type information describing the result type. The field names of {@link Row} - * are used to parse the JSON properties. - */ - public Builder(TypeInformation typeInfo) { - checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); - this.typeInfo = (RowTypeInfo) typeInfo; - } - - /** - * Creates a JSON deserialization schema for the given JSON schema. - * - * @param jsonSchema JSON schema describing the result type - * - * @see http://json-schema.org/ - */ - public Builder(String jsonSchema) { - this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema))); - } - - /** - * Configures schema to fail if a JSON field is missing. - * - *

By default, a missing field is ignored and the field is set to null. - */ - public Builder failOnMissingField() { - this.failOnMissingField = true; - return this; - } - - /** - * Configures schema to fail when parsing json failed. - * - *

By default, an exception will be thrown when parsing json fails. - */ - public Builder ignoreParseErrors() { - this.ignoreParseErrors = true; - return this; - } - - public JsonRowDeserializationSchema build() { - return new JsonRowDeserializationSchema(typeInfo, failOnMissingField, ignoreParseErrors); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - final JsonRowDeserializationSchema that = (JsonRowDeserializationSchema) o; - return Objects.equals(typeInfo, that.typeInfo) && - Objects.equals(failOnMissingField, that.failOnMissingField) && - Objects.equals(ignoreParseErrors, that.ignoreParseErrors); - } - - @Override - public int hashCode() { - return Objects.hash(typeInfo, failOnMissingField, ignoreParseErrors); - } - - /* - Runtime converter - */ - - /** - * Runtime converter that maps between {@link JsonNode}s and Java objects. - */ - @FunctionalInterface - private interface DeserializationRuntimeConverter extends Serializable { - Object convert(ObjectMapper mapper, JsonNode jsonNode); - } - - private DeserializationRuntimeConverter createConverter(TypeInformation typeInfo) { - DeserializationRuntimeConverter baseConverter = createConverterForSimpleType(typeInfo) - .orElseGet(() -> - createContainerConverter(typeInfo) - .orElseGet(() -> createFallbackConverter(typeInfo.getTypeClass()))); - return wrapIntoNullableConverter(baseConverter); - } - - private DeserializationRuntimeConverter wrapIntoNullableConverter(DeserializationRuntimeConverter converter) { - return (mapper, jsonNode) -> { - if (jsonNode.isNull()) { - return null; - } - try { - return converter.convert(mapper, jsonNode); - } catch (Throwable t) { - if (!ignoreParseErrors) { - throw t; - } - return null; - } - }; - } - - private Optional createContainerConverter(TypeInformation typeInfo) { - if (typeInfo instanceof RowTypeInfo) { - return Optional.of(createRowConverter((RowTypeInfo) typeInfo)); - } else if (typeInfo instanceof ObjectArrayTypeInfo) { - return Optional.of(createObjectArrayConverter(((ObjectArrayTypeInfo) typeInfo).getComponentInfo())); - } else if (typeInfo instanceof BasicArrayTypeInfo) { - return Optional.of(createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo())); - } else if (isPrimitiveByteArray(typeInfo)) { - return Optional.of(createByteArrayConverter()); - } else if (typeInfo instanceof MapTypeInfo) { - MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo; - return Optional.of(createMapConverter(mapTypeInfo.getKeyTypeInfo(), mapTypeInfo.getValueTypeInfo())); - } else { - return Optional.empty(); - } - } - - private DeserializationRuntimeConverter createMapConverter(TypeInformation keyType, TypeInformation valueType) { - DeserializationRuntimeConverter valueConverter = createConverter(valueType); - DeserializationRuntimeConverter keyConverter = createConverter(keyType); - return (mapper, jsonNode) -> { - Iterator> fields = jsonNode.fields(); - Map result = new HashMap<>(); - while (fields.hasNext()) { - Map.Entry entry = fields.next(); - Object key = keyConverter.convert(mapper, TextNode.valueOf(entry.getKey())); - Object value = valueConverter.convert(mapper, entry.getValue()); - result.put(key, value); - } - return result; - }; - } - - private DeserializationRuntimeConverter createByteArrayConverter() { - return (mapper, jsonNode) -> { - try { - return jsonNode.binaryValue(); - } catch (IOException e) { - throw new JsonParseException("Unable to deserialize byte array.", e); - } - }; - } - - private boolean isPrimitiveByteArray(TypeInformation typeInfo) { - return typeInfo instanceof PrimitiveArrayTypeInfo && - ((PrimitiveArrayTypeInfo) typeInfo).getComponentType() == Types.BYTE; - } - - private DeserializationRuntimeConverter createObjectArrayConverter(TypeInformation elementTypeInfo) { - DeserializationRuntimeConverter elementConverter = createConverter(elementTypeInfo); - return assembleArrayConverter(elementTypeInfo, elementConverter); - } - - private DeserializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) { - List fieldConverters = Arrays.stream(typeInfo.getFieldTypes()) - .map(this::createConverter) - .collect(Collectors.toList()); - - return assembleRowConverter(typeInfo.getFieldNames(), fieldConverters); - } - - private DeserializationRuntimeConverter createFallbackConverter(Class valueType) { - return (mapper, jsonNode) -> { - // for types that were specified without JSON schema - // e.g. POJOs - try { - return mapper.treeToValue(jsonNode, valueType); - } catch (JsonProcessingException e) { - throw new JsonParseException(format("Could not convert node: %s", jsonNode), e); - } - }; - } - - private Optional createConverterForSimpleType(TypeInformation simpleTypeInfo) { - if (simpleTypeInfo == Types.VOID) { - return Optional.of((mapper, jsonNode) -> null); - } else if (simpleTypeInfo == Types.BOOLEAN) { - return Optional.of(this::convertToBoolean); - } else if (simpleTypeInfo == Types.STRING) { - return Optional.of(this::convertToString); - } else if (simpleTypeInfo == Types.INT) { - return Optional.of(this::convertToInt); - } else if (simpleTypeInfo == Types.LONG) { - return Optional.of(this::convertToLong); - } else if (simpleTypeInfo == Types.DOUBLE) { - return Optional.of(this::convertToDouble); - } else if (simpleTypeInfo == Types.FLOAT) { - return Optional.of((mapper, jsonNode) -> Float.parseFloat(jsonNode.asText().trim())); - } else if (simpleTypeInfo == Types.SHORT) { - return Optional.of((mapper, jsonNode) -> Short.parseShort(jsonNode.asText().trim())); - } else if (simpleTypeInfo == Types.BYTE) { - return Optional.of((mapper, jsonNode) -> Byte.parseByte(jsonNode.asText().trim())); - } else if (simpleTypeInfo == Types.BIG_DEC) { - return Optional.of(this::convertToBigDecimal); - } else if (simpleTypeInfo == Types.BIG_INT) { - return Optional.of(this::convertToBigInteger); - } else if (simpleTypeInfo == Types.SQL_DATE) { - return Optional.of(this::convertToDate); - } else if (simpleTypeInfo == Types.SQL_TIME) { - return Optional.of(this::convertToTime); - } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) { - return Optional.of(this::convertToTimestamp); - } else if (simpleTypeInfo == Types.LOCAL_DATE) { - return Optional.of(this::convertToLocalDate); - } else if (simpleTypeInfo == Types.LOCAL_TIME) { - return Optional.of(this::convertToLocalTime); - } else if (simpleTypeInfo == Types.LOCAL_DATE_TIME) { - return Optional.of(this::convertToLocalDateTime); - } else { - return Optional.empty(); - } - } - - private String convertToString(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.isContainerNode()) { - return jsonNode.toString(); - } else { - return jsonNode.asText(); - } - } - - private boolean convertToBoolean(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.isBoolean()) { - // avoid redundant toString and parseBoolean, for better performance - return jsonNode.asBoolean(); - } else { - return Boolean.parseBoolean(jsonNode.asText().trim()); - } - } - - private int convertToInt(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.canConvertToInt()) { - // avoid redundant toString and parseInt, for better performance - return jsonNode.asInt(); - } else { - return Integer.parseInt(jsonNode.asText().trim()); - } - } - - private long convertToLong(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.canConvertToLong()) { - // avoid redundant toString and parseLong, for better performance - return jsonNode.asLong(); - } else { - return Long.parseLong(jsonNode.asText().trim()); - } - } - - private double convertToDouble(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.isDouble()) { - // avoid redundant toString and parseDouble, for better performance - return jsonNode.asDouble(); - } else { - return Double.parseDouble(jsonNode.asText().trim()); - } - } - - private BigDecimal convertToBigDecimal(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.isBigDecimal()) { - // avoid redundant toString and toDecimal, for better performance - return jsonNode.decimalValue(); - } else { - return new BigDecimal(jsonNode.asText().trim()); - } - } - - private BigInteger convertToBigInteger(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.isBigInteger()) { - // avoid redundant toString and toBigInteger, for better performance - return jsonNode.bigIntegerValue(); - } else { - return new BigInteger(jsonNode.asText().trim()); - } - } - - private LocalDate convertToLocalDate(ObjectMapper mapper, JsonNode jsonNode) { - return ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); - } - - private Date convertToDate(ObjectMapper mapper, JsonNode jsonNode) { - return Date.valueOf(convertToLocalDate(mapper, jsonNode)); - } - - private LocalDateTime convertToLocalDateTime(ObjectMapper mapper, JsonNode jsonNode) { - // according to RFC 3339 every date-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - TemporalAccessor parsedTimestamp = RFC3339_TIMESTAMP_FORMAT.parse(jsonNode.asText()); - - ZoneOffset zoneOffset = parsedTimestamp.query(TemporalQueries.offset()); - - if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0) { - throw new IllegalStateException( - "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + - "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); - } - - LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); - LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); - - return LocalDateTime.of(localDate, localTime); - } - - private Timestamp convertToTimestamp(ObjectMapper mapper, JsonNode jsonNode) { - return Timestamp.valueOf(convertToLocalDateTime(mapper, jsonNode)); - } - - private LocalTime convertToLocalTime(ObjectMapper mapper, JsonNode jsonNode) { - // according to RFC 3339 every full-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - - TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText()); - - ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset()); - LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); - - if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0 || localTime.getNano() != 0) { - throw new IllegalStateException( - "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet."); - } - - return localTime; - } - - private Time convertToTime(ObjectMapper mapper, JsonNode jsonNode) { - return Time.valueOf(convertToLocalTime(mapper, jsonNode)); - } - - private DeserializationRuntimeConverter assembleRowConverter( - String[] fieldNames, - List fieldConverters) { - return (mapper, jsonNode) -> { - ObjectNode node = (ObjectNode) jsonNode; - int arity = fieldNames.length; - Row row = new Row(arity); - for (int i = 0; i < arity; i++) { - String fieldName = fieldNames[i]; - JsonNode field = node.get(fieldName); - Object convertField = convertField(mapper, fieldConverters.get(i), fieldName, field); - row.setField(i, convertField); - } - - return row; - }; - } - - private Object convertField( - ObjectMapper mapper, - DeserializationRuntimeConverter fieldConverter, - String fieldName, - JsonNode field) { - if (field == null) { - if (failOnMissingField) { - throw new IllegalStateException( - "Could not find field with name '" + fieldName + "'."); - } else { - return null; - } - } else { - return fieldConverter.convert(mapper, field); - } - } - - private DeserializationRuntimeConverter assembleArrayConverter( - TypeInformation elementType, - DeserializationRuntimeConverter elementConverter) { - - final Class elementClass = elementType.getTypeClass(); - - return (mapper, jsonNode) -> { - final ArrayNode node = (ArrayNode) jsonNode; - final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); - for (int i = 0; i < node.size(); i++) { - final JsonNode innerNode = node.get(i); - array[i] = elementConverter.convert(mapper, innerNode); - } - - return array; - }; - } - - /** - * Exception which refers to parse errors in converters. - * */ - private static final class JsonParseException extends RuntimeException { - private static final long serialVersionUID = 1L; - public JsonParseException(String message, Throwable cause) { - super(message, cause); - } - } + private static final long serialVersionUID = -228294330688809195L; + + /** Type information describing the result type. */ + private final RowTypeInfo typeInfo; + + private boolean failOnMissingField; + + /** Object mapper for parsing the JSON. */ + private final ObjectMapper objectMapper = new ObjectMapper(); + + private DeserializationRuntimeConverter runtimeConverter; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + private JsonRowDeserializationSchema( + TypeInformation typeInfo, boolean failOnMissingField, boolean ignoreParseErrors) { + checkNotNull(typeInfo, "Type information"); + checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + if (ignoreParseErrors && failOnMissingField) { + throw new IllegalArgumentException( + "JSON format doesn't support failOnMissingField and ignoreParseErrors are both true."); + } + this.typeInfo = (RowTypeInfo) typeInfo; + this.failOnMissingField = failOnMissingField; + this.runtimeConverter = createConverter(this.typeInfo); + this.ignoreParseErrors = ignoreParseErrors; + RowType rowType = (RowType) fromLegacyInfoToDataType(this.typeInfo).getLogicalType(); + boolean hasDecimalType = + LogicalTypeChecks.hasNested(rowType, t -> t.getTypeRoot().equals(DECIMAL)); + if (hasDecimalType) { + objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); + } + } + + /** @deprecated Use the provided {@link Builder} instead. */ + @Deprecated + public JsonRowDeserializationSchema(TypeInformation typeInfo) { + this(typeInfo, false, false); + } + + /** @deprecated Use the provided {@link Builder} instead. */ + @Deprecated + public JsonRowDeserializationSchema(String jsonSchema) { + this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)), false, false); + } + + /** @deprecated Use the provided {@link Builder} instead. */ + @Deprecated + public void setFailOnMissingField(boolean failOnMissingField) { + // TODO make this class immutable once we drop this method + this.failOnMissingField = failOnMissingField; + this.runtimeConverter = createConverter(this.typeInfo); + } + + @Override + public Row deserialize(byte[] message) throws IOException { + try { + final JsonNode root = objectMapper.readTree(message); + return (Row) runtimeConverter.convert(objectMapper, root); + } catch (Throwable t) { + if (ignoreParseErrors) { + return null; + } + throw new IOException( + format("Failed to deserialize JSON '%s'.", new String(message)), t); + } + } + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return typeInfo; + } + + /** Builder for {@link JsonRowDeserializationSchema}. */ + public static class Builder { + + private final RowTypeInfo typeInfo; + private boolean failOnMissingField = false; + private boolean ignoreParseErrors = false; + + /** + * Creates a JSON deserialization schema for the given type information. + * + * @param typeInfo Type information describing the result type. The field names of {@link + * Row} are used to parse the JSON properties. + */ + public Builder(TypeInformation typeInfo) { + checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + this.typeInfo = (RowTypeInfo) typeInfo; + } + + /** + * Creates a JSON deserialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * @see http://json-schema.org/ + */ + public Builder(String jsonSchema) { + this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema))); + } + + /** + * Configures schema to fail if a JSON field is missing. + * + *

By default, a missing field is ignored and the field is set to null. + */ + public Builder failOnMissingField() { + this.failOnMissingField = true; + return this; + } + + /** + * Configures schema to fail when parsing json failed. + * + *

By default, an exception will be thrown when parsing json fails. + */ + public Builder ignoreParseErrors() { + this.ignoreParseErrors = true; + return this; + } + + public JsonRowDeserializationSchema build() { + return new JsonRowDeserializationSchema( + typeInfo, failOnMissingField, ignoreParseErrors); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final JsonRowDeserializationSchema that = (JsonRowDeserializationSchema) o; + return Objects.equals(typeInfo, that.typeInfo) + && Objects.equals(failOnMissingField, that.failOnMissingField) + && Objects.equals(ignoreParseErrors, that.ignoreParseErrors); + } + + @Override + public int hashCode() { + return Objects.hash(typeInfo, failOnMissingField, ignoreParseErrors); + } + + /* + Runtime converter + */ + + /** Runtime converter that maps between {@link JsonNode}s and Java objects. */ + @FunctionalInterface + private interface DeserializationRuntimeConverter extends Serializable { + Object convert(ObjectMapper mapper, JsonNode jsonNode); + } + + private DeserializationRuntimeConverter createConverter(TypeInformation typeInfo) { + DeserializationRuntimeConverter baseConverter = + createConverterForSimpleType(typeInfo) + .orElseGet( + () -> + createContainerConverter(typeInfo) + .orElseGet( + () -> + createFallbackConverter( + typeInfo.getTypeClass()))); + return wrapIntoNullableConverter(baseConverter); + } + + private DeserializationRuntimeConverter wrapIntoNullableConverter( + DeserializationRuntimeConverter converter) { + return (mapper, jsonNode) -> { + if (jsonNode.isNull()) { + return null; + } + try { + return converter.convert(mapper, jsonNode); + } catch (Throwable t) { + if (!ignoreParseErrors) { + throw t; + } + return null; + } + }; + } + + private Optional createContainerConverter( + TypeInformation typeInfo) { + if (typeInfo instanceof RowTypeInfo) { + return Optional.of(createRowConverter((RowTypeInfo) typeInfo)); + } else if (typeInfo instanceof ObjectArrayTypeInfo) { + return Optional.of( + createObjectArrayConverter( + ((ObjectArrayTypeInfo) typeInfo).getComponentInfo())); + } else if (typeInfo instanceof BasicArrayTypeInfo) { + return Optional.of( + createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo())); + } else if (isPrimitiveByteArray(typeInfo)) { + return Optional.of(createByteArrayConverter()); + } else if (typeInfo instanceof MapTypeInfo) { + MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo; + return Optional.of( + createMapConverter( + mapTypeInfo.getKeyTypeInfo(), mapTypeInfo.getValueTypeInfo())); + } else { + return Optional.empty(); + } + } + + private DeserializationRuntimeConverter createMapConverter( + TypeInformation keyType, TypeInformation valueType) { + DeserializationRuntimeConverter valueConverter = createConverter(valueType); + DeserializationRuntimeConverter keyConverter = createConverter(keyType); + return (mapper, jsonNode) -> { + Iterator> fields = jsonNode.fields(); + Map result = new HashMap<>(); + while (fields.hasNext()) { + Map.Entry entry = fields.next(); + Object key = keyConverter.convert(mapper, TextNode.valueOf(entry.getKey())); + Object value = valueConverter.convert(mapper, entry.getValue()); + result.put(key, value); + } + return result; + }; + } + + private DeserializationRuntimeConverter createByteArrayConverter() { + return (mapper, jsonNode) -> { + try { + return jsonNode.binaryValue(); + } catch (IOException e) { + throw new JsonParseException("Unable to deserialize byte array.", e); + } + }; + } + + private boolean isPrimitiveByteArray(TypeInformation typeInfo) { + return typeInfo instanceof PrimitiveArrayTypeInfo + && ((PrimitiveArrayTypeInfo) typeInfo).getComponentType() == Types.BYTE; + } + + private DeserializationRuntimeConverter createObjectArrayConverter( + TypeInformation elementTypeInfo) { + DeserializationRuntimeConverter elementConverter = createConverter(elementTypeInfo); + return assembleArrayConverter(elementTypeInfo, elementConverter); + } + + private DeserializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) { + List fieldConverters = + Arrays.stream(typeInfo.getFieldTypes()) + .map(this::createConverter) + .collect(Collectors.toList()); + + return assembleRowConverter(typeInfo.getFieldNames(), fieldConverters); + } + + private DeserializationRuntimeConverter createFallbackConverter(Class valueType) { + return (mapper, jsonNode) -> { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.treeToValue(jsonNode, valueType); + } catch (JsonProcessingException e) { + throw new JsonParseException(format("Could not convert node: %s", jsonNode), e); + } + }; + } + + private Optional createConverterForSimpleType( + TypeInformation simpleTypeInfo) { + if (simpleTypeInfo == Types.VOID) { + return Optional.of((mapper, jsonNode) -> null); + } else if (simpleTypeInfo == Types.BOOLEAN) { + return Optional.of(this::convertToBoolean); + } else if (simpleTypeInfo == Types.STRING) { + return Optional.of(this::convertToString); + } else if (simpleTypeInfo == Types.INT) { + return Optional.of(this::convertToInt); + } else if (simpleTypeInfo == Types.LONG) { + return Optional.of(this::convertToLong); + } else if (simpleTypeInfo == Types.DOUBLE) { + return Optional.of(this::convertToDouble); + } else if (simpleTypeInfo == Types.FLOAT) { + return Optional.of((mapper, jsonNode) -> Float.parseFloat(jsonNode.asText().trim())); + } else if (simpleTypeInfo == Types.SHORT) { + return Optional.of((mapper, jsonNode) -> Short.parseShort(jsonNode.asText().trim())); + } else if (simpleTypeInfo == Types.BYTE) { + return Optional.of((mapper, jsonNode) -> Byte.parseByte(jsonNode.asText().trim())); + } else if (simpleTypeInfo == Types.BIG_DEC) { + return Optional.of(this::convertToBigDecimal); + } else if (simpleTypeInfo == Types.BIG_INT) { + return Optional.of(this::convertToBigInteger); + } else if (simpleTypeInfo == Types.SQL_DATE) { + return Optional.of(this::convertToDate); + } else if (simpleTypeInfo == Types.SQL_TIME) { + return Optional.of(this::convertToTime); + } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) { + return Optional.of(this::convertToTimestamp); + } else if (simpleTypeInfo == Types.LOCAL_DATE) { + return Optional.of(this::convertToLocalDate); + } else if (simpleTypeInfo == Types.LOCAL_TIME) { + return Optional.of(this::convertToLocalTime); + } else if (simpleTypeInfo == Types.LOCAL_DATE_TIME) { + return Optional.of(this::convertToLocalDateTime); + } else { + return Optional.empty(); + } + } + + private String convertToString(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isContainerNode()) { + return jsonNode.toString(); + } else { + return jsonNode.asText(); + } + } + + private boolean convertToBoolean(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isBoolean()) { + // avoid redundant toString and parseBoolean, for better performance + return jsonNode.asBoolean(); + } else { + return Boolean.parseBoolean(jsonNode.asText().trim()); + } + } + + private int convertToInt(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.canConvertToInt()) { + // avoid redundant toString and parseInt, for better performance + return jsonNode.asInt(); + } else { + return Integer.parseInt(jsonNode.asText().trim()); + } + } + + private long convertToLong(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.canConvertToLong()) { + // avoid redundant toString and parseLong, for better performance + return jsonNode.asLong(); + } else { + return Long.parseLong(jsonNode.asText().trim()); + } + } + + private double convertToDouble(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isDouble()) { + // avoid redundant toString and parseDouble, for better performance + return jsonNode.asDouble(); + } else { + return Double.parseDouble(jsonNode.asText().trim()); + } + } + + private BigDecimal convertToBigDecimal(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isBigDecimal()) { + // avoid redundant toString and toDecimal, for better performance + return jsonNode.decimalValue(); + } else { + return new BigDecimal(jsonNode.asText().trim()); + } + } + + private BigInteger convertToBigInteger(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isBigInteger()) { + // avoid redundant toString and toBigInteger, for better performance + return jsonNode.bigIntegerValue(); + } else { + return new BigInteger(jsonNode.asText().trim()); + } + } + + private LocalDate convertToLocalDate(ObjectMapper mapper, JsonNode jsonNode) { + return ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); + } + + private Date convertToDate(ObjectMapper mapper, JsonNode jsonNode) { + return Date.valueOf(convertToLocalDate(mapper, jsonNode)); + } + + private LocalDateTime convertToLocalDateTime(ObjectMapper mapper, JsonNode jsonNode) { + // according to RFC 3339 every date-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + TemporalAccessor parsedTimestamp = RFC3339_TIMESTAMP_FORMAT.parse(jsonNode.asText()); + + ZoneOffset zoneOffset = parsedTimestamp.query(TemporalQueries.offset()); + + if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0) { + throw new IllegalStateException( + "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + + "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + } + + LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); + LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); + + return LocalDateTime.of(localDate, localTime); + } + + private Timestamp convertToTimestamp(ObjectMapper mapper, JsonNode jsonNode) { + return Timestamp.valueOf(convertToLocalDateTime(mapper, jsonNode)); + } + + private LocalTime convertToLocalTime(ObjectMapper mapper, JsonNode jsonNode) { + // according to RFC 3339 every full-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + + TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText()); + + ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset()); + LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); + + if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0 || localTime.getNano() != 0) { + throw new IllegalStateException( + "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet."); + } + + return localTime; + } + + private Time convertToTime(ObjectMapper mapper, JsonNode jsonNode) { + return Time.valueOf(convertToLocalTime(mapper, jsonNode)); + } + + private DeserializationRuntimeConverter assembleRowConverter( + String[] fieldNames, List fieldConverters) { + return (mapper, jsonNode) -> { + ObjectNode node = (ObjectNode) jsonNode; + int arity = fieldNames.length; + Row row = new Row(arity); + for (int i = 0; i < arity; i++) { + String fieldName = fieldNames[i]; + JsonNode field = node.get(fieldName); + Object convertField = + convertField(mapper, fieldConverters.get(i), fieldName, field); + row.setField(i, convertField); + } + + return row; + }; + } + + private Object convertField( + ObjectMapper mapper, + DeserializationRuntimeConverter fieldConverter, + String fieldName, + JsonNode field) { + if (field == null) { + if (failOnMissingField) { + throw new IllegalStateException( + "Could not find field with name '" + fieldName + "'."); + } else { + return null; + } + } else { + return fieldConverter.convert(mapper, field); + } + } + + private DeserializationRuntimeConverter assembleArrayConverter( + TypeInformation elementType, DeserializationRuntimeConverter elementConverter) { + + final Class elementClass = elementType.getTypeClass(); + + return (mapper, jsonNode) -> { + final ArrayNode node = (ArrayNode) jsonNode; + final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); + for (int i = 0; i < node.size(); i++) { + final JsonNode innerNode = node.get(i); + array[i] = elementConverter.convert(mapper, innerNode); + } + + return array; + }; + } + + /** Exception which refers to parse errors in converters. */ + private static final class JsonParseException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public JsonParseException(String message, Throwable cause) { + super(message, cause); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java index 356594a27..00506e921 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -34,74 +34,81 @@ import java.util.Map; /** - * Table format factory for providing configured instances of JSON-to-row {@link SerializationSchema} - * and {@link DeserializationSchema}. + * Table format factory for providing configured instances of JSON-to-row {@link + * SerializationSchema} and {@link DeserializationSchema}. */ public class JsonRowFormatFactory extends TableFormatFactoryBase - implements SerializationSchemaFactory, DeserializationSchemaFactory { + implements SerializationSchemaFactory, DeserializationSchemaFactory { - public JsonRowFormatFactory() { - super(JsonValidator.FORMAT_TYPE_VALUE, 1, true); - } + public JsonRowFormatFactory() { + super(JsonValidator.FORMAT_TYPE_VALUE, 1, true); + } - @Override - protected List supportedFormatProperties() { - final List properties = new ArrayList<>(); - properties.add(JsonValidator.FORMAT_JSON_SCHEMA); - properties.add(JsonValidator.FORMAT_SCHEMA); - properties.add(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD); - properties.add(JsonValidator.FORMAT_IGNORE_PARSE_ERRORS); - return properties; - } + @Override + protected List supportedFormatProperties() { + final List properties = new ArrayList<>(); + properties.add(JsonValidator.FORMAT_JSON_SCHEMA); + properties.add(JsonValidator.FORMAT_SCHEMA); + properties.add(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD); + properties.add(JsonValidator.FORMAT_IGNORE_PARSE_ERRORS); + return properties; + } - @Override - public DeserializationSchema createDeserializationSchema(Map properties) { - final DescriptorProperties descriptorProperties = getValidatedProperties(properties); + @Override + public DeserializationSchema createDeserializationSchema(Map properties) { + final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - // create and configure - final JsonRowDeserializationSchema.Builder schema = - new JsonRowDeserializationSchema.Builder(createTypeInformation(descriptorProperties)); + // create and configure + final JsonRowDeserializationSchema.Builder schema = + new JsonRowDeserializationSchema.Builder( + createTypeInformation(descriptorProperties)); - descriptorProperties.getOptionalBoolean(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD) - .ifPresent(flag -> { - if (flag) { - schema.failOnMissingField(); - } - }); - descriptorProperties.getOptionalBoolean(JsonValidator.FORMAT_IGNORE_PARSE_ERRORS) - .ifPresent(flag -> { - if (flag) { - schema.ignoreParseErrors(); - } - }); - return schema.build(); - } + descriptorProperties + .getOptionalBoolean(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD) + .ifPresent( + flag -> { + if (flag) { + schema.failOnMissingField(); + } + }); + descriptorProperties + .getOptionalBoolean(JsonValidator.FORMAT_IGNORE_PARSE_ERRORS) + .ifPresent( + flag -> { + if (flag) { + schema.ignoreParseErrors(); + } + }); + return schema.build(); + } - @Override - public SerializationSchema createSerializationSchema(Map properties) { - final DescriptorProperties descriptorProperties = getValidatedProperties(properties); + @Override + public SerializationSchema createSerializationSchema(Map properties) { + final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - // create and configure - return new JsonRowSerializationSchema.Builder(createTypeInformation(descriptorProperties)).build(); - } + // create and configure + return new JsonRowSerializationSchema.Builder(createTypeInformation(descriptorProperties)) + .build(); + } - private TypeInformation createTypeInformation(DescriptorProperties descriptorProperties) { - if (descriptorProperties.containsKey(JsonValidator.FORMAT_SCHEMA)) { - return (RowTypeInfo) descriptorProperties.getType(JsonValidator.FORMAT_SCHEMA); - } else if (descriptorProperties.containsKey(JsonValidator.FORMAT_JSON_SCHEMA)) { - return JsonRowSchemaConverter.convert(descriptorProperties.getString(JsonValidator.FORMAT_JSON_SCHEMA)); - } else { - return deriveSchema(descriptorProperties.asMap()).toRowType(); - } - } + private TypeInformation createTypeInformation(DescriptorProperties descriptorProperties) { + if (descriptorProperties.containsKey(JsonValidator.FORMAT_SCHEMA)) { + return (RowTypeInfo) descriptorProperties.getType(JsonValidator.FORMAT_SCHEMA); + } else if (descriptorProperties.containsKey(JsonValidator.FORMAT_JSON_SCHEMA)) { + return JsonRowSchemaConverter.convert( + descriptorProperties.getString(JsonValidator.FORMAT_JSON_SCHEMA)); + } else { + return deriveSchema(descriptorProperties.asMap()).toRowType(); + } + } - private static DescriptorProperties getValidatedProperties(Map propertiesMap) { - final DescriptorProperties descriptorProperties = new DescriptorProperties(); - descriptorProperties.putProperties(propertiesMap); + private static DescriptorProperties getValidatedProperties(Map propertiesMap) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(); + descriptorProperties.putProperties(propertiesMap); - // validate - new JsonValidator().validate(descriptorProperties); + // validate + new JsonValidator().validate(descriptorProperties); - return descriptorProperties; - } + return descriptorProperties; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java index 320ca1f8c..fe4123204 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java @@ -41,322 +41,341 @@ * Converts a JSON schema into Flink's type information. It uses {@link Row} for representing * objects and tuple arrays. * - *

Note: This converter implements just a subset of the JSON schema specification. - * Union types (as well as "allOf", "anyOf", "not") are not supported yet. Simple - * references that link to a common definition in the document are supported. "oneOf" and - * arrays of types are only supported for specifying nullability. + *

Note: This converter implements just a subset of the JSON schema specification. Union types + * (as well as "allOf", "anyOf", "not") are not supported yet. Simple references that link to a + * common definition in the document are supported. "oneOf" and arrays of types are only supported + * for specifying nullability. * - *

This converter has been developed for JSON Schema draft-07 but also includes keywords of - * older drafts to be as compatible as possible. + *

This converter has been developed for JSON Schema draft-07 but also includes keywords of older + * drafts to be as compatible as possible. */ public final class JsonRowSchemaConverter { - private JsonRowSchemaConverter() { - // private - } - - // see https://spacetelescope.github.io/understanding-json-schema/UnderstandingJSONSchema.pdf - private static final String PROPERTIES = "properties"; - private static final String ADDITIONAL_PROPERTIES = "additionalProperties"; - private static final String TYPE = "type"; - private static final String FORMAT = "format"; - private static final String CONTENT_ENCODING = "contentEncoding"; - private static final String ITEMS = "items"; - private static final String ADDITIONAL_ITEMS = "additionalItems"; - private static final String REF = "$ref"; - private static final String ALL_OF = "allOf"; - private static final String ANY_OF = "anyOf"; - private static final String NOT = "not"; - private static final String ONE_OF = "oneOf"; - - // from https://tools.ietf.org/html/draft-zyp-json-schema-03#page-14 - private static final String DISALLOW = "disallow"; - private static final String EXTENDS = "extends"; - - private static final String TYPE_NULL = "null"; - private static final String TYPE_BOOLEAN = "boolean"; - private static final String TYPE_OBJECT = "object"; - private static final String TYPE_ARRAY = "array"; - private static final String TYPE_NUMBER = "number"; - private static final String TYPE_INTEGER = "integer"; - private static final String TYPE_STRING = "string"; - - private static final String FORMAT_DATE = "date"; - private static final String FORMAT_TIME = "time"; - private static final String FORMAT_DATE_TIME = "date-time"; - - private static final String CONTENT_ENCODING_BASE64 = "base64"; - - /** - * Converts a JSON schema into Flink's type information. Throws an exception if the schema - * cannot converted because of loss of precision or too flexible schema. - * - *

The converter can resolve simple schema references to solve those cases where entities - * are defined at the beginning and then used throughout a document. - */ - @SuppressWarnings("unchecked") - public static TypeInformation convert(String jsonSchema) { - Preconditions.checkNotNull(jsonSchema, "JSON schema"); - final ObjectMapper mapper = new ObjectMapper(); - mapper.getFactory() - .enable(JsonParser.Feature.ALLOW_COMMENTS) - .enable(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES) - .enable(JsonParser.Feature.ALLOW_SINGLE_QUOTES); - final JsonNode node; - try { - node = mapper.readTree(jsonSchema); - } catch (IOException e) { - throw new IllegalArgumentException( - "Invalid JSON schema.", e); - } - return (TypeInformation) convertType("", node, node); - } - - private static TypeInformation convertType(String location, JsonNode node, JsonNode root) { - // we use a set here to unify types (e.g. types that just add metadata such as 'multipleOf') - final Set> typeSet = new HashSet<>(); - - // search for ref - final Optional ref; - if (node.has(REF) && node.get(REF).isTextual()) { - // try a simple ref resolver to solve those cases where entities are defined at - // the beginning and then used throughout a document - ref = Optional.of(resolveReference(node.get(REF).asText(), node, root)); - } else { - ref = Optional.empty(); - } - - // use TYPE of this node - if (node.has(TYPE)) { - final JsonNode typeNode = node.get(TYPE); - - List types = new ArrayList<>(); - // array of types - if (typeNode.isArray()) { - final Iterator elements = typeNode.elements(); - while (elements.hasNext()) { - types.add(elements.next().asText()); - } - } - // single type - else if (typeNode.isTextual()) { - types.add(typeNode.asText()); - } - - for (String type : types) { - // set field type - switch (type) { - case TYPE_NULL: - typeSet.add(Types.VOID); - break; - case TYPE_BOOLEAN: - typeSet.add(Types.BOOLEAN); - break; - case TYPE_STRING: - if (node.has(FORMAT)) { - typeSet.add(convertStringFormat(location, node.get(FORMAT))); - } else if (node.has(CONTENT_ENCODING)) { - typeSet.add(convertStringEncoding(location, node.get(CONTENT_ENCODING))); - } else { - typeSet.add(Types.STRING); - } - break; - case TYPE_NUMBER: - typeSet.add(Types.BIG_DEC); - break; - case TYPE_INTEGER: - // use BigDecimal for easier interoperability - // without affecting the correctness of the result - typeSet.add(Types.BIG_DEC); - break; - case TYPE_OBJECT: - typeSet.add(convertObject(location, node, root)); - break; - case TYPE_ARRAY: - typeSet.add(convertArray(location, node, root)); - break; - default: - throw new IllegalArgumentException( - "Unsupported type '" + node.get(TYPE).asText() + "' in node: " + location); - } - } - } - // use TYPE of reference as fallback if present - else { - ref.filter(r -> r.has(TYPE)).ifPresent(r -> typeSet.add(convertType(node.get(REF).asText(), r, root))); - } - - // simple interpretation of ONE_OF for supporting "object or null" - if (node.has(ONE_OF) && node.get(ONE_OF).isArray()) { - final TypeInformation[] types = convertTypes(location + '/' + ONE_OF, node.get(ONE_OF), root); - typeSet.addAll(Arrays.asList(types)); - } - // use ONE_OF of reference as fallback - else if (ref.isPresent() && ref.get().has(ONE_OF) && ref.get().get(ONE_OF).isArray()) { - final TypeInformation[] types = convertTypes(node.get(REF).asText() + '/' + ONE_OF, ref.get().get(ONE_OF), root); - typeSet.addAll(Arrays.asList(types)); - } - - // validate no union types or extending - if (node.has(ALL_OF) || node.has(ANY_OF) || node.has(NOT) || node.has(EXTENDS) || node.has(DISALLOW)) { - throw new IllegalArgumentException( - "Union types are such as '" + ALL_OF + "', '" + ANY_OF + "' etc. " + - "and extending are not supported yet."); - } - - // only a type (with null) is supported yet - final List> types = new ArrayList<>(typeSet); - if (types.size() == 0) { - throw new IllegalArgumentException("No type could be found in node:" + location); - } else if (types.size() > 2 || (types.size() == 2 && !types.contains(Types.VOID))) { - throw new IllegalArgumentException( - "Union types with more than just a null type are not supported yet."); - } - - // return the first non-void type or void - if (types.size() == 2 && types.get(0) == Types.VOID) { - return types.get(1); - } else { - return types.get(0); - } - } - - private static TypeInformation convertObject(String location, JsonNode node, JsonNode root) { - // validate properties - if (!node.has(PROPERTIES)) { - return Types.ROW(); - } - if (!node.isObject()) { - throw new IllegalArgumentException( - "Invalid '" + PROPERTIES + "' property for object type in node: " + location); - } - final JsonNode props = node.get(PROPERTIES); - final String[] names = new String[props.size()]; - final TypeInformation[] types = new TypeInformation[props.size()]; - - final Iterator> fieldIter = props.fields(); - int i = 0; - while (fieldIter.hasNext()) { - final Map.Entry subNode = fieldIter.next(); - - // set field name - names[i] = subNode.getKey(); - - // set type - types[i] = convertType(location + '/' + subNode.getKey(), subNode.getValue(), root); - - i++; - } - - // validate that object does not contain additional properties - if (node.has(ADDITIONAL_PROPERTIES) && node.get(ADDITIONAL_PROPERTIES).isBoolean() && - node.get(ADDITIONAL_PROPERTIES).asBoolean()) { - throw new IllegalArgumentException( - "An object must not allow additional properties in node: " + location); - } - - return Types.ROW_NAMED(names, types); - } - - private static TypeInformation convertArray(String location, JsonNode node, JsonNode root) { - // validate items - if (!node.has(ITEMS)) { - throw new IllegalArgumentException( - "Arrays must specify an '" + ITEMS + "' property in node: " + location); - } - final JsonNode items = node.get(ITEMS); - - // list (translated to object array) - if (items.isObject()) { - final TypeInformation elementType = convertType( - location + '/' + ITEMS, - items, - root); - // result type might either be ObjectArrayTypeInfo or BasicArrayTypeInfo for Strings - return Types.OBJECT_ARRAY(elementType); - } - // tuple (translated to row) - else if (items.isArray()) { - final TypeInformation[] types = convertTypes(location + '/' + ITEMS, items, root); - - // validate that array does not contain additional items - if (node.has(ADDITIONAL_ITEMS) && node.get(ADDITIONAL_ITEMS).isBoolean() && - node.get(ADDITIONAL_ITEMS).asBoolean()) { - throw new IllegalArgumentException( - "An array tuple must not allow additional items in node: " + location); - } - - return Types.ROW(types); - } - throw new IllegalArgumentException( - "Invalid type for '" + ITEMS + "' property in node: " + location); - } - - private static TypeInformation convertStringFormat(String location, JsonNode node) { - if (!node.isTextual()) { - throw new IllegalArgumentException("Invalid '" + FORMAT + "' property in node: " + location); - } - - switch (node.asText()) { - case FORMAT_DATE: - return Types.SQL_DATE; - case FORMAT_TIME: - return Types.SQL_TIME; - case FORMAT_DATE_TIME: - return Types.SQL_TIMESTAMP; - default: - return Types.STRING; // unlikely that we will support other formats in the future - } - } - - private static TypeInformation convertStringEncoding(String location, JsonNode node) { - if (!node.isTextual()) { - throw new IllegalArgumentException("Invalid '" + CONTENT_ENCODING + "' property in node: " + location); - } - - // "If the instance value is a string, this property defines that the string SHOULD - // be interpreted as binary data and decoded using the encoding named by this property." - - switch (node.asText()) { - case CONTENT_ENCODING_BASE64: - return Types.PRIMITIVE_ARRAY(Types.BYTE); - default: - // we fail hard here: - // this gives us the chance to support more encodings in the future without problems - // of backwards compatibility - throw new IllegalArgumentException("Invalid encoding '" + node.asText() + "' in node: " + location); - } - } - - private static JsonNode resolveReference(String ref, JsonNode origin, JsonNode root) { - if (!ref.startsWith("#")) { - throw new IllegalArgumentException("Only JSON schemes with simple references " + - "(one indirection in the same document) are supported yet. But was: " + ref); - } - final String path = ref.substring(1); - final JsonNode foundNode = root.at(path); - if (foundNode.isMissingNode()) { - throw new IllegalArgumentException("Could not find reference: " + ref); - } - // prevent obvious cyclic references - if (foundNode == origin) { - throw new IllegalArgumentException("Cyclic references are not supported:" + ref); - } - return foundNode; - } - - private static TypeInformation[] convertTypes(String location, JsonNode arrayNode, JsonNode root) { - final TypeInformation[] types = new TypeInformation[arrayNode.size()]; - final Iterator elements = arrayNode.elements(); - int i = 0; - while (elements.hasNext()) { - final TypeInformation elementType = convertType( - location + '[' + i + ']', - elements.next(), - root); - types[i] = elementType; - i += 1; - } - return types; - } + private JsonRowSchemaConverter() { + // private + } + + // see https://spacetelescope.github.io/understanding-json-schema/UnderstandingJSONSchema.pdf + private static final String PROPERTIES = "properties"; + private static final String ADDITIONAL_PROPERTIES = "additionalProperties"; + private static final String TYPE = "type"; + private static final String FORMAT = "format"; + private static final String CONTENT_ENCODING = "contentEncoding"; + private static final String ITEMS = "items"; + private static final String ADDITIONAL_ITEMS = "additionalItems"; + private static final String REF = "$ref"; + private static final String ALL_OF = "allOf"; + private static final String ANY_OF = "anyOf"; + private static final String NOT = "not"; + private static final String ONE_OF = "oneOf"; + + // from https://tools.ietf.org/html/draft-zyp-json-schema-03#page-14 + private static final String DISALLOW = "disallow"; + private static final String EXTENDS = "extends"; + + private static final String TYPE_NULL = "null"; + private static final String TYPE_BOOLEAN = "boolean"; + private static final String TYPE_OBJECT = "object"; + private static final String TYPE_ARRAY = "array"; + private static final String TYPE_NUMBER = "number"; + private static final String TYPE_INTEGER = "integer"; + private static final String TYPE_STRING = "string"; + + private static final String FORMAT_DATE = "date"; + private static final String FORMAT_TIME = "time"; + private static final String FORMAT_DATE_TIME = "date-time"; + + private static final String CONTENT_ENCODING_BASE64 = "base64"; + + /** + * Converts a JSON schema into Flink's type information. Throws an exception if the schema + * cannot converted because of loss of precision or too flexible schema. + * + *

The converter can resolve simple schema references to solve those cases where entities are + * defined at the beginning and then used throughout a document. + */ + @SuppressWarnings("unchecked") + public static TypeInformation convert(String jsonSchema) { + Preconditions.checkNotNull(jsonSchema, "JSON schema"); + final ObjectMapper mapper = new ObjectMapper(); + mapper.getFactory() + .enable(JsonParser.Feature.ALLOW_COMMENTS) + .enable(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES) + .enable(JsonParser.Feature.ALLOW_SINGLE_QUOTES); + final JsonNode node; + try { + node = mapper.readTree(jsonSchema); + } catch (IOException e) { + throw new IllegalArgumentException("Invalid JSON schema.", e); + } + return (TypeInformation) convertType("", node, node); + } + + private static TypeInformation convertType(String location, JsonNode node, JsonNode root) { + // we use a set here to unify types (e.g. types that just add metadata such as 'multipleOf') + final Set> typeSet = new HashSet<>(); + + // search for ref + final Optional ref; + if (node.has(REF) && node.get(REF).isTextual()) { + // try a simple ref resolver to solve those cases where entities are defined at + // the beginning and then used throughout a document + ref = Optional.of(resolveReference(node.get(REF).asText(), node, root)); + } else { + ref = Optional.empty(); + } + + // use TYPE of this node + if (node.has(TYPE)) { + final JsonNode typeNode = node.get(TYPE); + + List types = new ArrayList<>(); + // array of types + if (typeNode.isArray()) { + final Iterator elements = typeNode.elements(); + while (elements.hasNext()) { + types.add(elements.next().asText()); + } + } + // single type + else if (typeNode.isTextual()) { + types.add(typeNode.asText()); + } + + for (String type : types) { + // set field type + switch (type) { + case TYPE_NULL: + typeSet.add(Types.VOID); + break; + case TYPE_BOOLEAN: + typeSet.add(Types.BOOLEAN); + break; + case TYPE_STRING: + if (node.has(FORMAT)) { + typeSet.add(convertStringFormat(location, node.get(FORMAT))); + } else if (node.has(CONTENT_ENCODING)) { + typeSet.add( + convertStringEncoding(location, node.get(CONTENT_ENCODING))); + } else { + typeSet.add(Types.STRING); + } + break; + case TYPE_NUMBER: + typeSet.add(Types.BIG_DEC); + break; + case TYPE_INTEGER: + // use BigDecimal for easier interoperability + // without affecting the correctness of the result + typeSet.add(Types.BIG_DEC); + break; + case TYPE_OBJECT: + typeSet.add(convertObject(location, node, root)); + break; + case TYPE_ARRAY: + typeSet.add(convertArray(location, node, root)); + break; + default: + throw new IllegalArgumentException( + "Unsupported type '" + + node.get(TYPE).asText() + + "' in node: " + + location); + } + } + } + // use TYPE of reference as fallback if present + else { + ref.filter(r -> r.has(TYPE)) + .ifPresent(r -> typeSet.add(convertType(node.get(REF).asText(), r, root))); + } + + // simple interpretation of ONE_OF for supporting "object or null" + if (node.has(ONE_OF) && node.get(ONE_OF).isArray()) { + final TypeInformation[] types = + convertTypes(location + '/' + ONE_OF, node.get(ONE_OF), root); + typeSet.addAll(Arrays.asList(types)); + } + // use ONE_OF of reference as fallback + else if (ref.isPresent() && ref.get().has(ONE_OF) && ref.get().get(ONE_OF).isArray()) { + final TypeInformation[] types = + convertTypes( + node.get(REF).asText() + '/' + ONE_OF, ref.get().get(ONE_OF), root); + typeSet.addAll(Arrays.asList(types)); + } + + // validate no union types or extending + if (node.has(ALL_OF) + || node.has(ANY_OF) + || node.has(NOT) + || node.has(EXTENDS) + || node.has(DISALLOW)) { + throw new IllegalArgumentException( + "Union types are such as '" + + ALL_OF + + "', '" + + ANY_OF + + "' etc. " + + "and extending are not supported yet."); + } + + // only a type (with null) is supported yet + final List> types = new ArrayList<>(typeSet); + if (types.size() == 0) { + throw new IllegalArgumentException("No type could be found in node:" + location); + } else if (types.size() > 2 || (types.size() == 2 && !types.contains(Types.VOID))) { + throw new IllegalArgumentException( + "Union types with more than just a null type are not supported yet."); + } + + // return the first non-void type or void + if (types.size() == 2 && types.get(0) == Types.VOID) { + return types.get(1); + } else { + return types.get(0); + } + } + + private static TypeInformation convertObject( + String location, JsonNode node, JsonNode root) { + // validate properties + if (!node.has(PROPERTIES)) { + return Types.ROW(); + } + if (!node.isObject()) { + throw new IllegalArgumentException( + "Invalid '" + PROPERTIES + "' property for object type in node: " + location); + } + final JsonNode props = node.get(PROPERTIES); + final String[] names = new String[props.size()]; + final TypeInformation[] types = new TypeInformation[props.size()]; + + final Iterator> fieldIter = props.fields(); + int i = 0; + while (fieldIter.hasNext()) { + final Map.Entry subNode = fieldIter.next(); + + // set field name + names[i] = subNode.getKey(); + + // set type + types[i] = convertType(location + '/' + subNode.getKey(), subNode.getValue(), root); + + i++; + } + + // validate that object does not contain additional properties + if (node.has(ADDITIONAL_PROPERTIES) + && node.get(ADDITIONAL_PROPERTIES).isBoolean() + && node.get(ADDITIONAL_PROPERTIES).asBoolean()) { + throw new IllegalArgumentException( + "An object must not allow additional properties in node: " + location); + } + + return Types.ROW_NAMED(names, types); + } + + private static TypeInformation convertArray(String location, JsonNode node, JsonNode root) { + // validate items + if (!node.has(ITEMS)) { + throw new IllegalArgumentException( + "Arrays must specify an '" + ITEMS + "' property in node: " + location); + } + final JsonNode items = node.get(ITEMS); + + // list (translated to object array) + if (items.isObject()) { + final TypeInformation elementType = convertType(location + '/' + ITEMS, items, root); + // result type might either be ObjectArrayTypeInfo or BasicArrayTypeInfo for Strings + return Types.OBJECT_ARRAY(elementType); + } + // tuple (translated to row) + else if (items.isArray()) { + final TypeInformation[] types = convertTypes(location + '/' + ITEMS, items, root); + + // validate that array does not contain additional items + if (node.has(ADDITIONAL_ITEMS) + && node.get(ADDITIONAL_ITEMS).isBoolean() + && node.get(ADDITIONAL_ITEMS).asBoolean()) { + throw new IllegalArgumentException( + "An array tuple must not allow additional items in node: " + location); + } + + return Types.ROW(types); + } + throw new IllegalArgumentException( + "Invalid type for '" + ITEMS + "' property in node: " + location); + } + + private static TypeInformation convertStringFormat(String location, JsonNode node) { + if (!node.isTextual()) { + throw new IllegalArgumentException( + "Invalid '" + FORMAT + "' property in node: " + location); + } + + switch (node.asText()) { + case FORMAT_DATE: + return Types.SQL_DATE; + case FORMAT_TIME: + return Types.SQL_TIME; + case FORMAT_DATE_TIME: + return Types.SQL_TIMESTAMP; + default: + return Types.STRING; // unlikely that we will support other formats in the future + } + } + + private static TypeInformation convertStringEncoding(String location, JsonNode node) { + if (!node.isTextual()) { + throw new IllegalArgumentException( + "Invalid '" + CONTENT_ENCODING + "' property in node: " + location); + } + + // "If the instance value is a string, this property defines that the string SHOULD + // be interpreted as binary data and decoded using the encoding named by this property." + + switch (node.asText()) { + case CONTENT_ENCODING_BASE64: + return Types.PRIMITIVE_ARRAY(Types.BYTE); + default: + // we fail hard here: + // this gives us the chance to support more encodings in the future without problems + // of backwards compatibility + throw new IllegalArgumentException( + "Invalid encoding '" + node.asText() + "' in node: " + location); + } + } + + private static JsonNode resolveReference(String ref, JsonNode origin, JsonNode root) { + if (!ref.startsWith("#")) { + throw new IllegalArgumentException( + "Only JSON schemes with simple references " + + "(one indirection in the same document) are supported yet. But was: " + + ref); + } + final String path = ref.substring(1); + final JsonNode foundNode = root.at(path); + if (foundNode.isMissingNode()) { + throw new IllegalArgumentException("Could not find reference: " + ref); + } + // prevent obvious cyclic references + if (foundNode == origin) { + throw new IllegalArgumentException("Cyclic references are not supported:" + ref); + } + return foundNode; + } + + private static TypeInformation[] convertTypes( + String location, JsonNode arrayNode, JsonNode root) { + final TypeInformation[] types = new TypeInformation[arrayNode.size()]; + final Iterator elements = arrayNode.elements(); + int i = 0; + while (elements.hasNext()) { + final TypeInformation elementType = + convertType(location + '[' + i + ']', elements.next(), root); + types[i] = elementType; + i += 1; + } + return types; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index 74418767d..9c50fc66d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -60,344 +60,366 @@ /** * Serialization schema that serializes an object of Flink types into a JSON bytes. * - *

Serializes the input Flink object into a JSON string and - * converts it into byte[]. + *

Serializes the input Flink object into a JSON string and converts it into byte[]. * - *

Result byte[] messages can be deserialized using {@link JsonRowDeserializationSchema}. + *

Result byte[] messages can be deserialized using {@link + * JsonRowDeserializationSchema}. */ @PublicEvolving public class JsonRowSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = -2885556750743978636L; - - /** Type information describing the input type. */ - private final RowTypeInfo typeInfo; - - /** Object mapper that is used to create output JSON objects. */ - private final ObjectMapper mapper = new ObjectMapper(); - - private final SerializationRuntimeConverter runtimeConverter; - - /** Reusable object node. */ - private transient ObjectNode node; - - private JsonRowSerializationSchema(TypeInformation typeInfo) { - Preconditions.checkNotNull(typeInfo, "Type information"); - Preconditions.checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); - this.typeInfo = (RowTypeInfo) typeInfo; - this.runtimeConverter = createConverter(typeInfo); - } - - /** - * Builder for {@link JsonRowSerializationSchema}. - */ - @PublicEvolving - public static class Builder { - - private RowTypeInfo typeInfo; - - private Builder() { - // private constructor - } - - /** - * Creates a JSON serialization schema for the given type information. - * - * @param typeInfo Type information describing the result type. The field names of {@link Row} - * are used to parse the JSON properties. - * - * @deprecated Use {@link JsonRowSerializationSchema#builder()} instead. - */ - @Deprecated - public Builder(TypeInformation typeInfo) { - checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); - this.typeInfo = (RowTypeInfo) typeInfo; - } - - /** - * Creates a JSON serialization schema for the given JSON schema. - * - * @param jsonSchema JSON schema describing the result type - * - * @see http://json-schema.org/ - * - * @deprecated Use {@link JsonRowSerializationSchema#builder()} instead. - */ - @Deprecated - public Builder(String jsonSchema) { - this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema))); - } - - /** - * Sets type information for JSON serialization schema. - * - * @param typeInfo Type information describing the result type. The field names of {@link Row} - * are used to parse the JSON properties. - */ - public Builder withTypeInfo(TypeInformation typeInfo) { - checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); - this.typeInfo = (RowTypeInfo) typeInfo; - return this; - } - - /** - * Finalizes the configuration and checks validity. - * @return Configured {@link JsonRowSerializationSchema} - */ - public JsonRowSerializationSchema build() { - checkArgument(typeInfo != null, "typeInfo should be set."); - return new JsonRowSerializationSchema(typeInfo); - } - } - - /** - * Creates a builder for {@link JsonRowSerializationSchema.Builder}. - */ - public static Builder builder() { - return new Builder(); - } - - @Override - public byte[] serialize(Row row) { - if (node == null) { - node = mapper.createObjectNode(); - } - - try { - runtimeConverter.convert(mapper, node, row); - return mapper.writeValueAsBytes(node); - } catch (Throwable t) { - throw new RuntimeException("Could not serialize row '" + row + "'. " + - "Make sure that the schema matches the input.", t); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - final JsonRowSerializationSchema that = (JsonRowSerializationSchema) o; - return Objects.equals(typeInfo, that.typeInfo); - } - - @Override - public int hashCode() { - return Objects.hash(typeInfo); - } - - /* - Runtime converters - */ - - /** - * Runtime converter that maps between Java objects and corresponding {@link JsonNode}s. - */ - @FunctionalInterface - private interface SerializationRuntimeConverter extends Serializable { - JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object object); - } - - private SerializationRuntimeConverter createConverter(TypeInformation typeInfo) { - SerializationRuntimeConverter baseConverter = createConverterForSimpleType(typeInfo) - .orElseGet(() -> - createContainerConverter(typeInfo) - .orElseGet(this::createFallbackConverter)); - return wrapIntoNullableConverter(baseConverter); - } - - private SerializationRuntimeConverter wrapIntoNullableConverter(SerializationRuntimeConverter converter) { - return (mapper, reuse, object) -> { - if (object == null) { - return mapper.getNodeFactory().nullNode(); - } - - return converter.convert(mapper, reuse, object); - }; - } - - private Optional createContainerConverter(TypeInformation typeInfo) { - if (typeInfo instanceof RowTypeInfo) { - return Optional.of(createRowConverter((RowTypeInfo) typeInfo)); - } else if (typeInfo instanceof ObjectArrayTypeInfo) { - return Optional.of(createObjectArrayConverter(((ObjectArrayTypeInfo) typeInfo).getComponentInfo())); - } else if (typeInfo instanceof BasicArrayTypeInfo) { - return Optional.of(createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo())); - } else if (isPrimitiveByteArray(typeInfo)) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().binaryNode((byte[]) object)); - } else { - return Optional.empty(); - } - } - - private boolean isPrimitiveByteArray(TypeInformation typeInfo) { - return typeInfo instanceof PrimitiveArrayTypeInfo && - ((PrimitiveArrayTypeInfo) typeInfo).getComponentType() == Types.BYTE; - } - - private SerializationRuntimeConverter createObjectArrayConverter(TypeInformation elementTypeInfo) { - SerializationRuntimeConverter elementConverter = createConverter(elementTypeInfo); - return assembleArrayConverter(elementConverter); - } - - private SerializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) { - List fieldConverters = Arrays.stream(typeInfo.getFieldTypes()) - .map(this::createConverter) - .collect(Collectors.toList()); - - return assembleRowConverter(typeInfo.getFieldNames(), fieldConverters); - } - - private SerializationRuntimeConverter createFallbackConverter() { - return (mapper, reuse, object) -> { - // for types that were specified without JSON schema - // e.g. POJOs - try { - return mapper.valueToTree(object); - } catch (IllegalArgumentException e) { - throw new WrappingRuntimeException(format("Could not convert object: %s", object), e); - } - }; - } - - private Optional createConverterForSimpleType(TypeInformation simpleTypeInfo) { - if (simpleTypeInfo == Types.VOID) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().nullNode()); - } else if (simpleTypeInfo == Types.BOOLEAN) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().booleanNode((Boolean) object)); - } else if (simpleTypeInfo == Types.STRING) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().textNode((String) object)); - } else if (simpleTypeInfo == Types.INT) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Integer) object)); - } else if (simpleTypeInfo == Types.LONG) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Long) object)); - } else if (simpleTypeInfo == Types.DOUBLE) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Double) object)); - } else if (simpleTypeInfo == Types.FLOAT) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Float) object)); - } else if (simpleTypeInfo == Types.SHORT) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Short) object)); - } else if (simpleTypeInfo == Types.BYTE) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Byte) object)); - } else if (simpleTypeInfo == Types.BIG_DEC) { - return Optional.of(createBigDecimalConverter()); - } else if (simpleTypeInfo == Types.BIG_INT) { - return Optional.of(createBigIntegerConverter()); - } else if (simpleTypeInfo == Types.SQL_DATE) { - return Optional.of(this::convertDate); - } else if (simpleTypeInfo == Types.SQL_TIME) { - return Optional.of(this::convertTime); - } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) { - return Optional.of(this::convertTimestamp); - } else if (simpleTypeInfo == Types.LOCAL_DATE) { - return Optional.of(this::convertLocalDate); - } else if (simpleTypeInfo == Types.LOCAL_TIME) { - return Optional.of(this::convertLocalTime); - } else if (simpleTypeInfo == Types.LOCAL_DATE_TIME) { - return Optional.of(this::convertLocalDateTime); - } else { - return Optional.empty(); - } - } - - private JsonNode convertLocalDate(ObjectMapper mapper, JsonNode reuse, Object object) { - return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format((LocalDate) object)); - } - - private JsonNode convertDate(ObjectMapper mapper, JsonNode reuse, Object object) { - Date date = (Date) object; - return convertLocalDate(mapper, reuse, date.toLocalDate()); - } - - private JsonNode convertLocalDateTime(ObjectMapper mapper, JsonNode reuse, Object object) { - return mapper.getNodeFactory() - .textNode(RFC3339_TIMESTAMP_FORMAT.format((LocalDateTime) object)); - } - - private JsonNode convertTimestamp(ObjectMapper mapper, JsonNode reuse, Object object) { - Timestamp timestamp = (Timestamp) object; - return convertLocalDateTime(mapper, reuse, timestamp.toLocalDateTime()); - } - - private JsonNode convertLocalTime(ObjectMapper mapper, JsonNode reuse, Object object) { - JsonNodeFactory nodeFactory = mapper.getNodeFactory(); - return nodeFactory.textNode(RFC3339_TIME_FORMAT.format((LocalTime) object)); - } - - private JsonNode convertTime(ObjectMapper mapper, JsonNode reuse, Object object) { - final Time time = (Time) object; - return convertLocalTime(mapper, reuse, time.toLocalTime()); - } - - private SerializationRuntimeConverter createBigDecimalConverter() { - return (mapper, reuse, object) -> { - // convert decimal if necessary - JsonNodeFactory nodeFactory = mapper.getNodeFactory(); - if (object instanceof BigDecimal) { - return nodeFactory.numberNode((BigDecimal) object); - } - return nodeFactory.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); - }; - } - - private SerializationRuntimeConverter createBigIntegerConverter() { - return (mapper, reuse, object) -> { - // convert decimal if necessary - JsonNodeFactory nodeFactory = mapper.getNodeFactory(); - if (object instanceof BigInteger) { - return nodeFactory.numberNode((BigInteger) object); - } - return nodeFactory.numberNode(BigInteger.valueOf(((Number) object).longValue())); - }; - } - - private SerializationRuntimeConverter assembleRowConverter( - String[] fieldNames, - List fieldConverters) { - return (mapper, reuse, object) -> { - ObjectNode node; - - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createObjectNode(); - } else { - node = (ObjectNode) reuse; - } - - Row row = (Row) object; - - for (int i = 0; i < fieldNames.length; i++) { - String fieldName = fieldNames[i]; - node.set(fieldName, - fieldConverters.get(i).convert(mapper, node.get(fieldNames[i]), row.getField(i))); - } - - return node; - }; - } - - private SerializationRuntimeConverter assembleArrayConverter(SerializationRuntimeConverter elementConverter) { - return (mapper, reuse, object) -> { - ArrayNode node; - - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createArrayNode(); - } else { - node = (ArrayNode) reuse; - node.removeAll(); - } - - Object[] array = (Object[]) object; - - for (Object element : array) { - node.add(elementConverter.convert(mapper, null, element)); - } - - return node; - }; - } + private static final long serialVersionUID = -2885556750743978636L; + + /** Type information describing the input type. */ + private final RowTypeInfo typeInfo; + + /** Object mapper that is used to create output JSON objects. */ + private final ObjectMapper mapper = new ObjectMapper(); + + private final SerializationRuntimeConverter runtimeConverter; + + /** Reusable object node. */ + private transient ObjectNode node; + + private JsonRowSerializationSchema(TypeInformation typeInfo) { + Preconditions.checkNotNull(typeInfo, "Type information"); + Preconditions.checkArgument( + typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + this.typeInfo = (RowTypeInfo) typeInfo; + this.runtimeConverter = createConverter(typeInfo); + } + + /** Builder for {@link JsonRowSerializationSchema}. */ + @PublicEvolving + public static class Builder { + + private RowTypeInfo typeInfo; + + private Builder() { + // private constructor + } + + /** + * Creates a JSON serialization schema for the given type information. + * + * @param typeInfo Type information describing the result type. The field names of {@link + * Row} are used to parse the JSON properties. + * @deprecated Use {@link JsonRowSerializationSchema#builder()} instead. + */ + @Deprecated + public Builder(TypeInformation typeInfo) { + checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + this.typeInfo = (RowTypeInfo) typeInfo; + } + + /** + * Creates a JSON serialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * @see http://json-schema.org/ + * @deprecated Use {@link JsonRowSerializationSchema#builder()} instead. + */ + @Deprecated + public Builder(String jsonSchema) { + this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema))); + } + + /** + * Sets type information for JSON serialization schema. + * + * @param typeInfo Type information describing the result type. The field names of {@link + * Row} are used to parse the JSON properties. + */ + public Builder withTypeInfo(TypeInformation typeInfo) { + checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + this.typeInfo = (RowTypeInfo) typeInfo; + return this; + } + + /** + * Finalizes the configuration and checks validity. + * + * @return Configured {@link JsonRowSerializationSchema} + */ + public JsonRowSerializationSchema build() { + checkArgument(typeInfo != null, "typeInfo should be set."); + return new JsonRowSerializationSchema(typeInfo); + } + } + + /** Creates a builder for {@link JsonRowSerializationSchema.Builder}. */ + public static Builder builder() { + return new Builder(); + } + + @Override + public byte[] serialize(Row row) { + if (node == null) { + node = mapper.createObjectNode(); + } + + try { + runtimeConverter.convert(mapper, node, row); + return mapper.writeValueAsBytes(node); + } catch (Throwable t) { + throw new RuntimeException( + "Could not serialize row '" + + row + + "'. " + + "Make sure that the schema matches the input.", + t); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final JsonRowSerializationSchema that = (JsonRowSerializationSchema) o; + return Objects.equals(typeInfo, that.typeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(typeInfo); + } + + /* + Runtime converters + */ + + /** Runtime converter that maps between Java objects and corresponding {@link JsonNode}s. */ + @FunctionalInterface + private interface SerializationRuntimeConverter extends Serializable { + JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object object); + } + + private SerializationRuntimeConverter createConverter(TypeInformation typeInfo) { + SerializationRuntimeConverter baseConverter = + createConverterForSimpleType(typeInfo) + .orElseGet( + () -> + createContainerConverter(typeInfo) + .orElseGet(this::createFallbackConverter)); + return wrapIntoNullableConverter(baseConverter); + } + + private SerializationRuntimeConverter wrapIntoNullableConverter( + SerializationRuntimeConverter converter) { + return (mapper, reuse, object) -> { + if (object == null) { + return mapper.getNodeFactory().nullNode(); + } + + return converter.convert(mapper, reuse, object); + }; + } + + private Optional createContainerConverter( + TypeInformation typeInfo) { + if (typeInfo instanceof RowTypeInfo) { + return Optional.of(createRowConverter((RowTypeInfo) typeInfo)); + } else if (typeInfo instanceof ObjectArrayTypeInfo) { + return Optional.of( + createObjectArrayConverter( + ((ObjectArrayTypeInfo) typeInfo).getComponentInfo())); + } else if (typeInfo instanceof BasicArrayTypeInfo) { + return Optional.of( + createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo())); + } else if (isPrimitiveByteArray(typeInfo)) { + return Optional.of( + (mapper, reuse, object) -> mapper.getNodeFactory().binaryNode((byte[]) object)); + } else { + return Optional.empty(); + } + } + + private boolean isPrimitiveByteArray(TypeInformation typeInfo) { + return typeInfo instanceof PrimitiveArrayTypeInfo + && ((PrimitiveArrayTypeInfo) typeInfo).getComponentType() == Types.BYTE; + } + + private SerializationRuntimeConverter createObjectArrayConverter( + TypeInformation elementTypeInfo) { + SerializationRuntimeConverter elementConverter = createConverter(elementTypeInfo); + return assembleArrayConverter(elementConverter); + } + + private SerializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) { + List fieldConverters = + Arrays.stream(typeInfo.getFieldTypes()) + .map(this::createConverter) + .collect(Collectors.toList()); + + return assembleRowConverter(typeInfo.getFieldNames(), fieldConverters); + } + + private SerializationRuntimeConverter createFallbackConverter() { + return (mapper, reuse, object) -> { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.valueToTree(object); + } catch (IllegalArgumentException e) { + throw new WrappingRuntimeException( + format("Could not convert object: %s", object), e); + } + }; + } + + private Optional createConverterForSimpleType( + TypeInformation simpleTypeInfo) { + if (simpleTypeInfo == Types.VOID) { + return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().nullNode()); + } else if (simpleTypeInfo == Types.BOOLEAN) { + return Optional.of( + (mapper, reuse, object) -> + mapper.getNodeFactory().booleanNode((Boolean) object)); + } else if (simpleTypeInfo == Types.STRING) { + return Optional.of( + (mapper, reuse, object) -> mapper.getNodeFactory().textNode((String) object)); + } else if (simpleTypeInfo == Types.INT) { + return Optional.of( + (mapper, reuse, object) -> + mapper.getNodeFactory().numberNode((Integer) object)); + } else if (simpleTypeInfo == Types.LONG) { + return Optional.of( + (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Long) object)); + } else if (simpleTypeInfo == Types.DOUBLE) { + return Optional.of( + (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Double) object)); + } else if (simpleTypeInfo == Types.FLOAT) { + return Optional.of( + (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Float) object)); + } else if (simpleTypeInfo == Types.SHORT) { + return Optional.of( + (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Short) object)); + } else if (simpleTypeInfo == Types.BYTE) { + return Optional.of( + (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Byte) object)); + } else if (simpleTypeInfo == Types.BIG_DEC) { + return Optional.of(createBigDecimalConverter()); + } else if (simpleTypeInfo == Types.BIG_INT) { + return Optional.of(createBigIntegerConverter()); + } else if (simpleTypeInfo == Types.SQL_DATE) { + return Optional.of(this::convertDate); + } else if (simpleTypeInfo == Types.SQL_TIME) { + return Optional.of(this::convertTime); + } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) { + return Optional.of(this::convertTimestamp); + } else if (simpleTypeInfo == Types.LOCAL_DATE) { + return Optional.of(this::convertLocalDate); + } else if (simpleTypeInfo == Types.LOCAL_TIME) { + return Optional.of(this::convertLocalTime); + } else if (simpleTypeInfo == Types.LOCAL_DATE_TIME) { + return Optional.of(this::convertLocalDateTime); + } else { + return Optional.empty(); + } + } + + private JsonNode convertLocalDate(ObjectMapper mapper, JsonNode reuse, Object object) { + return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format((LocalDate) object)); + } + + private JsonNode convertDate(ObjectMapper mapper, JsonNode reuse, Object object) { + Date date = (Date) object; + return convertLocalDate(mapper, reuse, date.toLocalDate()); + } + + private JsonNode convertLocalDateTime(ObjectMapper mapper, JsonNode reuse, Object object) { + return mapper.getNodeFactory() + .textNode(RFC3339_TIMESTAMP_FORMAT.format((LocalDateTime) object)); + } + + private JsonNode convertTimestamp(ObjectMapper mapper, JsonNode reuse, Object object) { + Timestamp timestamp = (Timestamp) object; + return convertLocalDateTime(mapper, reuse, timestamp.toLocalDateTime()); + } + + private JsonNode convertLocalTime(ObjectMapper mapper, JsonNode reuse, Object object) { + JsonNodeFactory nodeFactory = mapper.getNodeFactory(); + return nodeFactory.textNode(RFC3339_TIME_FORMAT.format((LocalTime) object)); + } + + private JsonNode convertTime(ObjectMapper mapper, JsonNode reuse, Object object) { + final Time time = (Time) object; + return convertLocalTime(mapper, reuse, time.toLocalTime()); + } + + private SerializationRuntimeConverter createBigDecimalConverter() { + return (mapper, reuse, object) -> { + // convert decimal if necessary + JsonNodeFactory nodeFactory = mapper.getNodeFactory(); + if (object instanceof BigDecimal) { + return nodeFactory.numberNode((BigDecimal) object); + } + return nodeFactory.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); + }; + } + + private SerializationRuntimeConverter createBigIntegerConverter() { + return (mapper, reuse, object) -> { + // convert decimal if necessary + JsonNodeFactory nodeFactory = mapper.getNodeFactory(); + if (object instanceof BigInteger) { + return nodeFactory.numberNode((BigInteger) object); + } + return nodeFactory.numberNode(BigInteger.valueOf(((Number) object).longValue())); + }; + } + + private SerializationRuntimeConverter assembleRowConverter( + String[] fieldNames, List fieldConverters) { + return (mapper, reuse, object) -> { + ObjectNode node; + + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createObjectNode(); + } else { + node = (ObjectNode) reuse; + } + + Row row = (Row) object; + + for (int i = 0; i < fieldNames.length; i++) { + String fieldName = fieldNames[i]; + node.set( + fieldName, + fieldConverters + .get(i) + .convert(mapper, node.get(fieldNames[i]), row.getField(i))); + } + + return node; + }; + } + + private SerializationRuntimeConverter assembleArrayConverter( + SerializationRuntimeConverter elementConverter) { + return (mapper, reuse, object) -> { + ArrayNode node; + + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createArrayNode(); + } else { + node = (ArrayNode) reuse; + node.removeAll(); + } + + Object[] array = (Object[]) object; + + for (Object element : array) { + node.add(elementConverter.convert(mapper, null, element)); + } + + return node; + }; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java index 65feec353..b5d7ec825 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java @@ -64,337 +64,334 @@ import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; -/** Tool class used to convert from {@link JsonNode} to {@link RowData}. **/ +/** Tool class used to convert from {@link JsonNode} to {@link RowData}. * */ @Internal public class JsonToRowDataConverters implements Serializable { - private static final long serialVersionUID = 1L; - - /** - * Flag indicating whether to fail if a field is missing. - */ - private final boolean failOnMissingField; - - /** - * Flag indicating whether to ignore invalid fields/rows (default: throw an exception). - */ - private final boolean ignoreParseErrors; - - /** - * Timestamp format specification which is used to parse timestamp. - */ - private final TimestampFormat timestampFormat; - - public JsonToRowDataConverters( - boolean failOnMissingField, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - this.failOnMissingField = failOnMissingField; - this.ignoreParseErrors = ignoreParseErrors; - this.timestampFormat = timestampFormat; - } - - /** - * Runtime converter that converts {@link JsonNode}s into objects of Flink Table & SQL - * internal data structures. - */ - @FunctionalInterface - public interface JsonToRowDataConverter extends Serializable { - Object convert(JsonNode jsonNode); - } - - /** - * Creates a runtime converter which is null safe. - */ - public JsonToRowDataConverter createConverter(LogicalType type) { - return wrapIntoNullableConverter(createNotNullConverter(type)); - } - - /** - * Creates a runtime converter which assuming input object is not null. - */ - private JsonToRowDataConverter createNotNullConverter(LogicalType type) { - switch (type.getTypeRoot()) { - case NULL: - return jsonNode -> null; - case BOOLEAN: - return this::convertToBoolean; - case TINYINT: - return jsonNode -> Byte.parseByte(jsonNode.asText().trim()); - case SMALLINT: - return jsonNode -> Short.parseShort(jsonNode.asText().trim()); - case INTEGER: - case INTERVAL_YEAR_MONTH: - return this::convertToInt; - case BIGINT: - case INTERVAL_DAY_TIME: - return this::convertToLong; - case DATE: - return this::convertToDate; - case TIME_WITHOUT_TIME_ZONE: - return this::convertToTime; - case TIMESTAMP_WITHOUT_TIME_ZONE: - return this::convertToTimestamp; - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return this::convertToTimestampWithLocalZone; - case FLOAT: - return this::convertToFloat; - case DOUBLE: - return this::convertToDouble; - case CHAR: - case VARCHAR: - return this::convertToString; - case BINARY: - case VARBINARY: - return this::convertToBytes; - case DECIMAL: - return createDecimalConverter((DecimalType) type); - case ARRAY: - return createArrayConverter((ArrayType) type); - case MAP: - MapType mapType = (MapType) type; - return createMapConverter( - mapType.asSummaryString(), mapType.getKeyType(), mapType.getValueType()); - case MULTISET: - MultisetType multisetType = (MultisetType) type; - return createMapConverter( - multisetType.asSummaryString(), multisetType.getElementType(), new IntType()); - case ROW: - return createRowConverter((RowType) type); - case RAW: - default: - throw new UnsupportedOperationException("Unsupported type: " + type); - } - } - - private boolean convertToBoolean(JsonNode jsonNode) { - if (jsonNode.isBoolean()) { - // avoid redundant toString and parseBoolean, for better performance - return jsonNode.asBoolean(); - } else { - return Boolean.parseBoolean(jsonNode.asText().trim()); - } - } - - private int convertToInt(JsonNode jsonNode) { - if (jsonNode.canConvertToInt()) { - // avoid redundant toString and parseInt, for better performance - return jsonNode.asInt(); - } else { - return Integer.parseInt(jsonNode.asText().trim()); - } - } - - private long convertToLong(JsonNode jsonNode) { - if (jsonNode.canConvertToLong()) { - // avoid redundant toString and parseLong, for better performance - return jsonNode.asLong(); - } else { - return Long.parseLong(jsonNode.asText().trim()); - } - } - - private double convertToDouble(JsonNode jsonNode) { - if (jsonNode.isDouble()) { - // avoid redundant toString and parseDouble, for better performance - return jsonNode.asDouble(); - } else { - return Double.parseDouble(jsonNode.asText().trim()); - } - } - - private float convertToFloat(JsonNode jsonNode) { - if (jsonNode.isDouble()) { - // avoid redundant toString and parseDouble, for better performance - return (float) jsonNode.asDouble(); - } else { - return Float.parseFloat(jsonNode.asText().trim()); - } - } - - private int convertToDate(JsonNode jsonNode) { - LocalDate date = ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); - return (int) date.toEpochDay(); - } - - private int convertToTime(JsonNode jsonNode) { - TemporalAccessor parsedTime = SQL_TIME_FORMAT.parse(jsonNode.asText()); - LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); - - // get number of milliseconds of the day - return localTime.toSecondOfDay() * 1000; - } - - private TimestampData convertToTimestamp(JsonNode jsonNode) { - TemporalAccessor parsedTimestamp; - switch (timestampFormat) { - case SQL: - parsedTimestamp = SQL_TIMESTAMP_FORMAT.parse(jsonNode.asText()); - break; - case ISO_8601: - parsedTimestamp = ISO8601_TIMESTAMP_FORMAT.parse(jsonNode.asText()); - break; - default: - throw new TableException(String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); - } - LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); - LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); - - return TimestampData.fromLocalDateTime(LocalDateTime.of(localDate, localTime)); - } - - private TimestampData convertToTimestampWithLocalZone(JsonNode jsonNode){ - TemporalAccessor parsedTimestampWithLocalZone; - switch (timestampFormat){ - case SQL: - parsedTimestampWithLocalZone = SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); - break; - case ISO_8601: - parsedTimestampWithLocalZone = ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); - break; - default: - throw new TableException(String.format("Unsupported timestamp format '%s'. Validator should have checked that.", timestampFormat)); - } - LocalTime localTime = parsedTimestampWithLocalZone.query(TemporalQueries.localTime()); - LocalDate localDate = parsedTimestampWithLocalZone.query(TemporalQueries.localDate()); - - return TimestampData.fromInstant(LocalDateTime.of(localDate, localTime).toInstant(ZoneOffset.UTC)); - } - - private StringData convertToString(JsonNode jsonNode) { - if (jsonNode.isContainerNode()) { - return StringData.fromString(jsonNode.toString()); - } else { - return StringData.fromString(jsonNode.asText()); - } - } - - private byte[] convertToBytes(JsonNode jsonNode) { - try { - return jsonNode.binaryValue(); - } catch (IOException e) { - throw new JsonParseException("Unable to deserialize byte array.", e); - } - } - - private JsonToRowDataConverter createDecimalConverter(DecimalType decimalType) { - final int precision = decimalType.getPrecision(); - final int scale = decimalType.getScale(); - return jsonNode -> { - BigDecimal bigDecimal; - if (jsonNode.isBigDecimal()) { - bigDecimal = jsonNode.decimalValue(); - } else { - bigDecimal = new BigDecimal(jsonNode.asText()); - } - return DecimalData.fromBigDecimal(bigDecimal, precision, scale); - }; - } - - private JsonToRowDataConverter createArrayConverter(ArrayType arrayType) { - JsonToRowDataConverter elementConverter = createConverter(arrayType.getElementType()); - final Class elementClass = LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType()); - return jsonNode -> { - final ArrayNode node = (ArrayNode) jsonNode; - final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); - for (int i = 0; i < node.size(); i++) { - final JsonNode innerNode = node.get(i); - array[i] = elementConverter.convert(innerNode); - } - return new GenericArrayData(array); - }; - } - - private JsonToRowDataConverter createMapConverter( - String typeSummary, LogicalType keyType, LogicalType valueType) { - if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { - throw new UnsupportedOperationException( - "JSON format doesn't support non-string as key type of map. " + - "The type is: " + typeSummary); - } - final JsonToRowDataConverter keyConverter = createConverter(keyType); - final JsonToRowDataConverter valueConverter = createConverter(valueType); - - return jsonNode -> { - Iterator> fields = jsonNode.fields(); - Map result = new HashMap<>(); - while (fields.hasNext()) { - Map.Entry entry = fields.next(); - Object key = keyConverter.convert(TextNode.valueOf(entry.getKey())); - Object value = valueConverter.convert(entry.getValue()); - result.put(key, value); - } - return new GenericMapData(result); - }; - } - - public JsonToRowDataConverter createRowConverter(RowType rowType) { - final JsonToRowDataConverter[] fieldConverters = rowType.getFields().stream() - .map(RowType.RowField::getType) - .map(this::createConverter) - .toArray(JsonToRowDataConverter[]::new); - final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]); - - return jsonNode -> { - ObjectNode node = (ObjectNode) jsonNode; - int arity = fieldNames.length; - GenericRowData row = new GenericRowData(arity); - for (int i = 0; i < arity; i++) { - String fieldName = fieldNames[i]; - JsonNode field = node.get(fieldName); - Object convertedField = convertField(fieldConverters[i], fieldName, field); - row.setField(i, convertedField); - } - return row; - }; - } - - private Object convertField( - JsonToRowDataConverter fieldConverter, - String fieldName, - JsonNode field) { - if (field == null) { - if (failOnMissingField) { - throw new JsonParseException( - "Could not find field with name '" + fieldName + "'."); - } else { - return null; - } - } else { - return fieldConverter.convert(field); - } - } - - private JsonToRowDataConverter wrapIntoNullableConverter( - JsonToRowDataConverter converter) { - return jsonNode -> { - if (jsonNode == null || jsonNode.isNull() || jsonNode.isMissingNode()) { - return null; - } - try { - return converter.convert(jsonNode); - } catch (Throwable t) { - if (!ignoreParseErrors) { - throw t; - } - return null; - } - }; - } - - /** - * Exception which refers to parse errors in converters. - * */ - private static final class JsonParseException extends RuntimeException { - private static final long serialVersionUID = 1L; - - public JsonParseException(String message) { - super(message); - } - - public JsonParseException(String message, Throwable cause) { - super(message, cause); - } - } - + private static final long serialVersionUID = 1L; + + /** Flag indicating whether to fail if a field is missing. */ + private final boolean failOnMissingField; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + /** Timestamp format specification which is used to parse timestamp. */ + private final TimestampFormat timestampFormat; + + public JsonToRowDataConverters( + boolean failOnMissingField, + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { + this.failOnMissingField = failOnMissingField; + this.ignoreParseErrors = ignoreParseErrors; + this.timestampFormat = timestampFormat; + } + + /** + * Runtime converter that converts {@link JsonNode}s into objects of Flink Table & SQL internal + * data structures. + */ + @FunctionalInterface + public interface JsonToRowDataConverter extends Serializable { + Object convert(JsonNode jsonNode); + } + + /** Creates a runtime converter which is null safe. */ + public JsonToRowDataConverter createConverter(LogicalType type) { + return wrapIntoNullableConverter(createNotNullConverter(type)); + } + + /** Creates a runtime converter which assuming input object is not null. */ + private JsonToRowDataConverter createNotNullConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return jsonNode -> null; + case BOOLEAN: + return this::convertToBoolean; + case TINYINT: + return jsonNode -> Byte.parseByte(jsonNode.asText().trim()); + case SMALLINT: + return jsonNode -> Short.parseShort(jsonNode.asText().trim()); + case INTEGER: + case INTERVAL_YEAR_MONTH: + return this::convertToInt; + case BIGINT: + case INTERVAL_DAY_TIME: + return this::convertToLong; + case DATE: + return this::convertToDate; + case TIME_WITHOUT_TIME_ZONE: + return this::convertToTime; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return this::convertToTimestamp; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return this::convertToTimestampWithLocalZone; + case FLOAT: + return this::convertToFloat; + case DOUBLE: + return this::convertToDouble; + case CHAR: + case VARCHAR: + return this::convertToString; + case BINARY: + case VARBINARY: + return this::convertToBytes; + case DECIMAL: + return createDecimalConverter((DecimalType) type); + case ARRAY: + return createArrayConverter((ArrayType) type); + case MAP: + MapType mapType = (MapType) type; + return createMapConverter( + mapType.asSummaryString(), mapType.getKeyType(), mapType.getValueType()); + case MULTISET: + MultisetType multisetType = (MultisetType) type; + return createMapConverter( + multisetType.asSummaryString(), + multisetType.getElementType(), + new IntType()); + case ROW: + return createRowConverter((RowType) type); + case RAW: + default: + throw new UnsupportedOperationException("Unsupported type: " + type); + } + } + + private boolean convertToBoolean(JsonNode jsonNode) { + if (jsonNode.isBoolean()) { + // avoid redundant toString and parseBoolean, for better performance + return jsonNode.asBoolean(); + } else { + return Boolean.parseBoolean(jsonNode.asText().trim()); + } + } + + private int convertToInt(JsonNode jsonNode) { + if (jsonNode.canConvertToInt()) { + // avoid redundant toString and parseInt, for better performance + return jsonNode.asInt(); + } else { + return Integer.parseInt(jsonNode.asText().trim()); + } + } + + private long convertToLong(JsonNode jsonNode) { + if (jsonNode.canConvertToLong()) { + // avoid redundant toString and parseLong, for better performance + return jsonNode.asLong(); + } else { + return Long.parseLong(jsonNode.asText().trim()); + } + } + + private double convertToDouble(JsonNode jsonNode) { + if (jsonNode.isDouble()) { + // avoid redundant toString and parseDouble, for better performance + return jsonNode.asDouble(); + } else { + return Double.parseDouble(jsonNode.asText().trim()); + } + } + + private float convertToFloat(JsonNode jsonNode) { + if (jsonNode.isDouble()) { + // avoid redundant toString and parseDouble, for better performance + return (float) jsonNode.asDouble(); + } else { + return Float.parseFloat(jsonNode.asText().trim()); + } + } + + private int convertToDate(JsonNode jsonNode) { + LocalDate date = ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); + return (int) date.toEpochDay(); + } + + private int convertToTime(JsonNode jsonNode) { + TemporalAccessor parsedTime = SQL_TIME_FORMAT.parse(jsonNode.asText()); + LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); + + // get number of milliseconds of the day + return localTime.toSecondOfDay() * 1000; + } + + private TimestampData convertToTimestamp(JsonNode jsonNode) { + TemporalAccessor parsedTimestamp; + switch (timestampFormat) { + case SQL: + parsedTimestamp = SQL_TIMESTAMP_FORMAT.parse(jsonNode.asText()); + break; + case ISO_8601: + parsedTimestamp = ISO8601_TIMESTAMP_FORMAT.parse(jsonNode.asText()); + break; + default: + throw new TableException( + String.format( + "Unsupported timestamp format '%s'. Validator should have checked that.", + timestampFormat)); + } + LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); + LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); + + return TimestampData.fromLocalDateTime(LocalDateTime.of(localDate, localTime)); + } + + private TimestampData convertToTimestampWithLocalZone(JsonNode jsonNode) { + TemporalAccessor parsedTimestampWithLocalZone; + switch (timestampFormat) { + case SQL: + parsedTimestampWithLocalZone = + SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); + break; + case ISO_8601: + parsedTimestampWithLocalZone = + ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); + break; + default: + throw new TableException( + String.format( + "Unsupported timestamp format '%s'. Validator should have checked that.", + timestampFormat)); + } + LocalTime localTime = parsedTimestampWithLocalZone.query(TemporalQueries.localTime()); + LocalDate localDate = parsedTimestampWithLocalZone.query(TemporalQueries.localDate()); + + return TimestampData.fromInstant( + LocalDateTime.of(localDate, localTime).toInstant(ZoneOffset.UTC)); + } + + private StringData convertToString(JsonNode jsonNode) { + if (jsonNode.isContainerNode()) { + return StringData.fromString(jsonNode.toString()); + } else { + return StringData.fromString(jsonNode.asText()); + } + } + + private byte[] convertToBytes(JsonNode jsonNode) { + try { + return jsonNode.binaryValue(); + } catch (IOException e) { + throw new JsonParseException("Unable to deserialize byte array.", e); + } + } + + private JsonToRowDataConverter createDecimalConverter(DecimalType decimalType) { + final int precision = decimalType.getPrecision(); + final int scale = decimalType.getScale(); + return jsonNode -> { + BigDecimal bigDecimal; + if (jsonNode.isBigDecimal()) { + bigDecimal = jsonNode.decimalValue(); + } else { + bigDecimal = new BigDecimal(jsonNode.asText()); + } + return DecimalData.fromBigDecimal(bigDecimal, precision, scale); + }; + } + + private JsonToRowDataConverter createArrayConverter(ArrayType arrayType) { + JsonToRowDataConverter elementConverter = createConverter(arrayType.getElementType()); + final Class elementClass = + LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType()); + return jsonNode -> { + final ArrayNode node = (ArrayNode) jsonNode; + final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); + for (int i = 0; i < node.size(); i++) { + final JsonNode innerNode = node.get(i); + array[i] = elementConverter.convert(innerNode); + } + return new GenericArrayData(array); + }; + } + + private JsonToRowDataConverter createMapConverter( + String typeSummary, LogicalType keyType, LogicalType valueType) { + if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { + throw new UnsupportedOperationException( + "JSON format doesn't support non-string as key type of map. " + + "The type is: " + + typeSummary); + } + final JsonToRowDataConverter keyConverter = createConverter(keyType); + final JsonToRowDataConverter valueConverter = createConverter(valueType); + + return jsonNode -> { + Iterator> fields = jsonNode.fields(); + Map result = new HashMap<>(); + while (fields.hasNext()) { + Map.Entry entry = fields.next(); + Object key = keyConverter.convert(TextNode.valueOf(entry.getKey())); + Object value = valueConverter.convert(entry.getValue()); + result.put(key, value); + } + return new GenericMapData(result); + }; + } + + public JsonToRowDataConverter createRowConverter(RowType rowType) { + final JsonToRowDataConverter[] fieldConverters = + rowType.getFields().stream() + .map(RowType.RowField::getType) + .map(this::createConverter) + .toArray(JsonToRowDataConverter[]::new); + final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]); + + return jsonNode -> { + ObjectNode node = (ObjectNode) jsonNode; + int arity = fieldNames.length; + GenericRowData row = new GenericRowData(arity); + for (int i = 0; i < arity; i++) { + String fieldName = fieldNames[i]; + JsonNode field = node.get(fieldName); + Object convertedField = convertField(fieldConverters[i], fieldName, field); + row.setField(i, convertedField); + } + return row; + }; + } + + private Object convertField( + JsonToRowDataConverter fieldConverter, String fieldName, JsonNode field) { + if (field == null) { + if (failOnMissingField) { + throw new JsonParseException("Could not find field with name '" + fieldName + "'."); + } else { + return null; + } + } else { + return fieldConverter.convert(field); + } + } + + private JsonToRowDataConverter wrapIntoNullableConverter(JsonToRowDataConverter converter) { + return jsonNode -> { + if (jsonNode == null || jsonNode.isNull() || jsonNode.isMissingNode()) { + return null; + } + try { + return converter.convert(jsonNode); + } catch (Throwable t) { + if (!ignoreParseErrors) { + throw t; + } + return null; + } + }; + } + + /** Exception which refers to parse errors in converters. */ + private static final class JsonParseException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public JsonParseException(String message) { + super(message); + } + + public JsonParseException(String message, Throwable cause) { + super(message, cause); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java index 61a11fbf8..c30690a1c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java @@ -53,284 +53,296 @@ import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; -/** Tool class used to convert from {@link RowData} to {@link JsonNode}. **/ +/** Tool class used to convert from {@link RowData} to {@link JsonNode}. * */ @Internal public class RowDataToJsonConverters implements Serializable { - private static final long serialVersionUID = 1L; - - /** Timestamp format specification which is used to parse timestamp. */ - private final TimestampFormat timestampFormat; - - /** The handling mode when serializing null keys for map data. */ - private final JsonOptions.MapNullKeyMode mapNullKeyMode; - - /** The string literal when handling mode for map null key LITERAL. is */ - private final String mapNullKeyLiteral; - - public RowDataToJsonConverters( - TimestampFormat timestampFormat, - JsonOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral) { - this.timestampFormat = timestampFormat; - this.mapNullKeyMode = mapNullKeyMode; - this.mapNullKeyLiteral = mapNullKeyLiteral; - } - - /** - * Runtime converter that converts objects of Flink Table & SQL internal data structures - * to corresponding {@link JsonNode}s. - */ - public interface RowDataToJsonConverter extends Serializable { - JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value); - } - - /** - * Creates a runtime converter which is null safe. - */ - public RowDataToJsonConverter createConverter(LogicalType type) { - return wrapIntoNullableConverter(createNotNullConverter(type)); - } - - /** - * Creates a runtime converter which assuming input object is not null. - */ - private RowDataToJsonConverter createNotNullConverter(LogicalType type) { - switch (type.getTypeRoot()) { - case NULL: - return (mapper, reuse, value) -> mapper.getNodeFactory().nullNode(); - case BOOLEAN: - return (mapper, reuse, value) -> mapper.getNodeFactory().booleanNode((boolean) value); - case TINYINT: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((byte) value); - case SMALLINT: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((short) value); - case INTEGER: - case INTERVAL_YEAR_MONTH: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((int) value); - case BIGINT: - case INTERVAL_DAY_TIME: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((long) value); - case FLOAT: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((float) value); - case DOUBLE: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((double) value); - case CHAR: - case VARCHAR: - // value is BinaryString - return (mapper, reuse, value) -> mapper.getNodeFactory().textNode(value.toString()); - case BINARY: - case VARBINARY: - return (mapper, reuse, value) -> mapper.getNodeFactory().binaryNode((byte[]) value); - case DATE: - return createDateConverter(); - case TIME_WITHOUT_TIME_ZONE: - return createTimeConverter(); - case TIMESTAMP_WITHOUT_TIME_ZONE: - return createTimestampConverter(); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return createTimestampWithLocalZone(); - case DECIMAL: - return createDecimalConverter(); - case ARRAY: - return createArrayConverter((ArrayType) type); - case MAP: - MapType mapType = (MapType) type; - return createMapConverter( - mapType.asSummaryString(), mapType.getKeyType(), mapType.getValueType()); - case MULTISET: - MultisetType multisetType = (MultisetType) type; - return createMapConverter - (multisetType.asSummaryString(), multisetType.getElementType(), new IntType()); - case ROW: - return createRowConverter((RowType) type); - case RAW: - default: - throw new UnsupportedOperationException("Not support to parse type: " + type); - } - } - - private RowDataToJsonConverter createDecimalConverter() { - return (mapper, reuse, value) -> { - BigDecimal bd = ((DecimalData) value).toBigDecimal(); - return mapper.getNodeFactory().numberNode(bd); - }; - } - - private RowDataToJsonConverter createDateConverter() { - return (mapper, reuse, value) -> { - int days = (int) value; - LocalDate date = LocalDate.ofEpochDay(days); - return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format(date)); - }; - } - - private RowDataToJsonConverter createTimeConverter() { - return (mapper, reuse, value) -> { - int millisecond = (int) value; - LocalTime time = LocalTime.ofSecondOfDay(millisecond / 1000L); - return mapper.getNodeFactory().textNode(SQL_TIME_FORMAT.format(time)); - }; - } - - private RowDataToJsonConverter createTimestampConverter() { - switch (timestampFormat){ - case ISO_8601: - return (mapper, reuse, value) -> { - TimestampData timestamp = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(ISO8601_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); - }; - case SQL: - return (mapper, reuse, value) -> { - TimestampData timestamp = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(SQL_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); - }; - default: - throw new TableException("Unsupported timestamp format. Validator should have checked that."); - } - } - - private RowDataToJsonConverter createTimestampWithLocalZone() { - switch (timestampFormat){ - case ISO_8601: - return (mapper, reuse, value) -> { - TimestampData timestampWithLocalZone = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format(timestampWithLocalZone.toInstant().atOffset(ZoneOffset.UTC))); - }; - case SQL: - return (mapper, reuse, value) -> { - TimestampData timestampWithLocalZone = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format(timestampWithLocalZone.toInstant().atOffset(ZoneOffset.UTC))); - }; - default: - throw new TableException("Unsupported timestamp format. Validator should have checked that."); - } - } - - private RowDataToJsonConverter createArrayConverter(ArrayType type) { - final LogicalType elementType = type.getElementType(); - final RowDataToJsonConverter elementConverter = createConverter(elementType); - final ArrayData.ElementGetter elementGetter = ArrayData.createElementGetter(elementType); - return (mapper, reuse, value) -> { - ArrayNode node; - - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createArrayNode(); - } else { - node = (ArrayNode) reuse; - node.removeAll(); - } - - ArrayData array = (ArrayData) value; - int numElements = array.size(); - for (int i = 0; i < numElements; i++) { - Object element = elementGetter.getElementOrNull(array, i); - node.add(elementConverter.convert(mapper, null, element)); - } - - return node; - }; - } - - private RowDataToJsonConverter createMapConverter( - String typeSummary, LogicalType keyType, LogicalType valueType) { - if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { - throw new UnsupportedOperationException( - "JSON format doesn't support non-string as key type of map. " + - "The type is: " + typeSummary); - } - final RowDataToJsonConverter valueConverter = createConverter(valueType); - final ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(valueType); - return (mapper, reuse, object) -> { - ObjectNode node; - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createObjectNode(); - } else { - node = (ObjectNode) reuse; - node.removeAll(); - } - - MapData map = (MapData) object; - ArrayData keyArray = map.keyArray(); - ArrayData valueArray = map.valueArray(); - int numElements = map.size(); - for (int i = 0; i < numElements; i++) { - String fieldName = null; - if (keyArray.isNullAt(i)) { - // when map key is null - switch (mapNullKeyMode) { - case LITERAL: - fieldName = mapNullKeyLiteral; - break; - case DROP: - continue; - case FAIL: - throw new RuntimeException(String.format( - "JSON format doesn't support to serialize map data with null keys. " - + "You can drop null key entries or encode null in literals by specifying %s option.", - JsonOptions.MAP_NULL_KEY_MODE.key())); - default: - throw new RuntimeException("Unsupported map null key mode. Validator should have checked that."); - } - } else { - fieldName = keyArray.getString(i).toString(); - } - - Object value = valueGetter.getElementOrNull(valueArray, i); - node.set(fieldName, valueConverter.convert(mapper, node.get(fieldName), value)); - } - - return node; - }; - } - - private RowDataToJsonConverter createRowConverter(RowType type) { - final String[] fieldNames = type.getFieldNames().toArray(new String[0]); - final LogicalType[] fieldTypes = type.getFields().stream() - .map(RowType.RowField::getType) - .toArray(LogicalType[]::new); - final RowDataToJsonConverter[] fieldConverters = Arrays.stream(fieldTypes) - .map(this::createConverter) - .toArray(RowDataToJsonConverter[]::new); - final int fieldCount = type.getFieldCount(); - final RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[fieldTypes.length]; - for (int i = 0; i < fieldCount; i++) { - fieldGetters[i] = RowData.createFieldGetter(fieldTypes[i], i); - } - - return (mapper, reuse, value) -> { - ObjectNode node; - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createObjectNode(); - } else { - node = (ObjectNode) reuse; - } - RowData row = (RowData) value; - for (int i = 0; i < fieldCount; i++) { - String fieldName = fieldNames[i]; - Object field = fieldGetters[i].getFieldOrNull(row); - node.set(fieldName, fieldConverters[i].convert(mapper, node.get(fieldName), field)); - } - return node; - }; - } - - private RowDataToJsonConverter wrapIntoNullableConverter( - RowDataToJsonConverter converter) { - return (mapper, reuse, object) -> { - if (object == null) { - return mapper.getNodeFactory().nullNode(); - } - - return converter.convert(mapper, reuse, object); - }; - } - + private static final long serialVersionUID = 1L; + + /** Timestamp format specification which is used to parse timestamp. */ + private final TimestampFormat timestampFormat; + + /** The handling mode when serializing null keys for map data. */ + private final JsonOptions.MapNullKeyMode mapNullKeyMode; + + /** The string literal when handling mode for map null key LITERAL. is */ + private final String mapNullKeyLiteral; + + public RowDataToJsonConverters( + TimestampFormat timestampFormat, + JsonOptions.MapNullKeyMode mapNullKeyMode, + String mapNullKeyLiteral) { + this.timestampFormat = timestampFormat; + this.mapNullKeyMode = mapNullKeyMode; + this.mapNullKeyLiteral = mapNullKeyLiteral; + } + + /** + * Runtime converter that converts objects of Flink Table & SQL internal data structures to + * corresponding {@link JsonNode}s. + */ + public interface RowDataToJsonConverter extends Serializable { + JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value); + } + + /** Creates a runtime converter which is null safe. */ + public RowDataToJsonConverter createConverter(LogicalType type) { + return wrapIntoNullableConverter(createNotNullConverter(type)); + } + + /** Creates a runtime converter which assuming input object is not null. */ + private RowDataToJsonConverter createNotNullConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return (mapper, reuse, value) -> mapper.getNodeFactory().nullNode(); + case BOOLEAN: + return (mapper, reuse, value) -> + mapper.getNodeFactory().booleanNode((boolean) value); + case TINYINT: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((byte) value); + case SMALLINT: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((short) value); + case INTEGER: + case INTERVAL_YEAR_MONTH: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((int) value); + case BIGINT: + case INTERVAL_DAY_TIME: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((long) value); + case FLOAT: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((float) value); + case DOUBLE: + return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((double) value); + case CHAR: + case VARCHAR: + // value is BinaryString + return (mapper, reuse, value) -> mapper.getNodeFactory().textNode(value.toString()); + case BINARY: + case VARBINARY: + return (mapper, reuse, value) -> mapper.getNodeFactory().binaryNode((byte[]) value); + case DATE: + return createDateConverter(); + case TIME_WITHOUT_TIME_ZONE: + return createTimeConverter(); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return createTimestampConverter(); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return createTimestampWithLocalZone(); + case DECIMAL: + return createDecimalConverter(); + case ARRAY: + return createArrayConverter((ArrayType) type); + case MAP: + MapType mapType = (MapType) type; + return createMapConverter( + mapType.asSummaryString(), mapType.getKeyType(), mapType.getValueType()); + case MULTISET: + MultisetType multisetType = (MultisetType) type; + return createMapConverter( + multisetType.asSummaryString(), + multisetType.getElementType(), + new IntType()); + case ROW: + return createRowConverter((RowType) type); + case RAW: + default: + throw new UnsupportedOperationException("Not support to parse type: " + type); + } + } + + private RowDataToJsonConverter createDecimalConverter() { + return (mapper, reuse, value) -> { + BigDecimal bd = ((DecimalData) value).toBigDecimal(); + return mapper.getNodeFactory().numberNode(bd); + }; + } + + private RowDataToJsonConverter createDateConverter() { + return (mapper, reuse, value) -> { + int days = (int) value; + LocalDate date = LocalDate.ofEpochDay(days); + return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format(date)); + }; + } + + private RowDataToJsonConverter createTimeConverter() { + return (mapper, reuse, value) -> { + int millisecond = (int) value; + LocalTime time = LocalTime.ofSecondOfDay(millisecond / 1000L); + return mapper.getNodeFactory().textNode(SQL_TIME_FORMAT.format(time)); + }; + } + + private RowDataToJsonConverter createTimestampConverter() { + switch (timestampFormat) { + case ISO_8601: + return (mapper, reuse, value) -> { + TimestampData timestamp = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(ISO8601_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); + }; + case SQL: + return (mapper, reuse, value) -> { + TimestampData timestamp = (TimestampData) value; + return mapper.getNodeFactory() + .textNode(SQL_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); + }; + default: + throw new TableException( + "Unsupported timestamp format. Validator should have checked that."); + } + } + + private RowDataToJsonConverter createTimestampWithLocalZone() { + switch (timestampFormat) { + case ISO_8601: + return (mapper, reuse, value) -> { + TimestampData timestampWithLocalZone = (TimestampData) value; + return mapper.getNodeFactory() + .textNode( + ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format( + timestampWithLocalZone + .toInstant() + .atOffset(ZoneOffset.UTC))); + }; + case SQL: + return (mapper, reuse, value) -> { + TimestampData timestampWithLocalZone = (TimestampData) value; + return mapper.getNodeFactory() + .textNode( + SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format( + timestampWithLocalZone + .toInstant() + .atOffset(ZoneOffset.UTC))); + }; + default: + throw new TableException( + "Unsupported timestamp format. Validator should have checked that."); + } + } + + private RowDataToJsonConverter createArrayConverter(ArrayType type) { + final LogicalType elementType = type.getElementType(); + final RowDataToJsonConverter elementConverter = createConverter(elementType); + final ArrayData.ElementGetter elementGetter = ArrayData.createElementGetter(elementType); + return (mapper, reuse, value) -> { + ArrayNode node; + + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createArrayNode(); + } else { + node = (ArrayNode) reuse; + node.removeAll(); + } + + ArrayData array = (ArrayData) value; + int numElements = array.size(); + for (int i = 0; i < numElements; i++) { + Object element = elementGetter.getElementOrNull(array, i); + node.add(elementConverter.convert(mapper, null, element)); + } + + return node; + }; + } + + private RowDataToJsonConverter createMapConverter( + String typeSummary, LogicalType keyType, LogicalType valueType) { + if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { + throw new UnsupportedOperationException( + "JSON format doesn't support non-string as key type of map. " + + "The type is: " + + typeSummary); + } + final RowDataToJsonConverter valueConverter = createConverter(valueType); + final ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(valueType); + return (mapper, reuse, object) -> { + ObjectNode node; + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createObjectNode(); + } else { + node = (ObjectNode) reuse; + node.removeAll(); + } + + MapData map = (MapData) object; + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + int numElements = map.size(); + for (int i = 0; i < numElements; i++) { + String fieldName = null; + if (keyArray.isNullAt(i)) { + // when map key is null + switch (mapNullKeyMode) { + case LITERAL: + fieldName = mapNullKeyLiteral; + break; + case DROP: + continue; + case FAIL: + throw new RuntimeException( + String.format( + "JSON format doesn't support to serialize map data with null keys. " + + "You can drop null key entries or encode null in literals by specifying %s option.", + JsonOptions.MAP_NULL_KEY_MODE.key())); + default: + throw new RuntimeException( + "Unsupported map null key mode. Validator should have checked that."); + } + } else { + fieldName = keyArray.getString(i).toString(); + } + + Object value = valueGetter.getElementOrNull(valueArray, i); + node.set(fieldName, valueConverter.convert(mapper, node.get(fieldName), value)); + } + + return node; + }; + } + + private RowDataToJsonConverter createRowConverter(RowType type) { + final String[] fieldNames = type.getFieldNames().toArray(new String[0]); + final LogicalType[] fieldTypes = + type.getFields().stream() + .map(RowType.RowField::getType) + .toArray(LogicalType[]::new); + final RowDataToJsonConverter[] fieldConverters = + Arrays.stream(fieldTypes) + .map(this::createConverter) + .toArray(RowDataToJsonConverter[]::new); + final int fieldCount = type.getFieldCount(); + final RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[fieldTypes.length]; + for (int i = 0; i < fieldCount; i++) { + fieldGetters[i] = RowData.createFieldGetter(fieldTypes[i], i); + } + + return (mapper, reuse, value) -> { + ObjectNode node; + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { + node = mapper.createObjectNode(); + } else { + node = (ObjectNode) reuse; + } + RowData row = (RowData) value; + for (int i = 0; i < fieldCount; i++) { + String fieldName = fieldNames[i]; + Object field = fieldGetters[i].getFieldOrNull(row); + node.set(fieldName, fieldConverters[i].convert(mapper, node.get(fieldName), field)); + } + return node; + }; + } + + private RowDataToJsonConverter wrapIntoNullableConverter(RowDataToJsonConverter converter) { + return (mapper, reuse, object) -> { + if (object == null) { + return mapper.getNodeFactory().nullNode(); + } + + return converter.convert(mapper, reuse, object); + }; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java index f0b4b3adf..c81aa20b6 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java @@ -23,56 +23,65 @@ import java.time.temporal.ChronoField; /** - * Time formats and timestamp formats respecting the RFC3339 specification, ISO-8601 specification and SQL specification. + * Time formats and timestamp formats respecting the RFC3339 specification, ISO-8601 specification + * and SQL specification. */ class TimeFormats { - /** Formatter for RFC 3339-compliant string representation of a time value. */ - static final DateTimeFormatter RFC3339_TIME_FORMAT = new DateTimeFormatterBuilder() - .appendPattern("HH:mm:ss") - .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) - .appendPattern("'Z'") - .toFormatter(); + /** Formatter for RFC 3339-compliant string representation of a time value. */ + static final DateTimeFormatter RFC3339_TIME_FORMAT = + new DateTimeFormatterBuilder() + .appendPattern("HH:mm:ss") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) + .appendPattern("'Z'") + .toFormatter(); - /** Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC timezone). */ - static final DateTimeFormatter RFC3339_TIMESTAMP_FORMAT = new DateTimeFormatterBuilder() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral('T') - .append(RFC3339_TIME_FORMAT) - .toFormatter(); + /** + * Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC + * timezone). + */ + static final DateTimeFormatter RFC3339_TIMESTAMP_FORMAT = + new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral('T') + .append(RFC3339_TIME_FORMAT) + .toFormatter(); - /** Formatter for ISO8601 string representation of a timestamp value (without UTC timezone). */ - static final DateTimeFormatter ISO8601_TIMESTAMP_FORMAT = DateTimeFormatter.ISO_LOCAL_DATE_TIME; + /** Formatter for ISO8601 string representation of a timestamp value (without UTC timezone). */ + static final DateTimeFormatter ISO8601_TIMESTAMP_FORMAT = DateTimeFormatter.ISO_LOCAL_DATE_TIME; - /** Formatter for ISO8601 string representation of a timestamp value (with UTC timezone). */ - static final DateTimeFormatter ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT = new DateTimeFormatterBuilder() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral('T') - .append(DateTimeFormatter.ISO_LOCAL_TIME) - .appendPattern("'Z'") - .toFormatter(); + /** Formatter for ISO8601 string representation of a timestamp value (with UTC timezone). */ + static final DateTimeFormatter ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT = + new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral('T') + .append(DateTimeFormatter.ISO_LOCAL_TIME) + .appendPattern("'Z'") + .toFormatter(); - /** Formatter for SQL string representation of a time value. */ - static final DateTimeFormatter SQL_TIME_FORMAT = new DateTimeFormatterBuilder() - .appendPattern("HH:mm:ss") - .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) - .toFormatter(); + /** Formatter for SQL string representation of a time value. */ + static final DateTimeFormatter SQL_TIME_FORMAT = + new DateTimeFormatterBuilder() + .appendPattern("HH:mm:ss") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) + .toFormatter(); - /** Formatter for SQL string representation of a timestamp value (without UTC timezone). */ - static final DateTimeFormatter SQL_TIMESTAMP_FORMAT = new DateTimeFormatterBuilder() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral(' ') - .append(SQL_TIME_FORMAT) - .toFormatter(); + /** Formatter for SQL string representation of a timestamp value (without UTC timezone). */ + static final DateTimeFormatter SQL_TIMESTAMP_FORMAT = + new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(SQL_TIME_FORMAT) + .toFormatter(); - /** Formatter for SQL string representation of a timestamp value (with UTC timezone). */ - static final DateTimeFormatter SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT = new DateTimeFormatterBuilder() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral(' ') - .append(SQL_TIME_FORMAT) - .appendPattern("'Z'") - .toFormatter(); + /** Formatter for SQL string representation of a timestamp value (with UTC timezone). */ + static final DateTimeFormatter SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT = + new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(SQL_TIME_FORMAT) + .appendPattern("'Z'") + .toFormatter(); - private TimeFormats() { - } + private TimeFormats() {} } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java index f06bfada5..8dacfc123 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java @@ -20,16 +20,20 @@ import org.apache.flink.annotation.Internal; -/** - * Timestamp format Enums. - */ +/** Timestamp format Enums. */ @Internal public enum TimestampFormat { - /** Options to specify TIMESTAMP/TIMESTAMP_WITH_LOCAL_ZONE format. It will parse TIMESTAMP in "yyyy-MM-dd HH:mm:ss.s{precision}" format, - * TIMESTAMP_WITH_LOCAL_TIMEZONE in "yyyy-MM-dd HH:mm:ss.s{precision}'Z'" and output in the same format.*/ - SQL, + /** + * Options to specify TIMESTAMP/TIMESTAMP_WITH_LOCAL_ZONE format. It will parse TIMESTAMP in + * "yyyy-MM-dd HH:mm:ss.s{precision}" format, TIMESTAMP_WITH_LOCAL_TIMEZONE in "yyyy-MM-dd + * HH:mm:ss.s{precision}'Z'" and output in the same format. + */ + SQL, - /** Options to specify TIMESTAMP/TIMESTAMP_WITH_LOCAL_ZONE format. It will parse TIMESTAMP in "yyyy-MM-ddTHH:mm:ss.s{precision}" format, - * TIMESTAMP_WITH_LOCAL_TIMEZONE in "yyyy-MM-ddTHH:mm:ss.s{precision}'Z'" and output in the same format.*/ - ISO_8601 + /** + * Options to specify TIMESTAMP/TIMESTAMP_WITH_LOCAL_ZONE format. It will parse TIMESTAMP in + * "yyyy-MM-ddTHH:mm:ss.s{precision}" format, TIMESTAMP_WITH_LOCAL_TIMEZONE in + * "yyyy-MM-ddTHH:mm:ss.s{precision}'Z'" and output in the same format. + */ + ISO_8601 } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index fe9227a30..6a81f315a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -41,245 +41,239 @@ import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** - * Deserialization schema from Canal JSON to Flink Table/SQL internal data structure {@link RowData}. - * The deserialization schema knows Debezium's schema definition and can extract the database data - * and convert into {@link RowData} with {@link RowKind}. + * Deserialization schema from Canal JSON to Flink Table/SQL internal data structure {@link + * RowData}. The deserialization schema knows Debezium's schema definition and can extract the + * database data and convert into {@link RowData} with {@link RowKind}. * - *

Deserializes a byte[] message as a JSON object and reads - * the specified fields. + *

Deserializes a byte[] message as a JSON object and reads the specified fields. * *

Failures during deserialization are forwarded as wrapped IOExceptions. * * @see Alibaba Canal */ public final class CanalJsonDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - - private static final String OP_INSERT = "INSERT"; - private static final String OP_UPDATE = "UPDATE"; - private static final String OP_DELETE = "DELETE"; - private static final String OP_CREATE = "CREATE"; - - /** The deserializer to deserialize Debezium JSON data. */ - private final JsonRowDataDeserializationSchema jsonDeserializer; - - /** TypeInformation of the produced {@link RowData}. **/ - private final TypeInformation resultTypeInfo; - - /** Only read changelogs from the specific database. */ - private final @Nullable String database; - - /** Only read changelogs from the specific table. */ - private final @Nullable String table; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - /** Number of fields. */ - private final int fieldCount; - - private CanalJsonDeserializationSchema( - RowType rowType, - TypeInformation resultTypeInfo, - @Nullable String database, - @Nullable String table, - boolean ignoreParseErrors, - TimestampFormat timestampFormatOption) { - this.resultTypeInfo = resultTypeInfo; - this.database = database; - this.table = table; - this.ignoreParseErrors = ignoreParseErrors; - this.fieldCount = rowType.getFieldCount(); - this.jsonDeserializer = new JsonRowDataDeserializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - // the result type is never used, so it's fine to pass in Canal's result type - resultTypeInfo, - false, // ignoreParseErrors already contains the functionality of failOnMissingField - ignoreParseErrors, - timestampFormatOption); - - } - - // ------------------------------------------------------------------------------------------ - // Builder - // ------------------------------------------------------------------------------------------ - - /** - * Creates A builder for building a {@link CanalJsonDeserializationSchema}. - */ - public static Builder builder(RowType rowType, TypeInformation resultTypeInfo) { - return new Builder(rowType, resultTypeInfo); - } - - /** - * A builder for creating a {@link CanalJsonDeserializationSchema}. - */ - @Internal - public static final class Builder { - private final RowType rowType; - private final TypeInformation resultTypeInfo; - private String database = null; - private String table = null; - private boolean ignoreParseErrors = false; - private TimestampFormat timestampFormat = TimestampFormat.SQL; - - private Builder(RowType rowType, TypeInformation resultTypeInfo) { - this.rowType = rowType; - this.resultTypeInfo = resultTypeInfo; - } - - public Builder setDatabase(String database) { - this.database = database; - return this; - } - - public Builder setTable(String table) { - this.table = table; - return this; - } - - public Builder setIgnoreParseErrors(boolean ignoreParseErrors) { - this.ignoreParseErrors = ignoreParseErrors; - return this; - } - - public Builder setTimestampFormat(TimestampFormat timestampFormat) { - this.timestampFormat = timestampFormat; - return this; - } - - public CanalJsonDeserializationSchema build() { - return new CanalJsonDeserializationSchema( - rowType, - resultTypeInfo, - database, - table, - ignoreParseErrors, - timestampFormat - ); - } - } - - - // ------------------------------------------------------------------------------------------ - - @Override - public RowData deserialize(byte[] message) throws IOException { - throw new RuntimeException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); - } - - @Override - public void deserialize(byte[] message, Collector out) throws IOException { - try { - RowData row = jsonDeserializer.deserialize(message); - if (database != null) { - String currentDatabase = row.getString(3).toString(); - if (!database.equals(currentDatabase)) { - return; - } - } - if (table != null) { - String currentTable = row.getString(4).toString(); - if (!table.equals(currentTable)) { - return; - } - } - String type = row.getString(2).toString(); // "type" field - if (OP_INSERT.equals(type)) { - // "data" field is an array of row, contains inserted rows - ArrayData data = row.getArray(0); - for (int i = 0; i < data.size(); i++) { - RowData insert = data.getRow(i, fieldCount); - insert.setRowKind(RowKind.INSERT); - out.collect(insert); - } - } else if (OP_UPDATE.equals(type)) { - // "data" field is an array of row, contains new rows - ArrayData data = row.getArray(0); - // "old" field is an array of row, contains old values - ArrayData old = row.getArray(1); - for (int i = 0; i < data.size(); i++) { - // the underlying JSON deserialization schema always produce GenericRowData. - GenericRowData after = (GenericRowData) data.getRow(i, fieldCount); - GenericRowData before = (GenericRowData) old.getRow(i, fieldCount); - for (int f = 0; f < fieldCount; f++) { - if (before.isNullAt(f)) { - // not null fields in "old" (before) means the fields are changed - // null/empty fields in "old" (before) means the fields are not changed - // so we just copy the not changed fields into before - before.setField(f, after.getField(f)); - } - } - before.setRowKind(RowKind.UPDATE_BEFORE); - after.setRowKind(RowKind.UPDATE_AFTER); - out.collect(before); - out.collect(after); - } - } else if (OP_DELETE.equals(type)) { - // "data" field is an array of row, contains deleted rows - ArrayData data = row.getArray(0); - for (int i = 0; i < data.size(); i++) { - RowData insert = data.getRow(i, fieldCount); - insert.setRowKind(RowKind.DELETE); - out.collect(insert); - } - } else if (OP_CREATE.equals(type)){ - // "data" field is null and "type" is "CREATE" which means - // this is a DDL change event, and we should skip it. - return; - } else { - if (!ignoreParseErrors) { - throw new IOException(format( - "Unknown \"type\" value \"%s\". The Canal JSON message is '%s'", type, new String(message))); - } - } - } catch (Throwable t) { - // a big try catch to protect the processing. - if (!ignoreParseErrors) { - throw new IOException(format( - "Corrupt Canal JSON message '%s'.", new String(message)), t); - } - } - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return resultTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - CanalJsonDeserializationSchema that = (CanalJsonDeserializationSchema) o; - return ignoreParseErrors == that.ignoreParseErrors && - fieldCount == that.fieldCount && - Objects.equals(jsonDeserializer, that.jsonDeserializer) && - Objects.equals(resultTypeInfo, that.resultTypeInfo); - } - - @Override - public int hashCode() { - return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors, fieldCount); - } - - private static RowType createJsonRowType(DataType databaseSchema) { - // Canal JSON contains other information, e.g. "ts", "sql", but we don't need them - return (RowType) DataTypes.ROW( - DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)), - DataTypes.FIELD("old", DataTypes.ARRAY(databaseSchema)), - DataTypes.FIELD("type", DataTypes.STRING()), - DataTypes.FIELD("database", DataTypes.STRING()), - DataTypes.FIELD("table", DataTypes.STRING())).getLogicalType(); - } + private static final long serialVersionUID = 1L; + + private static final String OP_INSERT = "INSERT"; + private static final String OP_UPDATE = "UPDATE"; + private static final String OP_DELETE = "DELETE"; + private static final String OP_CREATE = "CREATE"; + + /** The deserializer to deserialize Debezium JSON data. */ + private final JsonRowDataDeserializationSchema jsonDeserializer; + + /** TypeInformation of the produced {@link RowData}. * */ + private final TypeInformation resultTypeInfo; + + /** Only read changelogs from the specific database. */ + private final @Nullable String database; + + /** Only read changelogs from the specific table. */ + private final @Nullable String table; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + /** Number of fields. */ + private final int fieldCount; + + private CanalJsonDeserializationSchema( + RowType rowType, + TypeInformation resultTypeInfo, + @Nullable String database, + @Nullable String table, + boolean ignoreParseErrors, + TimestampFormat timestampFormatOption) { + this.resultTypeInfo = resultTypeInfo; + this.database = database; + this.table = table; + this.ignoreParseErrors = ignoreParseErrors; + this.fieldCount = rowType.getFieldCount(); + this.jsonDeserializer = + new JsonRowDataDeserializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + // the result type is never used, so it's fine to pass in Canal's result + // type + resultTypeInfo, + false, // ignoreParseErrors already contains the functionality of + // failOnMissingField + ignoreParseErrors, + timestampFormatOption); + } + + // ------------------------------------------------------------------------------------------ + // Builder + // ------------------------------------------------------------------------------------------ + + /** Creates A builder for building a {@link CanalJsonDeserializationSchema}. */ + public static Builder builder(RowType rowType, TypeInformation resultTypeInfo) { + return new Builder(rowType, resultTypeInfo); + } + + /** A builder for creating a {@link CanalJsonDeserializationSchema}. */ + @Internal + public static final class Builder { + private final RowType rowType; + private final TypeInformation resultTypeInfo; + private String database = null; + private String table = null; + private boolean ignoreParseErrors = false; + private TimestampFormat timestampFormat = TimestampFormat.SQL; + + private Builder(RowType rowType, TypeInformation resultTypeInfo) { + this.rowType = rowType; + this.resultTypeInfo = resultTypeInfo; + } + + public Builder setDatabase(String database) { + this.database = database; + return this; + } + + public Builder setTable(String table) { + this.table = table; + return this; + } + + public Builder setIgnoreParseErrors(boolean ignoreParseErrors) { + this.ignoreParseErrors = ignoreParseErrors; + return this; + } + + public Builder setTimestampFormat(TimestampFormat timestampFormat) { + this.timestampFormat = timestampFormat; + return this; + } + + public CanalJsonDeserializationSchema build() { + return new CanalJsonDeserializationSchema( + rowType, resultTypeInfo, database, table, ignoreParseErrors, timestampFormat); + } + } + + // ------------------------------------------------------------------------------------------ + + @Override + public RowData deserialize(byte[] message) throws IOException { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + + @Override + public void deserialize(byte[] message, Collector out) throws IOException { + try { + RowData row = jsonDeserializer.deserialize(message); + if (database != null) { + String currentDatabase = row.getString(3).toString(); + if (!database.equals(currentDatabase)) { + return; + } + } + if (table != null) { + String currentTable = row.getString(4).toString(); + if (!table.equals(currentTable)) { + return; + } + } + String type = row.getString(2).toString(); // "type" field + if (OP_INSERT.equals(type)) { + // "data" field is an array of row, contains inserted rows + ArrayData data = row.getArray(0); + for (int i = 0; i < data.size(); i++) { + RowData insert = data.getRow(i, fieldCount); + insert.setRowKind(RowKind.INSERT); + out.collect(insert); + } + } else if (OP_UPDATE.equals(type)) { + // "data" field is an array of row, contains new rows + ArrayData data = row.getArray(0); + // "old" field is an array of row, contains old values + ArrayData old = row.getArray(1); + for (int i = 0; i < data.size(); i++) { + // the underlying JSON deserialization schema always produce GenericRowData. + GenericRowData after = (GenericRowData) data.getRow(i, fieldCount); + GenericRowData before = (GenericRowData) old.getRow(i, fieldCount); + for (int f = 0; f < fieldCount; f++) { + if (before.isNullAt(f)) { + // not null fields in "old" (before) means the fields are changed + // null/empty fields in "old" (before) means the fields are not changed + // so we just copy the not changed fields into before + before.setField(f, after.getField(f)); + } + } + before.setRowKind(RowKind.UPDATE_BEFORE); + after.setRowKind(RowKind.UPDATE_AFTER); + out.collect(before); + out.collect(after); + } + } else if (OP_DELETE.equals(type)) { + // "data" field is an array of row, contains deleted rows + ArrayData data = row.getArray(0); + for (int i = 0; i < data.size(); i++) { + RowData insert = data.getRow(i, fieldCount); + insert.setRowKind(RowKind.DELETE); + out.collect(insert); + } + } else if (OP_CREATE.equals(type)) { + // "data" field is null and "type" is "CREATE" which means + // this is a DDL change event, and we should skip it. + return; + } else { + if (!ignoreParseErrors) { + throw new IOException( + format( + "Unknown \"type\" value \"%s\". The Canal JSON message is '%s'", + type, new String(message))); + } + } + } catch (Throwable t) { + // a big try catch to protect the processing. + if (!ignoreParseErrors) { + throw new IOException( + format("Corrupt Canal JSON message '%s'.", new String(message)), t); + } + } + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return resultTypeInfo; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CanalJsonDeserializationSchema that = (CanalJsonDeserializationSchema) o; + return ignoreParseErrors == that.ignoreParseErrors + && fieldCount == that.fieldCount + && Objects.equals(jsonDeserializer, that.jsonDeserializer) + && Objects.equals(resultTypeInfo, that.resultTypeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors, fieldCount); + } + + private static RowType createJsonRowType(DataType databaseSchema) { + // Canal JSON contains other information, e.g. "ts", "sql", but we don't need them + return (RowType) + DataTypes.ROW( + DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)), + DataTypes.FIELD("old", DataTypes.ARRAY(databaseSchema)), + DataTypes.FIELD("type", DataTypes.STRING()), + DataTypes.FIELD("database", DataTypes.STRING()), + DataTypes.FIELD("table", DataTypes.STRING())) + .getLogicalType(); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java index 41eb86c02..b199c6f2b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java @@ -53,109 +53,103 @@ import static org.apache.flink.formats.json.canal.CanalJsonOptions.validateEncodingFormatOptions; /** - * Format factory for providing configured instances of Canal JSON to RowData {@link DeserializationSchema}. + * Format factory for providing configured instances of Canal JSON to RowData {@link + * DeserializationSchema}. */ -public class CanalJsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "canal-json"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateDecodingFormatOptions(formatOptions); - - final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); - String database = formatOptions.getOptional(DATABASE_INCLUDE).orElse(null); - String table = formatOptions.getOptional(TABLE_INCLUDE).orElse(null); - - return new DecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType producedDataType) { - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation rowDataTypeInfo = - context.createTypeInformation(producedDataType); - return CanalJsonDeserializationSchema - .builder(rowType, rowDataTypeInfo) - .setIgnoreParseErrors(ignoreParseErrors) - .setTimestampFormat(timestampFormatOption) - .setDatabase(database) - .setTable(table) - .build(); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - }; - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateEncodingFormatOptions(formatOptions); - - TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); - String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); - - return new EncodingFormat>() { - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - @Override - public SerializationSchema createRuntimeEncoder(DynamicTableSink.Context context, DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new CanalJsonSerializationSchema( - rowType, - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral - ); - } - }; - - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return Collections.emptySet(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(IGNORE_PARSE_ERRORS); - options.add(TIMESTAMP_FORMAT); - options.add(DATABASE_INCLUDE); - options.add(TABLE_INCLUDE); - options.add(JSON_MAP_NULL_KEY_MODE); - options.add(JSON_MAP_NULL_KEY_LITERAL); - return options; - } - +public class CanalJsonFormatFactory + implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "canal-json"; + + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateDecodingFormatOptions(formatOptions); + + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); + String database = formatOptions.getOptional(DATABASE_INCLUDE).orElse(null); + String table = formatOptions.getOptional(TABLE_INCLUDE).orElse(null); + + return new DecodingFormat>() { + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + context.createTypeInformation(producedDataType); + return CanalJsonDeserializationSchema.builder(rowType, rowDataTypeInfo) + .setIgnoreParseErrors(ignoreParseErrors) + .setTimestampFormat(timestampFormatOption) + .setDatabase(database) + .setTable(table) + .build(); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + }; + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateEncodingFormatOptions(formatOptions); + + TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); + + return new EncodingFormat>() { + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new CanalJsonSerializationSchema( + rowType, timestampFormat, mapNullKeyMode, mapNullKeyLiteral); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); + options.add(DATABASE_INCLUDE); + options.add(TABLE_INCLUDE); + options.add(JSON_MAP_NULL_KEY_MODE); + options.add(JSON_MAP_NULL_KEY_LITERAL); + return options; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java index a4ce7a586..f39c0d0c0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java @@ -23,47 +23,43 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.formats.json.JsonOptions; -/** - * Option utils for canal-json format. - */ +/** Option utils for canal-json format. */ public class CanalJsonOptions { - public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; - - public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; - public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; + public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; - public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = JsonOptions.MAP_NULL_KEY_LITERAL; + public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; - public static final ConfigOption DATABASE_INCLUDE = ConfigOptions - .key("database.include") - .stringType() - .noDefaultValue() - .withDescription("Only read changelog rows which match the specific database (by comparing the \"database\" meta field in the record)."); + public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = + JsonOptions.MAP_NULL_KEY_LITERAL; - public static final ConfigOption TABLE_INCLUDE = ConfigOptions - .key("table.include") - .stringType() - .noDefaultValue() - .withDescription("Only read changelog rows which match the specific table (by comparing the \"table\" meta field in the record)."); + public static final ConfigOption DATABASE_INCLUDE = + ConfigOptions.key("database.include") + .stringType() + .noDefaultValue() + .withDescription( + "Only read changelog rows which match the specific database (by comparing the \"database\" meta field in the record)."); - // -------------------------------------------------------------------------------------------- - // Validation - // -------------------------------------------------------------------------------------------- + public static final ConfigOption TABLE_INCLUDE = + ConfigOptions.key("table.include") + .stringType() + .noDefaultValue() + .withDescription( + "Only read changelog rows which match the specific table (by comparing the \"table\" meta field in the record)."); - /** - * Validator for canal decoding format. - */ - public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateDecodingFormatOptions(tableOptions); - } + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- - /** - * Validator for canal encoding format. - */ - public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateEncodingFormatOptions(tableOptions); - } + /** Validator for canal decoding format. */ + public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateDecodingFormatOptions(tableOptions); + } + /** Validator for canal encoding format. */ + public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateEncodingFormatOptions(tableOptions); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java index 1db9660e3..9b5a2c03e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java @@ -37,91 +37,93 @@ import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** - * Serialization schema that serializes an object of Flink Table/SQL internal data structure {@link RowData} into a Canal JSON bytes. + * Serialization schema that serializes an object of Flink Table/SQL internal data structure {@link + * RowData} into a Canal JSON bytes. * * @see Alibaba Canal */ public class CanalJsonSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = 1L; - - private static final StringData OP_INSERT = StringData.fromString("INSERT"); - private static final StringData OP_DELETE = StringData.fromString("DELETE"); - - private transient GenericRowData reuse; - - /** - * The serializer to serialize Canal JSON data. - */ - private final JsonRowDataSerializationSchema jsonSerializer; - - public CanalJsonSerializationSchema( - RowType rowType, - TimestampFormat timestampFormat, - JsonOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral) { - jsonSerializer = new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral); - } - - @Override - public void open(InitializationContext context) { - reuse = new GenericRowData(2); - } - - @Override - public byte[] serialize(RowData row) { - try { - StringData opType = rowKind2String(row.getRowKind()); - ArrayData arrayData = new GenericArrayData(new RowData[]{row}); - reuse.setField(0, arrayData); - reuse.setField(1, opType); - return jsonSerializer.serialize(reuse); - } catch (Throwable t) { - throw new RuntimeException("Could not serialize row '" + row + "'.", t); - } - - } - - private StringData rowKind2String(RowKind rowKind) { - switch (rowKind) { - case INSERT: - case UPDATE_AFTER: - return OP_INSERT; - case UPDATE_BEFORE: - case DELETE: - return OP_DELETE; - default: - throw new UnsupportedOperationException("Unsupported operation '" + rowKind + "' for row kind."); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - CanalJsonSerializationSchema that = (CanalJsonSerializationSchema) o; - return Objects.equals(jsonSerializer, that.jsonSerializer); - } - - @Override - public int hashCode() { - return Objects.hash(jsonSerializer); - } - - private static RowType createJsonRowType(DataType databaseSchema) { - // Canal JSON contains other information, e.g. "database", "ts" - // but we don't need them - // and we don't need "old" , because can not support UPDATE_BEFORE,UPDATE_AFTER - return (RowType) DataTypes.ROW( - DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)), - DataTypes.FIELD("type", DataTypes.STRING())).getLogicalType(); - } + private static final long serialVersionUID = 1L; + + private static final StringData OP_INSERT = StringData.fromString("INSERT"); + private static final StringData OP_DELETE = StringData.fromString("DELETE"); + + private transient GenericRowData reuse; + + /** The serializer to serialize Canal JSON data. */ + private final JsonRowDataSerializationSchema jsonSerializer; + + public CanalJsonSerializationSchema( + RowType rowType, + TimestampFormat timestampFormat, + JsonOptions.MapNullKeyMode mapNullKeyMode, + String mapNullKeyLiteral) { + jsonSerializer = + new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral); + } + + @Override + public void open(InitializationContext context) { + reuse = new GenericRowData(2); + } + + @Override + public byte[] serialize(RowData row) { + try { + StringData opType = rowKind2String(row.getRowKind()); + ArrayData arrayData = new GenericArrayData(new RowData[] {row}); + reuse.setField(0, arrayData); + reuse.setField(1, opType); + return jsonSerializer.serialize(reuse); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'.", t); + } + } + + private StringData rowKind2String(RowKind rowKind) { + switch (rowKind) { + case INSERT: + case UPDATE_AFTER: + return OP_INSERT; + case UPDATE_BEFORE: + case DELETE: + return OP_DELETE; + default: + throw new UnsupportedOperationException( + "Unsupported operation '" + rowKind + "' for row kind."); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CanalJsonSerializationSchema that = (CanalJsonSerializationSchema) o; + return Objects.equals(jsonSerializer, that.jsonSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(jsonSerializer); + } + + private static RowType createJsonRowType(DataType databaseSchema) { + // Canal JSON contains other information, e.g. "database", "ts" + // but we don't need them + // and we don't need "old" , because can not support UPDATE_BEFORE,UPDATE_AFTER + return (RowType) + DataTypes.ROW( + DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)), + DataTypes.FIELD("type", DataTypes.STRING())) + .getLogicalType(); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java index ab3203f5c..22de3c9df 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java @@ -42,241 +42,241 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -/** - * {@link DecodingFormat} for Debezium using JSON encoding. - */ +/** {@link DecodingFormat} for Debezium using JSON encoding. */ public class DebeziumJsonDecodingFormat implements DecodingFormat> { - // -------------------------------------------------------------------------------------------- - // Mutable attributes - // -------------------------------------------------------------------------------------------- - - private List metadataKeys; - - // -------------------------------------------------------------------------------------------- - // Debezium-specific attributes - // -------------------------------------------------------------------------------------------- - - private final boolean schemaInclude; - - private final boolean ignoreParseErrors; - - private final TimestampFormat timestampFormat; - - public DebeziumJsonDecodingFormat( - boolean schemaInclude, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - this.schemaInclude = schemaInclude; - this.ignoreParseErrors = ignoreParseErrors; - this.timestampFormat = timestampFormat; - this.metadataKeys = Collections.emptyList(); - } - - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, - DataType physicalDataType) { - - final List readableMetadata = metadataKeys.stream() - .map(k -> - Stream.of(ReadableMetadata.values()) - .filter(rm -> rm.key.equals(k)) - .findFirst() - .orElseThrow(IllegalStateException::new)) - .collect(Collectors.toList()); - - final List metadataFields = readableMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList()); - - final DataType producedDataType = DataTypeUtils.appendRowFields(physicalDataType, metadataFields); - - final TypeInformation producedTypeInfo = - context.createTypeInformation(producedDataType); - - return new DebeziumJsonDeserializationSchema( - physicalDataType, - readableMetadata, - producedTypeInfo, - schemaInclude, - ignoreParseErrors, - timestampFormat); - } - - @Override - public Map listReadableMetadata() { - final Map metadataMap = new LinkedHashMap<>(); - Stream.of(ReadableMetadata.values()).forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); - return metadataMap; - } - - @Override - public void applyReadableMetadata(List metadataKeys) { - this.metadataKeys = metadataKeys; - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - // -------------------------------------------------------------------------------------------- - // Metadata handling - // -------------------------------------------------------------------------------------------- - - /** - * List of metadata that can be read with this format. - */ - enum ReadableMetadata { - SCHEMA( - "schema", - DataTypes.STRING().nullable(), - false, - DataTypes.FIELD("schema", DataTypes.STRING()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - @Override - public Object convert(GenericRowData row, int pos) { - return row.getString(pos); - } - } - ), - - INGESTION_TIMESTAMP( - "ingestion-timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), - true, - DataTypes.FIELD("ts_ms", DataTypes.BIGINT()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - @Override - public Object convert(GenericRowData row, int pos) { - if (row.isNullAt(pos)) { - return null; - } - return TimestampData.fromEpochMillis(row.getLong(pos)); - } - } - ), - - SOURCE_TIMESTAMP( - "source.timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - @Override - public Object convert(GenericRowData row, int pos) { - final StringData timestamp = (StringData) readProperty(row, pos, KEY_SOURCE_TIMESTAMP); - if (timestamp == null) { - return null; - } - return TimestampData.fromEpochMillis(Long.parseLong(timestamp.toString())); - } - } - ), - - SOURCE_DATABASE( - "source.database", - DataTypes.STRING().nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - @Override - public Object convert(GenericRowData row, int pos) { - return readProperty(row, pos, KEY_SOURCE_DATABASE); - } - } - ), - - SOURCE_SCHEMA( - "source.schema", - DataTypes.STRING().nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - @Override - public Object convert(GenericRowData row, int pos) { - return readProperty(row, pos, KEY_SOURCE_SCHEMA); - } - } - ), - - SOURCE_TABLE( - "source.table", - DataTypes.STRING().nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - @Override - public Object convert(GenericRowData row, int pos) { - return readProperty(row, pos, KEY_SOURCE_TABLE); - } - } - ), - - SOURCE_PROPERTIES( - "source.properties", - // key and value of the map are nullable to make handling easier in queries - DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()).nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - @Override - public Object convert(GenericRowData row, int pos) { - return row.getMap(pos); - } - } - ); - - final String key; - - final DataType dataType; - - final boolean isJsonPayload; - - final DataTypes.Field requiredJsonField; - - final MetadataConverter converter; - - ReadableMetadata( - String key, - DataType dataType, - boolean isJsonPayload, - DataTypes.Field requiredJsonField, - MetadataConverter converter) { - this.key = key; - this.dataType = dataType; - this.isJsonPayload = isJsonPayload; - this.requiredJsonField = requiredJsonField; - this.converter = converter; - } - } - - private static final StringData KEY_SOURCE_TIMESTAMP = StringData.fromString("ts_ms"); - - private static final StringData KEY_SOURCE_DATABASE = StringData.fromString("db"); - - private static final StringData KEY_SOURCE_SCHEMA = StringData.fromString("schema"); - - private static final StringData KEY_SOURCE_TABLE = StringData.fromString("table"); - - private static Object readProperty(GenericRowData row, int pos, StringData key) { - final GenericMapData map = (GenericMapData) row.getMap(pos); - if (map == null) { - return null; - } - return map.get(key); - } + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + private List metadataKeys; + + // -------------------------------------------------------------------------------------------- + // Debezium-specific attributes + // -------------------------------------------------------------------------------------------- + + private final boolean schemaInclude; + + private final boolean ignoreParseErrors; + + private final TimestampFormat timestampFormat; + + public DebeziumJsonDecodingFormat( + boolean schemaInclude, boolean ignoreParseErrors, TimestampFormat timestampFormat) { + this.schemaInclude = schemaInclude; + this.ignoreParseErrors = ignoreParseErrors; + this.timestampFormat = timestampFormat; + this.metadataKeys = Collections.emptyList(); + } + + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType physicalDataType) { + + final List readableMetadata = + metadataKeys.stream() + .map( + k -> + Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .collect(Collectors.toList()); + + final List metadataFields = + readableMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList()); + + final DataType producedDataType = + DataTypeUtils.appendRowFields(physicalDataType, metadataFields); + + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + + return new DebeziumJsonDeserializationSchema( + physicalDataType, + readableMetadata, + producedTypeInfo, + schemaInclude, + ignoreParseErrors, + timestampFormat); + } + + @Override + public Map listReadableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + Stream.of(ReadableMetadata.values()) + .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyReadableMetadata(List metadataKeys) { + this.metadataKeys = metadataKeys; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + + /** List of metadata that can be read with this format. */ + enum ReadableMetadata { + SCHEMA( + "schema", + DataTypes.STRING().nullable(), + false, + DataTypes.FIELD("schema", DataTypes.STRING()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getString(pos); + } + }), + + INGESTION_TIMESTAMP( + "ingestion-timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + true, + DataTypes.FIELD("ts_ms", DataTypes.BIGINT()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + return TimestampData.fromEpochMillis(row.getLong(pos)); + } + }), + + SOURCE_TIMESTAMP( + "source.timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + final StringData timestamp = + (StringData) readProperty(row, pos, KEY_SOURCE_TIMESTAMP); + if (timestamp == null) { + return null; + } + return TimestampData.fromEpochMillis(Long.parseLong(timestamp.toString())); + } + }), + + SOURCE_DATABASE( + "source.database", + DataTypes.STRING().nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return readProperty(row, pos, KEY_SOURCE_DATABASE); + } + }), + + SOURCE_SCHEMA( + "source.schema", + DataTypes.STRING().nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return readProperty(row, pos, KEY_SOURCE_SCHEMA); + } + }), + + SOURCE_TABLE( + "source.table", + DataTypes.STRING().nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return readProperty(row, pos, KEY_SOURCE_TABLE); + } + }), + + SOURCE_PROPERTIES( + "source.properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) + .nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getMap(pos); + } + }); + + final String key; + + final DataType dataType; + + final boolean isJsonPayload; + + final DataTypes.Field requiredJsonField; + + final MetadataConverter converter; + + ReadableMetadata( + String key, + DataType dataType, + boolean isJsonPayload, + DataTypes.Field requiredJsonField, + MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.isJsonPayload = isJsonPayload; + this.requiredJsonField = requiredJsonField; + this.converter = converter; + } + } + + private static final StringData KEY_SOURCE_TIMESTAMP = StringData.fromString("ts_ms"); + + private static final StringData KEY_SOURCE_DATABASE = StringData.fromString("db"); + + private static final StringData KEY_SOURCE_SCHEMA = StringData.fromString("schema"); + + private static final StringData KEY_SOURCE_TABLE = StringData.fromString("table"); + + private static Object readProperty(GenericRowData row, int pos, StringData key) { + final GenericMapData map = (GenericMapData) row.getMap(pos); + if (map == null) { + return null; + } + return map.get(key); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java index 2fe5ee49d..628e05342 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java @@ -42,12 +42,11 @@ import static java.lang.String.format; /** - * Deserialization schema from Debezium JSON to Flink Table/SQL internal data structure {@link RowData}. - * The deserialization schema knows Debezium's schema definition and can extract the database data - * and convert into {@link RowData} with {@link RowKind}. + * Deserialization schema from Debezium JSON to Flink Table/SQL internal data structure {@link + * RowData}. The deserialization schema knows Debezium's schema definition and can extract the + * database data and convert into {@link RowData} with {@link RowKind}. * - *

Deserializes a byte[] message as a JSON object and reads - * the specified fields. + *

Deserializes a byte[] message as a JSON object and reads the specified fields. * *

Failures during deserialization are forwarded as wrapped IOExceptions. * @@ -55,279 +54,286 @@ */ @Internal public final class DebeziumJsonDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - - private static final String OP_READ = "r"; // snapshot read - private static final String OP_CREATE = "c"; // insert - private static final String OP_UPDATE = "u"; // update - private static final String OP_DELETE = "d"; // delete - - private static final String REPLICA_IDENTITY_EXCEPTION = "The \"before\" field of %s message is null, " + - "if you are using Debezium Postgres Connector, " + - "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; - - /** The deserializer to deserialize Debezium JSON data. */ - private final JsonRowDataDeserializationSchema jsonDeserializer; - - /** Flag that indicates that an additional projection is required for metadata. */ - private final boolean hasMetadata; - - /** Metadata to be extracted for every record. */ - private final MetadataConverter[] metadataConverters; - - /** {@link TypeInformation} of the produced {@link RowData} (physical + meta data). */ - private final TypeInformation producedTypeInfo; - - /** - * Flag indicating whether the Debezium JSON data contains schema part or not. - * When Debezium Kafka Connect enables "value.converter.schemas.enable", the JSON - * will contain "schema" information, but we just ignore "schema" and extract data - * from "payload". - */ - private final boolean schemaInclude; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - public DebeziumJsonDeserializationSchema( - DataType physicalDataType, - List requestedMetadata, - TypeInformation producedTypeInfo, - boolean schemaInclude, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata, schemaInclude); - this.jsonDeserializer = new JsonRowDataDeserializationSchema( - jsonRowType, - // the result type is never used, so it's fine to pass in the produced type info - producedTypeInfo, - false, // ignoreParseErrors already contains the functionality of failOnMissingField - ignoreParseErrors, - timestampFormat); - this.hasMetadata = requestedMetadata.size() > 0; - this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata, schemaInclude); - this.producedTypeInfo = producedTypeInfo; - this.schemaInclude = schemaInclude; - this.ignoreParseErrors = ignoreParseErrors; - } - - @Override - public RowData deserialize(byte[] message) { - throw new RuntimeException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); - } - - @Override - public void deserialize(byte[] message, Collector out) throws IOException { - if (message == null || message.length == 0) { - // skip tombstone messages - return; - } - try { - GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); - GenericRowData payload; - if (schemaInclude) { - payload = (GenericRowData) row.getField(0); - } else { - payload = row; - } - - GenericRowData before = (GenericRowData) payload.getField(0); - GenericRowData after = (GenericRowData) payload.getField(1); - String op = payload.getField(2).toString(); - if (OP_CREATE.equals(op) || OP_READ.equals(op)) { - after.setRowKind(RowKind.INSERT); - emitRow(row, after, out); - } else if (OP_UPDATE.equals(op)) { - if (before == null) { - throw new IllegalStateException(String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - before.setRowKind(RowKind.UPDATE_BEFORE); - after.setRowKind(RowKind.UPDATE_AFTER); - emitRow(row, before, out); - emitRow(row, after, out); - } else if (OP_DELETE.equals(op)) { - if (before == null) { - throw new IllegalStateException(String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); - } - before.setRowKind(RowKind.DELETE); - emitRow(row, before, out); - } else { - if (!ignoreParseErrors) { - throw new IOException(format( - "Unknown \"op\" value \"%s\". The Debezium JSON message is '%s'", op, new String(message))); - } - } - } catch (Throwable t) { - // a big try catch to protect the processing. - if (!ignoreParseErrors) { - throw new IOException(format( - "Corrupt Debezium JSON message '%s'.", new String(message)), t); - } - } - } - - private void emitRow(GenericRowData rootRow, GenericRowData physicalRow, Collector out) { - // shortcut in case no output projection is required - if (!hasMetadata) { - out.collect(physicalRow); - return; - } - - final int physicalArity = physicalRow.getArity(); - final int metadataArity = metadataConverters.length; - - final GenericRowData producedRow = new GenericRowData( - physicalRow.getRowKind(), - physicalArity + metadataArity); - - for (int physicalPos = 0; physicalPos < physicalArity; physicalPos++) { - producedRow.setField(physicalPos, physicalRow.getField(physicalPos)); - } - - for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { - producedRow.setField(physicalArity + metadataPos, metadataConverters[metadataPos].convert(rootRow)); - } - - out.collect(producedRow); - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return producedTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DebeziumJsonDeserializationSchema that = (DebeziumJsonDeserializationSchema) o; - return Objects.equals(jsonDeserializer, that.jsonDeserializer) - && hasMetadata == that.hasMetadata - && Objects.equals(producedTypeInfo, that.producedTypeInfo) - && schemaInclude == that.schemaInclude - && ignoreParseErrors == that.ignoreParseErrors; - } - - @Override - public int hashCode() { - return Objects.hash( - jsonDeserializer, - hasMetadata, - producedTypeInfo, - schemaInclude, - ignoreParseErrors); - } - - // -------------------------------------------------------------------------------------------- - - private static RowType createJsonRowType( - DataType physicalDataType, - List readableMetadata, - boolean schemaInclude) { - DataType payload = DataTypes.ROW( - DataTypes.FIELD("before", physicalDataType), - DataTypes.FIELD("after", physicalDataType), - DataTypes.FIELD("op", DataTypes.STRING())); - - // append fields that are required for reading metadata in the payload - final List payloadMetadataFields = readableMetadata.stream() - .filter(m -> m.isJsonPayload) - .map(m -> m.requiredJsonField) - .distinct() - .collect(Collectors.toList()); - payload = DataTypeUtils.appendRowFields(payload, payloadMetadataFields); - - DataType root = payload; - if (schemaInclude) { - // when Debezium Kafka connect enables "value.converter.schemas.enable", - // the JSON will contain "schema" information and we need to extract data from "payload". - root = DataTypes.ROW(DataTypes.FIELD("payload", payload)); - } - - // append fields that are required for reading metadata in the root - final List rootMetadataFields = readableMetadata.stream() - .filter(m -> !m.isJsonPayload) - .map(m -> m.requiredJsonField) - .distinct() - .collect(Collectors.toList()); - root = DataTypeUtils.appendRowFields(root, rootMetadataFields); - - return (RowType) root.getLogicalType(); - } - - private static MetadataConverter[] createMetadataConverters( - RowType jsonRowType, - List requestedMetadata, - boolean schemaInclude) { - return requestedMetadata.stream() - .map(m -> { - if (m.isJsonPayload) { - return convertInPayload(jsonRowType, m, schemaInclude); - } else { - return convertInRoot(jsonRowType, m); - } - }) - .toArray(MetadataConverter[]::new); - } - - private static MetadataConverter convertInRoot( - RowType jsonRowType, - ReadableMetadata metadata) { - final int pos = findFieldPos(metadata, jsonRowType); - return new MetadataConverter() { - private static final long serialVersionUID = 1L; - @Override - public Object convert(GenericRowData root, int unused) { - return metadata.converter.convert(root, pos); - } - }; - } - - private static MetadataConverter convertInPayload( - RowType jsonRowType, - ReadableMetadata metadata, - boolean schemaInclude) { - if (schemaInclude) { - final int pos = findFieldPos(metadata, (RowType) jsonRowType.getChildren().get(0)); - return new MetadataConverter() { - private static final long serialVersionUID = 1L; - @Override - public Object convert(GenericRowData root, int unused) { - final GenericRowData payload = (GenericRowData) root.getField(0); - return metadata.converter.convert(payload, pos); - } - }; - } - return convertInRoot(jsonRowType, metadata); - } - - private static int findFieldPos(ReadableMetadata metadata, RowType jsonRowType) { - return jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Converter that extracts a metadata field from the row (root or payload) that comes out of the - * JSON schema and converts it to the desired data type. - */ - interface MetadataConverter extends Serializable { - - // Method for top-level access. - default Object convert(GenericRowData row) { - return convert(row, -1); - } - - Object convert(GenericRowData row, int pos); - } + private static final long serialVersionUID = 1L; + + private static final String OP_READ = "r"; // snapshot read + private static final String OP_CREATE = "c"; // insert + private static final String OP_UPDATE = "u"; // update + private static final String OP_DELETE = "d"; // delete + + private static final String REPLICA_IDENTITY_EXCEPTION = + "The \"before\" field of %s message is null, " + + "if you are using Debezium Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; + + /** The deserializer to deserialize Debezium JSON data. */ + private final JsonRowDataDeserializationSchema jsonDeserializer; + + /** Flag that indicates that an additional projection is required for metadata. */ + private final boolean hasMetadata; + + /** Metadata to be extracted for every record. */ + private final MetadataConverter[] metadataConverters; + + /** {@link TypeInformation} of the produced {@link RowData} (physical + meta data). */ + private final TypeInformation producedTypeInfo; + + /** + * Flag indicating whether the Debezium JSON data contains schema part or not. When Debezium + * Kafka Connect enables "value.converter.schemas.enable", the JSON will contain "schema" + * information, but we just ignore "schema" and extract data from "payload". + */ + private final boolean schemaInclude; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + public DebeziumJsonDeserializationSchema( + DataType physicalDataType, + List requestedMetadata, + TypeInformation producedTypeInfo, + boolean schemaInclude, + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { + final RowType jsonRowType = + createJsonRowType(physicalDataType, requestedMetadata, schemaInclude); + this.jsonDeserializer = + new JsonRowDataDeserializationSchema( + jsonRowType, + // the result type is never used, so it's fine to pass in the produced type + // info + producedTypeInfo, + false, // ignoreParseErrors already contains the functionality of + // failOnMissingField + ignoreParseErrors, + timestampFormat); + this.hasMetadata = requestedMetadata.size() > 0; + this.metadataConverters = + createMetadataConverters(jsonRowType, requestedMetadata, schemaInclude); + this.producedTypeInfo = producedTypeInfo; + this.schemaInclude = schemaInclude; + this.ignoreParseErrors = ignoreParseErrors; + } + + @Override + public RowData deserialize(byte[] message) { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + + @Override + public void deserialize(byte[] message, Collector out) throws IOException { + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } + try { + GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); + GenericRowData payload; + if (schemaInclude) { + payload = (GenericRowData) row.getField(0); + } else { + payload = row; + } + + GenericRowData before = (GenericRowData) payload.getField(0); + GenericRowData after = (GenericRowData) payload.getField(1); + String op = payload.getField(2).toString(); + if (OP_CREATE.equals(op) || OP_READ.equals(op)) { + after.setRowKind(RowKind.INSERT); + emitRow(row, after, out); + } else if (OP_UPDATE.equals(op)) { + if (before == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + before.setRowKind(RowKind.UPDATE_BEFORE); + after.setRowKind(RowKind.UPDATE_AFTER); + emitRow(row, before, out); + emitRow(row, after, out); + } else if (OP_DELETE.equals(op)) { + if (before == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); + } + before.setRowKind(RowKind.DELETE); + emitRow(row, before, out); + } else { + if (!ignoreParseErrors) { + throw new IOException( + format( + "Unknown \"op\" value \"%s\". The Debezium JSON message is '%s'", + op, new String(message))); + } + } + } catch (Throwable t) { + // a big try catch to protect the processing. + if (!ignoreParseErrors) { + throw new IOException( + format("Corrupt Debezium JSON message '%s'.", new String(message)), t); + } + } + } + + private void emitRow( + GenericRowData rootRow, GenericRowData physicalRow, Collector out) { + // shortcut in case no output projection is required + if (!hasMetadata) { + out.collect(physicalRow); + return; + } + + final int physicalArity = physicalRow.getArity(); + final int metadataArity = metadataConverters.length; + + final GenericRowData producedRow = + new GenericRowData(physicalRow.getRowKind(), physicalArity + metadataArity); + + for (int physicalPos = 0; physicalPos < physicalArity; physicalPos++) { + producedRow.setField(physicalPos, physicalRow.getField(physicalPos)); + } + + for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { + producedRow.setField( + physicalArity + metadataPos, metadataConverters[metadataPos].convert(rootRow)); + } + + out.collect(producedRow); + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return producedTypeInfo; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DebeziumJsonDeserializationSchema that = (DebeziumJsonDeserializationSchema) o; + return Objects.equals(jsonDeserializer, that.jsonDeserializer) + && hasMetadata == that.hasMetadata + && Objects.equals(producedTypeInfo, that.producedTypeInfo) + && schemaInclude == that.schemaInclude + && ignoreParseErrors == that.ignoreParseErrors; + } + + @Override + public int hashCode() { + return Objects.hash( + jsonDeserializer, hasMetadata, producedTypeInfo, schemaInclude, ignoreParseErrors); + } + + // -------------------------------------------------------------------------------------------- + + private static RowType createJsonRowType( + DataType physicalDataType, + List readableMetadata, + boolean schemaInclude) { + DataType payload = + DataTypes.ROW( + DataTypes.FIELD("before", physicalDataType), + DataTypes.FIELD("after", physicalDataType), + DataTypes.FIELD("op", DataTypes.STRING())); + + // append fields that are required for reading metadata in the payload + final List payloadMetadataFields = + readableMetadata.stream() + .filter(m -> m.isJsonPayload) + .map(m -> m.requiredJsonField) + .distinct() + .collect(Collectors.toList()); + payload = DataTypeUtils.appendRowFields(payload, payloadMetadataFields); + + DataType root = payload; + if (schemaInclude) { + // when Debezium Kafka connect enables "value.converter.schemas.enable", + // the JSON will contain "schema" information and we need to extract data from + // "payload". + root = DataTypes.ROW(DataTypes.FIELD("payload", payload)); + } + + // append fields that are required for reading metadata in the root + final List rootMetadataFields = + readableMetadata.stream() + .filter(m -> !m.isJsonPayload) + .map(m -> m.requiredJsonField) + .distinct() + .collect(Collectors.toList()); + root = DataTypeUtils.appendRowFields(root, rootMetadataFields); + + return (RowType) root.getLogicalType(); + } + + private static MetadataConverter[] createMetadataConverters( + RowType jsonRowType, List requestedMetadata, boolean schemaInclude) { + return requestedMetadata.stream() + .map( + m -> { + if (m.isJsonPayload) { + return convertInPayload(jsonRowType, m, schemaInclude); + } else { + return convertInRoot(jsonRowType, m); + } + }) + .toArray(MetadataConverter[]::new); + } + + private static MetadataConverter convertInRoot(RowType jsonRowType, ReadableMetadata metadata) { + final int pos = findFieldPos(metadata, jsonRowType); + return new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData root, int unused) { + return metadata.converter.convert(root, pos); + } + }; + } + + private static MetadataConverter convertInPayload( + RowType jsonRowType, ReadableMetadata metadata, boolean schemaInclude) { + if (schemaInclude) { + final int pos = findFieldPos(metadata, (RowType) jsonRowType.getChildren().get(0)); + return new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData root, int unused) { + final GenericRowData payload = (GenericRowData) root.getField(0); + return metadata.converter.convert(payload, pos); + } + }; + } + return convertInRoot(jsonRowType, metadata); + } + + private static int findFieldPos(ReadableMetadata metadata, RowType jsonRowType) { + return jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); + } + + // -------------------------------------------------------------------------------------------- + + /** + * Converter that extracts a metadata field from the row (root or payload) that comes out of the + * JSON schema and converts it to the desired data type. + */ + interface MetadataConverter extends Serializable { + + // Method for top-level access. + default Object convert(GenericRowData row) { + return convert(row, -1); + } + + Object convert(GenericRowData row, int pos); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java index c2458176d..79934a3b1 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -50,79 +50,81 @@ import static org.apache.flink.formats.json.debezium.DebeziumJsonOptions.validateEncodingFormatOptions; /** - * Format factory for providing configured instances of Debezium JSON to RowData {@link DeserializationSchema}. + * Format factory for providing configured instances of Debezium JSON to RowData {@link + * DeserializationSchema}. */ -public class DebeziumJsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "debezium-json"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateDecodingFormatOptions(formatOptions); - - final boolean schemaInclude = formatOptions.get(SCHEMA_INCLUDE); - - final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - - final TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - - return new DebeziumJsonDecodingFormat(schemaInclude, ignoreParseErrors, timestampFormat); - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateEncodingFormatOptions(formatOptions); - - TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); - String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); - - return new EncodingFormat>() { - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - @Override - public SerializationSchema createRuntimeEncoder(DynamicTableSink.Context context, DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new DebeziumJsonSerializationSchema(rowType, timestampFormat, mapNullKeyMode, mapNullKeyLiteral); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return Collections.emptySet(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(SCHEMA_INCLUDE); - options.add(IGNORE_PARSE_ERRORS); - options.add(TIMESTAMP_FORMAT); - options.add(JSON_MAP_NULL_KEY_MODE); - options.add(JSON_MAP_NULL_KEY_LITERAL); - return options; - } +public class DebeziumJsonFormatFactory + implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "debezium-json"; + + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateDecodingFormatOptions(formatOptions); + + final boolean schemaInclude = formatOptions.get(SCHEMA_INCLUDE); + + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + + final TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + + return new DebeziumJsonDecodingFormat(schemaInclude, ignoreParseErrors, timestampFormat); + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateEncodingFormatOptions(formatOptions); + + TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); + + return new EncodingFormat>() { + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new DebeziumJsonSerializationSchema( + rowType, timestampFormat, mapNullKeyMode, mapNullKeyLiteral); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(SCHEMA_INCLUDE); + options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); + options.add(JSON_MAP_NULL_KEY_MODE); + options.add(JSON_MAP_NULL_KEY_LITERAL); + return options; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java index 7cafd69cf..fa1c1e1bb 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java @@ -26,52 +26,47 @@ import static org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory.IDENTIFIER; -/** - * Option utils for debezium-json format. - */ +/** Option utils for debezium-json format. */ public class DebeziumJsonOptions { - public static final ConfigOption SCHEMA_INCLUDE = ConfigOptions - .key("schema-include") - .booleanType() - .defaultValue(false) - .withDescription("When setting up a Debezium Kafka Connect, users can enable " + - "a Kafka configuration 'value.converter.schemas.enable' to include schema in the message. " + - "This option indicates the Debezium JSON data include the schema in the message or not. " + - "Default is false."); + public static final ConfigOption SCHEMA_INCLUDE = + ConfigOptions.key("schema-include") + .booleanType() + .defaultValue(false) + .withDescription( + "When setting up a Debezium Kafka Connect, users can enable " + + "a Kafka configuration 'value.converter.schemas.enable' to include schema in the message. " + + "This option indicates the Debezium JSON data include the schema in the message or not. " + + "Default is false."); - public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; - public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; - public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; + public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; - public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = JsonOptions.MAP_NULL_KEY_LITERAL; + public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = + JsonOptions.MAP_NULL_KEY_LITERAL; - // -------------------------------------------------------------------------------------------- - // Validation - // -------------------------------------------------------------------------------------------- + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- - /** - * Validator for debezium decoding format. - */ - public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateDecodingFormatOptions(tableOptions); - } + /** Validator for debezium decoding format. */ + public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateDecodingFormatOptions(tableOptions); + } - /** - * Validator for debezium encoding format. - */ - public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateEncodingFormatOptions(tableOptions); + /** Validator for debezium encoding format. */ + public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateEncodingFormatOptions(tableOptions); - // validator for {@link SCHEMA_INCLUDE} - if (tableOptions.get(SCHEMA_INCLUDE)) { - throw new ValidationException(String.format( - "Debezium JSON serialization doesn't support '%s.%s' option been set to true.", - IDENTIFIER, - SCHEMA_INCLUDE.key() - )); - } - } + // validator for {@link SCHEMA_INCLUDE} + if (tableOptions.get(SCHEMA_INCLUDE)) { + throw new ValidationException( + String.format( + "Debezium JSON serialization doesn't support '%s.%s' option been set to true.", + IDENTIFIER, SCHEMA_INCLUDE.key())); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java index d3defc008..4f71a09b5 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java @@ -35,85 +35,92 @@ import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** - * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium JSON. + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium + * JSON. * * @see Debezium */ public class DebeziumJsonSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = 1L; + private static final long serialVersionUID = 1L; - private static final StringData OP_INSERT = StringData.fromString("c"); // insert - private static final StringData OP_DELETE = StringData.fromString("d"); // delete + private static final StringData OP_INSERT = StringData.fromString("c"); // insert + private static final StringData OP_DELETE = StringData.fromString("d"); // delete - /** The serializer to serialize Debezium JSON data. **/ - private final JsonRowDataSerializationSchema jsonSerializer; + /** The serializer to serialize Debezium JSON data. * */ + private final JsonRowDataSerializationSchema jsonSerializer; - private transient GenericRowData genericRowData; + private transient GenericRowData genericRowData; - public DebeziumJsonSerializationSchema( - RowType rowType, - TimestampFormat timestampFormat, - JsonOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral) { - jsonSerializer = new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral); - } + public DebeziumJsonSerializationSchema( + RowType rowType, + TimestampFormat timestampFormat, + JsonOptions.MapNullKeyMode mapNullKeyMode, + String mapNullKeyLiteral) { + jsonSerializer = + new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral); + } - @Override - public void open(InitializationContext context) { - genericRowData = new GenericRowData(3); - } + @Override + public void open(InitializationContext context) { + genericRowData = new GenericRowData(3); + } - @Override - public byte[] serialize(RowData rowData) { - try { - switch (rowData.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - genericRowData.setField(0, null); - genericRowData.setField(1, rowData); - genericRowData.setField(2, OP_INSERT); - return jsonSerializer.serialize(genericRowData); - case UPDATE_BEFORE: - case DELETE: - genericRowData.setField(0, rowData); - genericRowData.setField(1, null); - genericRowData.setField(2, OP_DELETE); - return jsonSerializer.serialize(genericRowData); - default: - throw new UnsupportedOperationException(format("Unsupported operation '%s' for row kind.", rowData.getRowKind())); - } - } catch (Throwable t) { - throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t); - } - } + @Override + public byte[] serialize(RowData rowData) { + try { + switch (rowData.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + genericRowData.setField(0, null); + genericRowData.setField(1, rowData); + genericRowData.setField(2, OP_INSERT); + return jsonSerializer.serialize(genericRowData); + case UPDATE_BEFORE: + case DELETE: + genericRowData.setField(0, rowData); + genericRowData.setField(1, null); + genericRowData.setField(2, OP_DELETE); + return jsonSerializer.serialize(genericRowData); + default: + throw new UnsupportedOperationException( + format( + "Unsupported operation '%s' for row kind.", + rowData.getRowKind())); + } + } catch (Throwable t) { + throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t); + } + } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DebeziumJsonSerializationSchema that = (DebeziumJsonSerializationSchema) o; - return Objects.equals(jsonSerializer, that.jsonSerializer); - } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DebeziumJsonSerializationSchema that = (DebeziumJsonSerializationSchema) o; + return Objects.equals(jsonSerializer, that.jsonSerializer); + } - @Override - public int hashCode() { - return Objects.hash(jsonSerializer); - } + @Override + public int hashCode() { + return Objects.hash(jsonSerializer); + } - private static RowType createJsonRowType(DataType databaseSchema) { - // Debezium JSON contains some other information, e.g. "source", "ts_ms" - // but we don't need them. - return (RowType) DataTypes.ROW( - DataTypes.FIELD("before", databaseSchema), - DataTypes.FIELD("after", databaseSchema), - DataTypes.FIELD("op", DataTypes.STRING())).getLogicalType(); - } + private static RowType createJsonRowType(DataType databaseSchema) { + // Debezium JSON contains some other information, e.g. "source", "ts_ms" + // but we don't need them. + return (RowType) + DataTypes.ROW( + DataTypes.FIELD("before", databaseSchema), + DataTypes.FIELD("after", databaseSchema), + DataTypes.FIELD("op", DataTypes.STRING())) + .getLogicalType(); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java index c34ab5597..073870895 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java @@ -37,143 +37,150 @@ import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** - * Deserialization schema from Maxwell JSON to Flink Table/SQL internal data structure {@link RowData}. - * The deserialization schema knows Maxwell's schema definition and can extract the database data - * and convert into {@link RowData} with {@link RowKind}. + * Deserialization schema from Maxwell JSON to Flink Table/SQL internal data structure {@link + * RowData}. The deserialization schema knows Maxwell's schema definition and can extract the + * database data and convert into {@link RowData} with {@link RowKind}. * - *

Deserializes a byte[] message as a JSON object and reads - * the specified fields. + *

Deserializes a byte[] message as a JSON object and reads the specified fields. * *

Failures during deserialization are forwarded as wrapped IOExceptions. * * @see Maxwell */ public class MaxwellJsonDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - - private static final String OP_INSERT = "insert"; - private static final String OP_UPDATE = "update"; - private static final String OP_DELETE = "delete"; - - /** The deserializer to deserialize Maxwell JSON data. */ - private final JsonRowDataDeserializationSchema jsonDeserializer; - - /** TypeInformation of the produced {@link RowData}. **/ - private final TypeInformation resultTypeInfo; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - /** Number of fields. */ - private final int fieldCount; - - public MaxwellJsonDeserializationSchema(RowType rowType, - TypeInformation resultTypeInfo, - boolean ignoreParseErrors, - TimestampFormat timestampFormatOption) { - this.resultTypeInfo = resultTypeInfo; - this.ignoreParseErrors = ignoreParseErrors; - this.fieldCount = rowType.getFieldCount(); - this.jsonDeserializer = new JsonRowDataDeserializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - // the result type is never used, so it's fine to pass in Canal's result type - resultTypeInfo, - false, // ignoreParseErrors already contains the functionality of failOnMissingField - ignoreParseErrors, - timestampFormatOption); - } - - @Override - public RowData deserialize(byte[] message) throws IOException { - throw new RuntimeException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); - } - - @Override - public void deserialize(byte[] message, Collector out) throws IOException { - try { - RowData row = jsonDeserializer.deserialize(message); - String type = row.getString(2).toString(); // "type" field - if (OP_INSERT.equals(type)) { - // "data" field is a row, contains inserted rows - RowData insert = row.getRow(0, fieldCount); - insert.setRowKind(RowKind.INSERT); - out.collect(insert); - } else if (OP_UPDATE.equals(type)) { - // "data" field is a row, contains new rows - // "old" field is an array of row, contains old values - // the underlying JSON deserialization schema always produce GenericRowData. - GenericRowData after = (GenericRowData) row.getRow(0, fieldCount); // "data" field - GenericRowData before = (GenericRowData) row.getRow(1, fieldCount); // "old" field - for (int f = 0; f < fieldCount; f++) { - if (before.isNullAt(f)) { - // not null fields in "old" (before) means the fields are changed - // null/empty fields in "old" (before) means the fields are not changed - // so we just copy the not changed fields into before - before.setField(f, after.getField(f)); - } - } - before.setRowKind(RowKind.UPDATE_BEFORE); - after.setRowKind(RowKind.UPDATE_AFTER); - out.collect(before); - out.collect(after); - } else if (OP_DELETE.equals(type)) { - // "data" field is a row, contains deleted rows - RowData delete = row.getRow(0, fieldCount); - delete.setRowKind(RowKind.DELETE); - out.collect(delete); - - } else { - if (!ignoreParseErrors) { - throw new IOException(format( - "Unknown \"type\" value \"%s\". The Maxwell JSON message is '%s'", type, new String(message))); - } - } - } catch (Throwable t) { - // a big try catch to protect the processing. - if (!ignoreParseErrors) { - throw new IOException(format( - "Corrupt Maxwell JSON message '%s'.", new String(message)), t); - } - } - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return resultTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - MaxwellJsonDeserializationSchema that = (MaxwellJsonDeserializationSchema) o; - return ignoreParseErrors == that.ignoreParseErrors && - fieldCount == that.fieldCount && - Objects.equals(jsonDeserializer, that.jsonDeserializer) && - Objects.equals(resultTypeInfo, that.resultTypeInfo); - } - - @Override - public int hashCode() { - return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors, fieldCount); - } - - private RowType createJsonRowType(DataType databaseSchema) { - // Maxwell JSON contains other information, e.g. "database", "ts" - // but we don't need them - return (RowType) DataTypes.ROW( - DataTypes.FIELD("data", databaseSchema), - DataTypes.FIELD("old", databaseSchema), - DataTypes.FIELD("type", DataTypes.STRING())).getLogicalType(); - } + private static final long serialVersionUID = 1L; + + private static final String OP_INSERT = "insert"; + private static final String OP_UPDATE = "update"; + private static final String OP_DELETE = "delete"; + + /** The deserializer to deserialize Maxwell JSON data. */ + private final JsonRowDataDeserializationSchema jsonDeserializer; + + /** TypeInformation of the produced {@link RowData}. * */ + private final TypeInformation resultTypeInfo; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + /** Number of fields. */ + private final int fieldCount; + + public MaxwellJsonDeserializationSchema( + RowType rowType, + TypeInformation resultTypeInfo, + boolean ignoreParseErrors, + TimestampFormat timestampFormatOption) { + this.resultTypeInfo = resultTypeInfo; + this.ignoreParseErrors = ignoreParseErrors; + this.fieldCount = rowType.getFieldCount(); + this.jsonDeserializer = + new JsonRowDataDeserializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + // the result type is never used, so it's fine to pass in Canal's result + // type + resultTypeInfo, + false, // ignoreParseErrors already contains the functionality of + // failOnMissingField + ignoreParseErrors, + timestampFormatOption); + } + + @Override + public RowData deserialize(byte[] message) throws IOException { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + + @Override + public void deserialize(byte[] message, Collector out) throws IOException { + try { + RowData row = jsonDeserializer.deserialize(message); + String type = row.getString(2).toString(); // "type" field + if (OP_INSERT.equals(type)) { + // "data" field is a row, contains inserted rows + RowData insert = row.getRow(0, fieldCount); + insert.setRowKind(RowKind.INSERT); + out.collect(insert); + } else if (OP_UPDATE.equals(type)) { + // "data" field is a row, contains new rows + // "old" field is an array of row, contains old values + // the underlying JSON deserialization schema always produce GenericRowData. + GenericRowData after = (GenericRowData) row.getRow(0, fieldCount); // "data" field + GenericRowData before = (GenericRowData) row.getRow(1, fieldCount); // "old" field + for (int f = 0; f < fieldCount; f++) { + if (before.isNullAt(f)) { + // not null fields in "old" (before) means the fields are changed + // null/empty fields in "old" (before) means the fields are not changed + // so we just copy the not changed fields into before + before.setField(f, after.getField(f)); + } + } + before.setRowKind(RowKind.UPDATE_BEFORE); + after.setRowKind(RowKind.UPDATE_AFTER); + out.collect(before); + out.collect(after); + } else if (OP_DELETE.equals(type)) { + // "data" field is a row, contains deleted rows + RowData delete = row.getRow(0, fieldCount); + delete.setRowKind(RowKind.DELETE); + out.collect(delete); + + } else { + if (!ignoreParseErrors) { + throw new IOException( + format( + "Unknown \"type\" value \"%s\". The Maxwell JSON message is '%s'", + type, new String(message))); + } + } + } catch (Throwable t) { + // a big try catch to protect the processing. + if (!ignoreParseErrors) { + throw new IOException( + format("Corrupt Maxwell JSON message '%s'.", new String(message)), t); + } + } + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return resultTypeInfo; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MaxwellJsonDeserializationSchema that = (MaxwellJsonDeserializationSchema) o; + return ignoreParseErrors == that.ignoreParseErrors + && fieldCount == that.fieldCount + && Objects.equals(jsonDeserializer, that.jsonDeserializer) + && Objects.equals(resultTypeInfo, that.resultTypeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors, fieldCount); + } + + private RowType createJsonRowType(DataType databaseSchema) { + // Maxwell JSON contains other information, e.g. "database", "ts" + // but we don't need them + return (RowType) + DataTypes.ROW( + DataTypes.FIELD("data", databaseSchema), + DataTypes.FIELD("old", databaseSchema), + DataTypes.FIELD("type", DataTypes.STRING())) + .getLogicalType(); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java index 4f37723d5..0a57cbe43 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java @@ -51,102 +51,95 @@ import static org.apache.flink.formats.json.maxwell.MaxwellJsonOptions.validateEncodingFormatOptions; /** - * Format factory for providing configured instances of Maxwell JSON to RowData {@link DeserializationSchema}. + * Format factory for providing configured instances of Maxwell JSON to RowData {@link + * DeserializationSchema}. */ -public class MaxwellJsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "maxwell-json"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateDecodingFormatOptions(formatOptions); - - final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); - - return new DecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType producedDataType) { - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation rowDataTypeInfo = - context.createTypeInformation(producedDataType); - return new MaxwellJsonDeserializationSchema( - rowType, - rowDataTypeInfo, - ignoreParseErrors, - timestampFormatOption); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - }; - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, - ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateEncodingFormatOptions(formatOptions); - - TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); - String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); - - return new EncodingFormat>() { - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new MaxwellJsonSerializationSchema( - rowType, - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral - ); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return Collections.emptySet(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(IGNORE_PARSE_ERRORS); - options.add(TIMESTAMP_FORMAT); - options.add(JSON_MAP_NULL_KEY_MODE); - options.add(JSON_MAP_NULL_KEY_LITERAL); - return options; - } +public class MaxwellJsonFormatFactory + implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "maxwell-json"; + + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateDecodingFormatOptions(formatOptions); + + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); + + return new DecodingFormat>() { + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation rowDataTypeInfo = + context.createTypeInformation(producedDataType); + return new MaxwellJsonDeserializationSchema( + rowType, rowDataTypeInfo, ignoreParseErrors, timestampFormatOption); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + }; + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateEncodingFormatOptions(formatOptions); + + TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); + + return new EncodingFormat>() { + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new MaxwellJsonSerializationSchema( + rowType, timestampFormat, mapNullKeyMode, mapNullKeyLiteral); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); + options.add(JSON_MAP_NULL_KEY_MODE); + options.add(JSON_MAP_NULL_KEY_LITERAL); + return options; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java index 1e68c5479..6d66c28e7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java @@ -22,34 +22,29 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.formats.json.JsonOptions; -/** - * Option utils for maxwell-json format. - */ +/** Option utils for maxwell-json format. */ public class MaxwellJsonOptions { - public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; - public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; - public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; + public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; - public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = JsonOptions.MAP_NULL_KEY_LITERAL; + public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = + JsonOptions.MAP_NULL_KEY_LITERAL; - // -------------------------------------------------------------------------------------------- - // Validation - // -------------------------------------------------------------------------------------------- + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- - /** - * Validator for maxwell decoding format. - */ - public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateDecodingFormatOptions(tableOptions); - } + /** Validator for maxwell decoding format. */ + public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateDecodingFormatOptions(tableOptions); + } - /** - * Validator for maxwell encoding format. - */ - public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateEncodingFormatOptions(tableOptions); - } + /** Validator for maxwell encoding format. */ + public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + JsonOptions.validateEncodingFormatOptions(tableOptions); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java index 2774f106a..36c52ae6f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java @@ -35,84 +35,85 @@ import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** - * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to maxwell-Json. + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to + * maxwell-Json. */ public class MaxwellJsonSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = 1L; - - private static final StringData OP_INSERT = StringData.fromString("insert"); - private static final StringData OP_DELETE = StringData.fromString("delete"); - - private final JsonRowDataSerializationSchema jsonSerializer; - - /** - * Timestamp format specification which is used to parse timestamp. - */ - private final TimestampFormat timestampFormat; - - private transient GenericRowData reuse; - - public MaxwellJsonSerializationSchema( - RowType rowType, - TimestampFormat timestampFormat, - JsonOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral) { - this.jsonSerializer = new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral); - this.timestampFormat = timestampFormat; - } - - @Override - public void open(InitializationContext context) throws Exception { - this.reuse = new GenericRowData(2); - } - - @Override - public byte[] serialize(RowData element) { - reuse.setField(0, element); - reuse.setField(1, rowKind2String(element.getRowKind())); - return jsonSerializer.serialize(reuse); - } - - private StringData rowKind2String(RowKind rowKind) { - switch (rowKind) { - case INSERT: - case UPDATE_AFTER: - return OP_INSERT; - case UPDATE_BEFORE: - case DELETE: - return OP_DELETE; - default: - throw new UnsupportedOperationException("Unsupported operation '" + rowKind + "' for row kind."); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - MaxwellJsonSerializationSchema that = (MaxwellJsonSerializationSchema) o; - return Objects.equals(jsonSerializer, that.jsonSerializer) && - timestampFormat == that.timestampFormat; - } - - @Override - public int hashCode() { - return Objects.hash(jsonSerializer, timestampFormat); - } - - private RowType createJsonRowType(DataType databaseSchema) { - DataType payload = DataTypes.ROW( - DataTypes.FIELD("data", databaseSchema), - DataTypes.FIELD("type", DataTypes.STRING())); - return (RowType) payload.getLogicalType(); - } - + private static final long serialVersionUID = 1L; + + private static final StringData OP_INSERT = StringData.fromString("insert"); + private static final StringData OP_DELETE = StringData.fromString("delete"); + + private final JsonRowDataSerializationSchema jsonSerializer; + + /** Timestamp format specification which is used to parse timestamp. */ + private final TimestampFormat timestampFormat; + + private transient GenericRowData reuse; + + public MaxwellJsonSerializationSchema( + RowType rowType, + TimestampFormat timestampFormat, + JsonOptions.MapNullKeyMode mapNullKeyMode, + String mapNullKeyLiteral) { + this.jsonSerializer = + new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral); + this.timestampFormat = timestampFormat; + } + + @Override + public void open(InitializationContext context) throws Exception { + this.reuse = new GenericRowData(2); + } + + @Override + public byte[] serialize(RowData element) { + reuse.setField(0, element); + reuse.setField(1, rowKind2String(element.getRowKind())); + return jsonSerializer.serialize(reuse); + } + + private StringData rowKind2String(RowKind rowKind) { + switch (rowKind) { + case INSERT: + case UPDATE_AFTER: + return OP_INSERT; + case UPDATE_BEFORE: + case DELETE: + return OP_DELETE; + default: + throw new UnsupportedOperationException( + "Unsupported operation '" + rowKind + "' for row kind."); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MaxwellJsonSerializationSchema that = (MaxwellJsonSerializationSchema) o; + return Objects.equals(jsonSerializer, that.jsonSerializer) + && timestampFormat == that.timestampFormat; + } + + @Override + public int hashCode() { + return Objects.hash(jsonSerializer, timestampFormat); + } + + private RowType createJsonRowType(DataType databaseSchema) { + DataType payload = + DataTypes.ROW( + DataTypes.FIELD("data", databaseSchema), + DataTypes.FIELD("type", DataTypes.STRING())); + return (RowType) payload.getLogicalType(); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java index 564c88b01..7f1d500b2 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -32,131 +32,127 @@ import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_SCHEMA; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_TYPE_VALUE; -/** - * Format descriptor for JSON. - */ +/** Format descriptor for JSON. */ public class Json extends FormatDescriptor { - private Boolean failOnMissingField; - private Boolean deriveSchema; - private Boolean ignoreParseErrors; - private String jsonSchema; - private String schema; - - /** - * Format descriptor for JSON. - */ - public Json() { - super(FORMAT_TYPE_VALUE, 1); - } - - /** - * Sets flag whether to fail if a field is missing or not. - * - * @param failOnMissingField If set to true, the operation fails if there is a missing field. - * If set to false, a missing field is set to null. - */ - public Json failOnMissingField(boolean failOnMissingField) { - this.failOnMissingField = failOnMissingField; - return this; - } - - /** - * Sets flag whether to fail when parsing json fails. - * - * @param ignoreParseErrors If set to true, the operation will ignore parse errors. - * If set to false, the operation fails when parsing json fails. - */ - public Json ignoreParseErrors(boolean ignoreParseErrors) { - this.ignoreParseErrors = ignoreParseErrors; - return this; - } - - /** - * Sets the JSON schema string with field names and the types according to the JSON schema - * specification [[http://json-schema.org/specification.html]]. - * - *

The schema might be nested. - * - * @param jsonSchema JSON schema - * @deprecated {@link Json} supports derive schema from table schema by default, - * it is no longer necessary to explicitly declare the format schema. - * This method will be removed in the future. - */ - @Deprecated - public Json jsonSchema(String jsonSchema) { - Preconditions.checkNotNull(jsonSchema); - this.jsonSchema = jsonSchema; - this.schema = null; - this.deriveSchema = null; - return this; - } - - /** - * Sets the schema using type information. - * - *

JSON objects are represented as ROW types. - * - *

The schema might be nested. - * - * @param schemaType type information that describes the schema - * @deprecated {@link Json} supports derive schema from table schema by default, - * it is no longer necessary to explicitly declare the format schema. - * This method will be removed in the future. - */ - @Deprecated - public Json schema(TypeInformation schemaType) { - Preconditions.checkNotNull(schemaType); - this.schema = TypeStringUtils.writeTypeInfo(schemaType); - this.jsonSchema = null; - this.deriveSchema = null; - return this; - } - - /** - * Derives the format schema from the table's schema described. - * - *

This allows for defining schema information only once. - * - *

The names, types, and fields' order of the format are determined by the table's - * schema. Time attributes are ignored if their origin is not a field. A "from" definition - * is interpreted as a field renaming in the format. - * - * @deprecated Derivation format schema from table's schema is the default behavior now. - * So there is no need to explicitly declare to derive schema. - */ - @Deprecated - public Json deriveSchema() { - this.deriveSchema = true; - this.schema = null; - this.jsonSchema = null; - return this; - } - - @Override - protected Map toFormatProperties() { - final DescriptorProperties properties = new DescriptorProperties(); - - if (deriveSchema != null) { - properties.putBoolean(FORMAT_DERIVE_SCHEMA, deriveSchema); - } - - if (jsonSchema != null) { - properties.putString(FORMAT_JSON_SCHEMA, jsonSchema); - } - - if (schema != null) { - properties.putString(FORMAT_SCHEMA, schema); - } - - if (failOnMissingField != null) { - properties.putBoolean(FORMAT_FAIL_ON_MISSING_FIELD, failOnMissingField); - } - - if (ignoreParseErrors != null) { - properties.putBoolean(FORMAT_IGNORE_PARSE_ERRORS, ignoreParseErrors); - } - - return properties.asMap(); - } + private Boolean failOnMissingField; + private Boolean deriveSchema; + private Boolean ignoreParseErrors; + private String jsonSchema; + private String schema; + + /** Format descriptor for JSON. */ + public Json() { + super(FORMAT_TYPE_VALUE, 1); + } + + /** + * Sets flag whether to fail if a field is missing or not. + * + * @param failOnMissingField If set to true, the operation fails if there is a missing field. If + * set to false, a missing field is set to null. + */ + public Json failOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + return this; + } + + /** + * Sets flag whether to fail when parsing json fails. + * + * @param ignoreParseErrors If set to true, the operation will ignore parse errors. If set to + * false, the operation fails when parsing json fails. + */ + public Json ignoreParseErrors(boolean ignoreParseErrors) { + this.ignoreParseErrors = ignoreParseErrors; + return this; + } + + /** + * Sets the JSON schema string with field names and the types according to the JSON schema + * specification [[http://json-schema.org/specification.html]]. + * + *

The schema might be nested. + * + * @param jsonSchema JSON schema + * @deprecated {@link Json} supports derive schema from table schema by default, it is no longer + * necessary to explicitly declare the format schema. This method will be removed in the + * future. + */ + @Deprecated + public Json jsonSchema(String jsonSchema) { + Preconditions.checkNotNull(jsonSchema); + this.jsonSchema = jsonSchema; + this.schema = null; + this.deriveSchema = null; + return this; + } + + /** + * Sets the schema using type information. + * + *

JSON objects are represented as ROW types. + * + *

The schema might be nested. + * + * @param schemaType type information that describes the schema + * @deprecated {@link Json} supports derive schema from table schema by default, it is no longer + * necessary to explicitly declare the format schema. This method will be removed in the + * future. + */ + @Deprecated + public Json schema(TypeInformation schemaType) { + Preconditions.checkNotNull(schemaType); + this.schema = TypeStringUtils.writeTypeInfo(schemaType); + this.jsonSchema = null; + this.deriveSchema = null; + return this; + } + + /** + * Derives the format schema from the table's schema described. + * + *

This allows for defining schema information only once. + * + *

The names, types, and fields' order of the format are determined by the table's schema. + * Time attributes are ignored if their origin is not a field. A "from" definition is + * interpreted as a field renaming in the format. + * + * @deprecated Derivation format schema from table's schema is the default behavior now. So + * there is no need to explicitly declare to derive schema. + */ + @Deprecated + public Json deriveSchema() { + this.deriveSchema = true; + this.schema = null; + this.jsonSchema = null; + return this; + } + + @Override + protected Map toFormatProperties() { + final DescriptorProperties properties = new DescriptorProperties(); + + if (deriveSchema != null) { + properties.putBoolean(FORMAT_DERIVE_SCHEMA, deriveSchema); + } + + if (jsonSchema != null) { + properties.putString(FORMAT_JSON_SCHEMA, jsonSchema); + } + + if (schema != null) { + properties.putString(FORMAT_SCHEMA, schema); + } + + if (failOnMissingField != null) { + properties.putBoolean(FORMAT_FAIL_ON_MISSING_FIELD, failOnMissingField); + } + + if (ignoreParseErrors != null) { + properties.putBoolean(FORMAT_IGNORE_PARSE_ERRORS, ignoreParseErrors); + } + + return properties.asMap(); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java index 2d0b04e11..8fcdaf517 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java @@ -21,43 +21,51 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.ValidationException; -/** - * Validator for {@link Json}. - */ +/** Validator for {@link Json}. */ @Internal public class JsonValidator extends FormatDescriptorValidator { - public static final String FORMAT_TYPE_VALUE = "json"; - public static final String FORMAT_SCHEMA = "format.schema"; - public static final String FORMAT_JSON_SCHEMA = "format.json-schema"; - public static final String FORMAT_FAIL_ON_MISSING_FIELD = "format.fail-on-missing-field"; - public static final String FORMAT_IGNORE_PARSE_ERRORS = "format.ignore-parse-errors"; + public static final String FORMAT_TYPE_VALUE = "json"; + public static final String FORMAT_SCHEMA = "format.schema"; + public static final String FORMAT_JSON_SCHEMA = "format.json-schema"; + public static final String FORMAT_FAIL_ON_MISSING_FIELD = "format.fail-on-missing-field"; + public static final String FORMAT_IGNORE_PARSE_ERRORS = "format.ignore-parse-errors"; - @Override - public void validate(DescriptorProperties properties) { - super.validate(properties); - properties.validateBoolean(FORMAT_DERIVE_SCHEMA, true); - final boolean deriveSchema = properties.getOptionalBoolean(FORMAT_DERIVE_SCHEMA).orElse(true); - final boolean hasSchema = properties.containsKey(FORMAT_SCHEMA); - final boolean hasSchemaString = properties.containsKey(FORMAT_JSON_SCHEMA); - // if a schema is defined, no matter derive schema is set or not, will use the defined schema - if (!deriveSchema && hasSchema && hasSchemaString) { - throw new ValidationException("A definition of both a schema and JSON schema is not allowed."); - } else if (!deriveSchema && !hasSchema && !hasSchemaString) { - throw new ValidationException("A definition of a schema or JSON schema is required " + - "if derivation from table's schema is disabled."); - } else if (hasSchema) { - properties.validateType(FORMAT_SCHEMA, false, true); - } else if (hasSchemaString) { - properties.validateString(FORMAT_JSON_SCHEMA, false, 1); - } + @Override + public void validate(DescriptorProperties properties) { + super.validate(properties); + properties.validateBoolean(FORMAT_DERIVE_SCHEMA, true); + final boolean deriveSchema = + properties.getOptionalBoolean(FORMAT_DERIVE_SCHEMA).orElse(true); + final boolean hasSchema = properties.containsKey(FORMAT_SCHEMA); + final boolean hasSchemaString = properties.containsKey(FORMAT_JSON_SCHEMA); + // if a schema is defined, no matter derive schema is set or not, will use the defined + // schema + if (!deriveSchema && hasSchema && hasSchemaString) { + throw new ValidationException( + "A definition of both a schema and JSON schema is not allowed."); + } else if (!deriveSchema && !hasSchema && !hasSchemaString) { + throw new ValidationException( + "A definition of a schema or JSON schema is required " + + "if derivation from table's schema is disabled."); + } else if (hasSchema) { + properties.validateType(FORMAT_SCHEMA, false, true); + } else if (hasSchemaString) { + properties.validateString(FORMAT_JSON_SCHEMA, false, 1); + } - properties.validateBoolean(FORMAT_FAIL_ON_MISSING_FIELD, true); - properties.validateBoolean(FORMAT_IGNORE_PARSE_ERRORS, true); - boolean failOnMissingField = properties.getOptionalBoolean(FORMAT_FAIL_ON_MISSING_FIELD).orElse(false); - boolean ignoreParseErrors = properties.getOptionalBoolean(FORMAT_IGNORE_PARSE_ERRORS).orElse(false); - if (ignoreParseErrors && failOnMissingField) { - throw new ValidationException(FORMAT_FAIL_ON_MISSING_FIELD + " and " + FORMAT_IGNORE_PARSE_ERRORS + " shouldn't both be true."); - } - } + properties.validateBoolean(FORMAT_FAIL_ON_MISSING_FIELD, true); + properties.validateBoolean(FORMAT_IGNORE_PARSE_ERRORS, true); + boolean failOnMissingField = + properties.getOptionalBoolean(FORMAT_FAIL_ON_MISSING_FIELD).orElse(false); + boolean ignoreParseErrors = + properties.getOptionalBoolean(FORMAT_IGNORE_PARSE_ERRORS).orElse(false); + if (ignoreParseErrors && failOnMissingField) { + throw new ValidationException( + FORMAT_FAIL_ON_MISSING_FIELD + + " and " + + FORMAT_IGNORE_PARSE_ERRORS + + " shouldn't both be true."); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java index 7ba344c02..c677a2f9d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java @@ -34,81 +34,81 @@ import java.util.Arrays; import java.util.List; -/** - * ITCase to test json format for {@link JsonFormatFactory}. - */ +/** ITCase to test json format for {@link JsonFormatFactory}. */ public class JsonBatchFileSystemITCase extends BatchFileSystemITCaseBase { - @Override - public String[] formatProperties() { - List ret = new ArrayList<>(); - ret.add("'format'='json'"); - ret.add("'json.ignore-parse-errors'='true'"); - return ret.toArray(new String[0]); - } - - @Test - public void testParseError() throws Exception { - String path = new URI(resultPath()).getPath(); - new File(path).mkdirs(); - File file = new File(path, "temp_file"); - file.createNewFile(); - FileUtils.writeFileUtf8(file, - "{\"x\":\"x5\",\"y\":5,\"a\":1,\"b\":1}\n" + - "{I am a wrong json.}\n" + - "{\"x\":\"x5\",\"y\":5,\"a\":1,\"b\":1}"); - - check("select * from nonPartitionedTable", - Arrays.asList( - Row.of("x5,5,1,1"), - Row.of("x5,5,1,1"))); - } - - @Test - public void bigDataTest() throws IOException { - int numRecords = 1000; - File dir = generateTestData(numRecords); - - env().setParallelism(1); - - String sql = String.format( - "CREATE TABLE bigdata_source ( " + - " id INT, " + - " content STRING" + - ") PARTITIONED by (id) WITH (" + - " 'connector' = 'filesystem'," + - " 'path' = '%s'," + - " 'format' = 'json'" + - ")", dir); - tEnv().executeSql(sql); - TableResult result = tEnv().executeSql("select * from bigdata_source"); - List elements = new ArrayList<>(); - result.collect().forEachRemaining(r -> elements.add((String) r.getField(1))); - Assert.assertEquals(numRecords, elements.size()); - elements.sort(String::compareTo); - - List expected = new ArrayList<>(); - for (int i = 0; i < numRecords; i++) { - expected.add(String.valueOf(i)); - } - expected.sort(String::compareTo); - - Assert.assertEquals(expected, elements); - } - - private static File generateTestData(int numRecords) throws IOException { - File tempDir = TEMPORARY_FOLDER.newFolder(); - - File root = new File(tempDir, "id=0"); - root.mkdir(); - - File dataFile = new File(root, "testdata"); - try (PrintWriter writer = new PrintWriter(dataFile)) { - for (int i = 0; i < numRecords; ++i) { - writer.println(String.format("{\"content\":\"%s\"}", i)); - } - } - - return tempDir; - } + @Override + public String[] formatProperties() { + List ret = new ArrayList<>(); + ret.add("'format'='json'"); + ret.add("'json.ignore-parse-errors'='true'"); + return ret.toArray(new String[0]); + } + + @Test + public void testParseError() throws Exception { + String path = new URI(resultPath()).getPath(); + new File(path).mkdirs(); + File file = new File(path, "temp_file"); + file.createNewFile(); + FileUtils.writeFileUtf8( + file, + "{\"x\":\"x5\",\"y\":5,\"a\":1,\"b\":1}\n" + + "{I am a wrong json.}\n" + + "{\"x\":\"x5\",\"y\":5,\"a\":1,\"b\":1}"); + + check( + "select * from nonPartitionedTable", + Arrays.asList(Row.of("x5,5,1,1"), Row.of("x5,5,1,1"))); + } + + @Test + public void bigDataTest() throws IOException { + int numRecords = 1000; + File dir = generateTestData(numRecords); + + env().setParallelism(1); + + String sql = + String.format( + "CREATE TABLE bigdata_source ( " + + " id INT, " + + " content STRING" + + ") PARTITIONED by (id) WITH (" + + " 'connector' = 'filesystem'," + + " 'path' = '%s'," + + " 'format' = 'json'" + + ")", + dir); + tEnv().executeSql(sql); + TableResult result = tEnv().executeSql("select * from bigdata_source"); + List elements = new ArrayList<>(); + result.collect().forEachRemaining(r -> elements.add((String) r.getField(1))); + Assert.assertEquals(numRecords, elements.size()); + elements.sort(String::compareTo); + + List expected = new ArrayList<>(); + for (int i = 0; i < numRecords; i++) { + expected.add(String.valueOf(i)); + } + expected.sort(String::compareTo); + + Assert.assertEquals(expected, elements); + } + + private static File generateTestData(int numRecords) throws IOException { + File tempDir = TEMPORARY_FOLDER.newFolder(); + + File root = new File(tempDir, "id=0"); + root.mkdir(); + + File dataFile = new File(root, "testdata"); + try (PrintWriter writer = new PrintWriter(dataFile)) { + for (int i = 0; i < numRecords; ++i) { + writer.println(String.format("{\"content\":\"%s\"}", i)); + } + } + + return tempDir; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java index dd1a8cc76..9e2cde3a2 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java @@ -20,13 +20,11 @@ import org.apache.flink.table.planner.runtime.stream.sql.FileCompactionITCaseBase; -/** - * Compaction it case for json. - */ +/** Compaction it case for json. */ public class JsonFileCompactionITCase extends FileCompactionITCaseBase { - @Override - protected String format() { - return "json"; - } + @Override + protected String format() { + return "json"; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index 441050cf6..ba6e23958 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -48,176 +48,189 @@ import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.junit.Assert.assertEquals; -/** - * Tests for the {@link JsonFormatFactory}. - */ +/** Tests for the {@link JsonFormatFactory}. */ public class JsonFormatFactoryTest extends TestLogger { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - private static final TableSchema SCHEMA = TableSchema.builder() - .field("field1", DataTypes.BOOLEAN()) - .field("field2", DataTypes.INT()) - .build(); - - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); - - @Test - public void testSeDeSchema() { - final Map tableOptions = getAllOptions(); - - testSchemaSerializationSchema(tableOptions); - - testSchemaDeserializationSchema(tableOptions); - } - - @Test - public void testFailOnMissingField() { - final Map tableOptions = getModifyOptions( - options -> options.put("json.fail-on-missing-field", "true")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new ValidationException("fail-on-missing-field and ignore-parse-errors shouldn't both be true."))); - testSchemaDeserializationSchema(tableOptions); - } - - @Test - public void testInvalidOptionForIgnoreParseErrors() { - final Map tableOptions = getModifyOptions( - options -> options.put("json.ignore-parse-errors", "abc")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new IllegalArgumentException("Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); - testSchemaDeserializationSchema(tableOptions); - } - - @Test - public void testInvalidOptionForTimestampFormat() { - final Map tableOptions = getModifyOptions( - options -> options.put("json.timestamp-format.standard", "test")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new ValidationException("Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - testSchemaDeserializationSchema(tableOptions); - } - - @Test - public void testLowerCaseOptionForTimestampFormat() { - final Map tableOptions = getModifyOptions( - options -> options.put("json.timestamp-format.standard", "iso-8601")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new ValidationException("Unsupported value 'iso-8601' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - testSchemaDeserializationSchema(tableOptions); - } - - @Test - public void testInvalidOptionForMapNullKeyMode() { - final Map tableOptions = getModifyOptions( - options -> options.put("json.map-null-key.mode", "invalid")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new ValidationException("Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - testSchemaSerializationSchema(tableOptions); - } - - @Test - public void testLowerCaseOptionForMapNullKeyMode() { - final Map tableOptions = getModifyOptions( - options -> options.put("json.map-null-key.mode", "fail")); - - testSchemaDeserializationSchema(tableOptions); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - private void testSchemaDeserializationSchema(Map options) { - final JsonRowDataDeserializationSchema expectedDeser = - new JsonRowDataDeserializationSchema( - ROW_TYPE, - InternalTypeInfo.of(ROW_TYPE), - false, - true, - TimestampFormat.ISO_8601); - - final DynamicTableSource actualSource = createTableSource(options); - assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - DeserializationSchema actualDeser = scanSourceMock.valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, - SCHEMA.toRowDataType()); - - assertEquals(expectedDeser, actualDeser); - } - - private void testSchemaSerializationSchema(Map options) { - final JsonRowDataSerializationSchema expectedSer = new JsonRowDataSerializationSchema( - ROW_TYPE, - TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, - "null"); - - final DynamicTableSink actualSink = createTableSink(options); - assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = sinkMock.valueFormat - .createRuntimeEncoder( - new SinkRuntimeProviderContext(false), - SCHEMA.toRowDataType()); - - assertEquals(expectedSer, actualSer); - } - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifyOptions(Consumer> optionModifier) { - Map options = getAllOptions(); - optionModifier.accept(options); - return options; - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", JsonFormatFactory.IDENTIFIER); - options.put("json.fail-on-missing-field", "false"); - options.put("json.ignore-parse-errors", "true"); - options.put("json.timestamp-format.standard", "ISO-8601"); - options.put("json.map-null-key.mode", "LITERAL"); - options.put("json.map-null-key.literal", "null"); - return options; - } - - private static DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "Mock scan table"), - new Configuration(), - JsonFormatFactoryTest.class.getClassLoader(), - false); - } - - private static DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "Mock sink table"), - new Configuration(), - JsonFormatFactoryTest.class.getClassLoader(), - false); - } + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = + TableSchema.builder() + .field("field1", DataTypes.BOOLEAN()) + .field("field2", DataTypes.INT()) + .build(); + + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + @Test + public void testSeDeSchema() { + final Map tableOptions = getAllOptions(); + + testSchemaSerializationSchema(tableOptions); + + testSchemaDeserializationSchema(tableOptions); + } + + @Test + public void testFailOnMissingField() { + final Map tableOptions = + getModifyOptions(options -> options.put("json.fail-on-missing-field", "true")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "fail-on-missing-field and ignore-parse-errors shouldn't both be true."))); + testSchemaDeserializationSchema(tableOptions); + } + + @Test + public void testInvalidOptionForIgnoreParseErrors() { + final Map tableOptions = + getModifyOptions(options -> options.put("json.ignore-parse-errors", "abc")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new IllegalArgumentException( + "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); + testSchemaDeserializationSchema(tableOptions); + } + + @Test + public void testInvalidOptionForTimestampFormat() { + final Map tableOptions = + getModifyOptions(options -> options.put("json.timestamp-format.standard", "test")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + testSchemaDeserializationSchema(tableOptions); + } + + @Test + public void testLowerCaseOptionForTimestampFormat() { + final Map tableOptions = + getModifyOptions( + options -> options.put("json.timestamp-format.standard", "iso-8601")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'iso-8601' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + testSchemaDeserializationSchema(tableOptions); + } + + @Test + public void testInvalidOptionForMapNullKeyMode() { + final Map tableOptions = + getModifyOptions(options -> options.put("json.map-null-key.mode", "invalid")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); + testSchemaSerializationSchema(tableOptions); + } + + @Test + public void testLowerCaseOptionForMapNullKeyMode() { + final Map tableOptions = + getModifyOptions(options -> options.put("json.map-null-key.mode", "fail")); + + testSchemaDeserializationSchema(tableOptions); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + private void testSchemaDeserializationSchema(Map options) { + final JsonRowDataDeserializationSchema expectedDeser = + new JsonRowDataDeserializationSchema( + ROW_TYPE, + InternalTypeInfo.of(ROW_TYPE), + false, + true, + TimestampFormat.ISO_8601); + + final DynamicTableSource actualSource = createTableSource(options); + assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = + scanSourceMock.valueFormat.createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); + + assertEquals(expectedDeser, actualDeser); + } + + private void testSchemaSerializationSchema(Map options) { + final JsonRowDataSerializationSchema expectedSer = + new JsonRowDataSerializationSchema( + ROW_TYPE, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); + + final DynamicTableSink actualSink = createTableSink(options); + assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = + sinkMock.valueFormat.createRuntimeEncoder( + new SinkRuntimeProviderContext(false), SCHEMA.toRowDataType()); + + assertEquals(expectedSer, actualSer); + } + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifyOptions(Consumer> optionModifier) { + Map options = getAllOptions(); + optionModifier.accept(options); + return options; + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", JsonFormatFactory.IDENTIFIER); + options.put("json.fail-on-missing-field", "false"); + options.put("json.ignore-parse-errors", "true"); + options.put("json.timestamp-format.standard", "ISO-8601"); + options.put("json.map-null-key.mode", "LITERAL"); + options.put("json.map-null-key.literal", "null"); + return options; + } + + private static DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "Mock scan table"), + new Configuration(), + JsonFormatFactoryTest.class.getClassLoader(), + false); + } + + private static DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "Mock sink table"), + new Configuration(), + JsonFormatFactoryTest.class.getClassLoader(), + false); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java index f2cd6d4d0..253707a30 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java @@ -23,17 +23,15 @@ import java.util.ArrayList; import java.util.List; -/** - * Test checkpoint for file system table factory with json format. - */ +/** Test checkpoint for file system table factory with json format. */ public class JsonFsStreamSinkITCase extends FsStreamingSinkITCaseBase { - @Override - public String[] additionalProperties() { - List ret = new ArrayList<>(); - ret.add("'format'='json'"); - // for test purpose - ret.add("'sink.rolling-policy.file-size'='1b'"); - return ret.toArray(new String[0]); - } + @Override + public String[] additionalProperties() { + List ret = new ArrayList<>(); + ret.add("'format'='json'"); + // for test purpose + ret.add("'sink.rolling-policy.file-size'='1b'"); + return ret.toArray(new String[0]); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java index 1462a262e..43ca2033c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java @@ -26,22 +26,20 @@ import static org.junit.Assert.assertEquals; -/** - * Tests for the {@link JsonNodeDeserializationSchema}. - */ +/** Tests for the {@link JsonNodeDeserializationSchema}. */ public class JsonNodeDeserializationSchemaTest { - @Test - public void testDeserialize() throws IOException { - ObjectMapper mapper = new ObjectMapper(); - ObjectNode initialValue = mapper.createObjectNode(); - initialValue.put("key", 4).put("value", "world"); - byte[] serializedValue = mapper.writeValueAsBytes(initialValue); + @Test + public void testDeserialize() throws IOException { + ObjectMapper mapper = new ObjectMapper(); + ObjectNode initialValue = mapper.createObjectNode(); + initialValue.put("key", 4).put("value", "world"); + byte[] serializedValue = mapper.writeValueAsBytes(initialValue); - JsonNodeDeserializationSchema schema = new JsonNodeDeserializationSchema(); - ObjectNode deserializedValue = schema.deserialize(serializedValue); + JsonNodeDeserializationSchema schema = new JsonNodeDeserializationSchema(); + ObjectNode deserializedValue = schema.deserialize(serializedValue); - assertEquals(4, deserializedValue.get("key").asInt()); - assertEquals("world", deserializedValue.get("value").asText()); - } + assertEquals(4, deserializedValue.get("key").asInt()); + assertEquals("world", deserializedValue.get("value").asText()); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index b49326b2d..ae3c2c2d4 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -79,655 +79,675 @@ */ public class JsonRowDataSerDeSchemaTest { - @Test - public void testSerDe() throws Exception { - byte tinyint = 'c'; - short smallint = 128; - int intValue = 45536; - float floatValue = 33.333F; - long bigint = 1238123899121L; - String name = "asdlkjasjkdla998y1122"; - byte[] bytes = new byte[1024]; - ThreadLocalRandom.current().nextBytes(bytes); - BigDecimal decimal = new BigDecimal("123.456789"); - Double[] doubles = new Double[]{1.1, 2.2, 3.3}; - LocalDate date = LocalDate.parse("1990-10-14"); - LocalTime time = LocalTime.parse("12:12:43"); - Timestamp timestamp3 = Timestamp.valueOf("1990-10-14 12:12:43.123"); - Timestamp timestamp9 = Timestamp.valueOf("1990-10-14 12:12:43.123456789"); - Instant timestampWithLocalZone = - LocalDateTime.of(1990, 10, 14, 12, 12, 43, 123456789). - atOffset(ZoneOffset.of("Z")).toInstant(); - - Map map = new HashMap<>(); - map.put("flink", 123L); - - Map multiSet = new HashMap<>(); - multiSet.put("blink", 2); - - Map> nestedMap = new HashMap<>(); - Map innerMap = new HashMap<>(); - innerMap.put("key", 234); - nestedMap.put("inner_map", innerMap); - - ObjectMapper objectMapper = new ObjectMapper(); - ArrayNode doubleNode = objectMapper.createArrayNode().add(1.1D).add(2.2D).add(3.3D); - - // Root - ObjectNode root = objectMapper.createObjectNode(); - root.put("bool", true); - root.put("tinyint", tinyint); - root.put("smallint", smallint); - root.put("int", intValue); - root.put("bigint", bigint); - root.put("float", floatValue); - root.put("name", name); - root.put("bytes", bytes); - root.put("decimal", decimal); - root.set("doubles", doubleNode); - root.put("date", "1990-10-14"); - root.put("time", "12:12:43"); - root.put("timestamp3", "1990-10-14T12:12:43.123"); - root.put("timestamp9", "1990-10-14T12:12:43.123456789"); - root.put("timestampWithLocalZone", "1990-10-14T12:12:43.123456789Z"); - root.putObject("map").put("flink", 123); - root.putObject("multiSet").put("blink", 2); - root.putObject("map2map").putObject("inner_map").put("key", 234); - - byte[] serializedJson = objectMapper.writeValueAsBytes(root); - - DataType dataType = ROW( - FIELD("bool", BOOLEAN()), - FIELD("tinyint", TINYINT()), - FIELD("smallint", SMALLINT()), - FIELD("int", INT()), - FIELD("bigint", BIGINT()), - FIELD("float", FLOAT()), - FIELD("name", STRING()), - FIELD("bytes", BYTES()), - FIELD("decimal", DECIMAL(9, 6)), - FIELD("doubles", ARRAY(DOUBLE())), - FIELD("date", DATE()), - FIELD("time", TIME(0)), - FIELD("timestamp3", TIMESTAMP(3)), - FIELD("timestamp9", TIMESTAMP(9)), - FIELD("timestampWithLocalZone", TIMESTAMP_WITH_LOCAL_TIME_ZONE(9)), - FIELD("map", MAP(STRING(), BIGINT())), - FIELD("multiSet", MULTISET(STRING())), - FIELD("map2map", MAP(STRING(), MAP(STRING(), INT())))); - RowType schema = (RowType) dataType.getLogicalType(); - TypeInformation resultTypeInfo = InternalTypeInfo.of(schema); - - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); - - Row expected = new Row(18); - expected.setField(0, true); - expected.setField(1, tinyint); - expected.setField(2, smallint); - expected.setField(3, intValue); - expected.setField(4, bigint); - expected.setField(5, floatValue); - expected.setField(6, name); - expected.setField(7, bytes); - expected.setField(8, decimal); - expected.setField(9, doubles); - expected.setField(10, date); - expected.setField(11, time); - expected.setField(12, timestamp3.toLocalDateTime()); - expected.setField(13, timestamp9.toLocalDateTime()); - expected.setField(14, timestampWithLocalZone); - expected.setField(15, map); - expected.setField(16, multiSet); - expected.setField(17, nestedMap); - - RowData rowData = deserializationSchema.deserialize(serializedJson); - Row actual = convertToExternal(rowData, dataType); - assertEquals(expected, actual); - - // test serialization - JsonRowDataSerializationSchema serializationSchema = - new JsonRowDataSerializationSchema( - schema, - TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, - "null"); - - byte[] actualBytes = serializationSchema.serialize(rowData); - assertEquals(new String(serializedJson), new String(actualBytes)); - } - - /** - * Tests the deserialization slow path, - * e.g. convert into string and use {@link Double#parseDouble(String)}. - */ - @Test - public void testSlowDeserialization() throws Exception { - Random random = new Random(); - boolean bool = random.nextBoolean(); - int integer = random.nextInt(); - long bigint = random.nextLong(); - double doubleValue = random.nextDouble(); - float floatValue = random.nextFloat(); - - ObjectMapper objectMapper = new ObjectMapper(); - ObjectNode root = objectMapper.createObjectNode(); - root.put("bool", String.valueOf(bool)); - root.put("int", String.valueOf(integer)); - root.put("bigint", String.valueOf(bigint)); - root.put("double1", String.valueOf(doubleValue)); - root.put("double2", new BigDecimal(doubleValue)); - root.put("float1", String.valueOf(floatValue)); - root.put("float2", new BigDecimal(floatValue)); - - byte[] serializedJson = objectMapper.writeValueAsBytes(root); - - DataType dataType = ROW( - FIELD("bool", BOOLEAN()), - FIELD("int", INT()), - FIELD("bigint", BIGINT()), - FIELD("double1", DOUBLE()), - FIELD("double2", DOUBLE()), - FIELD("float1", FLOAT()), - FIELD("float2", FLOAT()) - ); - RowType rowType = (RowType) dataType.getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.ISO_8601); - - Row expected = new Row(7); - expected.setField(0, bool); - expected.setField(1, integer); - expected.setField(2, bigint); - expected.setField(3, doubleValue); - expected.setField(4, doubleValue); - expected.setField(5, floatValue); - expected.setField(6, floatValue); - - RowData rowData = deserializationSchema.deserialize(serializedJson); - Row actual = convertToExternal(rowData, dataType); - assertEquals(expected, actual); - } - - @Test - public void testSerDeMultiRows() throws Exception { - RowType rowType = (RowType) ROW( - FIELD("f1", INT()), - FIELD("f2", BOOLEAN()), - FIELD("f3", STRING()), - FIELD("f4", MAP(STRING(), STRING())), - FIELD("f5", ARRAY(STRING())), - FIELD("f6", ROW( - FIELD("f1", STRING()), - FIELD("f2", INT()))) - ).getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.ISO_8601); - JsonRowDataSerializationSchema serializationSchema = - new JsonRowDataSerializationSchema( - rowType, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, "null"); - - ObjectMapper objectMapper = new ObjectMapper(); - - // the first row - { - ObjectNode root = objectMapper.createObjectNode(); - root.put("f1", 1); - root.put("f2", true); - root.put("f3", "str"); - ObjectNode map = root.putObject("f4"); - map.put("hello1", "flink"); - ArrayNode array = root.putArray("f5"); - array.add("element1"); - array.add("element2"); - ObjectNode row = root.putObject("f6"); - row.put("f1", "this is row1"); - row.put("f2", 12); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); - RowData rowData = deserializationSchema.deserialize(serializedJson); - byte[] actual = serializationSchema.serialize(rowData); - assertEquals(new String(serializedJson), new String(actual)); - } - - // the second row - { - ObjectNode root = objectMapper.createObjectNode(); - root.put("f1", 10); - root.put("f2", false); - root.put("f3", "newStr"); - ObjectNode map = root.putObject("f4"); - map.put("hello2", "json"); - ArrayNode array = root.putArray("f5"); - array.add("element3"); - array.add("element4"); - ObjectNode row = root.putObject("f6"); - row.put("f1", "this is row2"); - row.putNull("f2"); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); - RowData rowData = deserializationSchema.deserialize(serializedJson); - byte[] actual = serializationSchema.serialize(rowData); - assertEquals(new String(serializedJson), new String(actual)); - } - } - - @Test - public void testSerDeMultiRowsWithNullValues() throws Exception { - String[] jsons = new String[] { - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{\"k1\":10.01,\"k2\":\"invalid\"}}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, " + - "\"ids\":[1, 2, 3]}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{}}", - }; - - String[] expected = new String[] { - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null,\"metrics\":{\"k1\":10.01,\"k2\":null}}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}," + - "\"ids\":[1,2,3],\"metrics\":null}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null,\"metrics\":{}}", - }; - - RowType rowType = (RowType) ROW( - FIELD("svt", STRING()), - FIELD("ops", ROW(FIELD("id", STRING()))), - FIELD("ids", ARRAY(INT())), - FIELD("metrics", MAP(STRING(), DOUBLE())) - ).getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, InternalTypeInfo.of(rowType), false, true, TimestampFormat.ISO_8601); - JsonRowDataSerializationSchema serializationSchema = - new JsonRowDataSerializationSchema( - rowType, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, "null"); - - for (int i = 0; i < jsons.length; i++) { - String json = jsons[i]; - RowData row = deserializationSchema.deserialize(json.getBytes()); - String result = new String(serializationSchema.serialize(row)); - assertEquals(expected[i], result); - } - } - - @Test - public void testDeserializationMissingNode() throws Exception { - DataType dataType = ROW(FIELD("name", STRING())); - RowType schema = (RowType) dataType.getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); - RowData rowData = deserializationSchema.deserialize("".getBytes()); - assertEquals(null, rowData); - } - - @Test - public void testDeserializationMissingField() throws Exception { - ObjectMapper objectMapper = new ObjectMapper(); - - // Root - ObjectNode root = objectMapper.createObjectNode(); - root.put("id", 123123123); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); - - DataType dataType = ROW(FIELD("name", STRING())); - RowType schema = (RowType) dataType.getLogicalType(); - - // pass on missing field - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), false, false, TimestampFormat.ISO_8601); - - Row expected = new Row(1); - Row actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); - assertEquals(expected, actual); - - // fail on missing field - deserializationSchema = deserializationSchema = new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); - - String errorMessage = "Failed to deserialize JSON '{\"id\":123123123}'."; - try { - deserializationSchema.deserialize(serializedJson); - fail("expecting exception message: " + errorMessage); - } catch (Throwable t) { - assertEquals(errorMessage, t.getMessage()); - } - - // ignore on parse error - deserializationSchema = new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), false, true, TimestampFormat.ISO_8601); - actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); - assertEquals(expected, actual); - - errorMessage = "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."; - try { - // failOnMissingField and ignoreParseErrors both enabled - new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), true, true, TimestampFormat.ISO_8601); - Assert.fail("expecting exception message: " + errorMessage); - } catch (Throwable t) { - assertEquals(errorMessage, t.getMessage()); - } - } - - @Test - public void testSerDeSQLTimestampFormat() throws Exception { - RowType rowType = (RowType) ROW( - FIELD("timestamp3", TIMESTAMP(3)), - FIELD("timestamp9", TIMESTAMP(9)), - FIELD("timestamp_with_local_timezone3", TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), - FIELD("timestamp_with_local_timezone9", TIMESTAMP_WITH_LOCAL_TIME_ZONE(9)) - ).getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); - JsonRowDataSerializationSchema serializationSchema = - new JsonRowDataSerializationSchema( - rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, "null"); - - ObjectMapper objectMapper = new ObjectMapper(); - - ObjectNode root = objectMapper.createObjectNode(); - root.put("timestamp3", "1990-10-14 12:12:43.123"); - root.put("timestamp9", "1990-10-14 12:12:43.123456789"); - root.put("timestamp_with_local_timezone3", "1990-10-14 12:12:43.123Z"); - root.put("timestamp_with_local_timezone9", "1990-10-14 12:12:43.123456789Z"); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); - RowData rowData = deserializationSchema.deserialize(serializedJson); - byte[] actual = serializationSchema.serialize(rowData); - assertEquals(new String(serializedJson), new String(actual)); - } - - @Test - public void testSerializationMapNullKey() throws Exception { - RowType rowType = (RowType) ROW( - FIELD("nestedMap", MAP(STRING(), MAP(STRING(), INT()))) - ).getLogicalType(); - - // test data - // use LinkedHashMap to make sure entries order - Map map = new LinkedHashMap<>(); - map.put(StringData.fromString("no-null key"), 1); - map.put(StringData.fromString(null), 2); - GenericMapData mapData = new GenericMapData(map); - - Map nestedMap = new LinkedHashMap<>(); - nestedMap.put(StringData.fromString("no-null key"), mapData); - nestedMap.put(StringData.fromString(null), mapData); - - GenericMapData nestedMapData = new GenericMapData(nestedMap); - GenericRowData rowData = new GenericRowData(1); - rowData.setField(0, nestedMapData); - - JsonRowDataSerializationSchema serializationSchema1 = - new JsonRowDataSerializationSchema( - rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.FAIL, "null"); - // expect message for serializationSchema1 - String errorMessage1 = "JSON format doesn't support to serialize map data with null keys." - + " You can drop null key entries or encode null in literals by specifying map-null-key.mode option."; - - JsonRowDataSerializationSchema serializationSchema2 = - new JsonRowDataSerializationSchema( - rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.DROP, "null"); - // expect result for serializationSchema2 - String expectResult2 = "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1}}}"; - - JsonRowDataSerializationSchema serializationSchema3 = - new JsonRowDataSerializationSchema( - rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, "nullKey"); - // expect result for serializationSchema3 - String expectResult3 = - "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1,\"nullKey\":2},\"nullKey\":{\"no-null key\":1,\"nullKey\":2}}}"; - - try { - // throw exception when mapNullKey Mode is fail - serializationSchema1.serialize(rowData); - Assert.fail("expecting exception message: " + errorMessage1); - } catch (Throwable t) { - assertEquals(errorMessage1, t.getCause().getMessage()); - } - - // mapNullKey Mode is drop - byte[] actual2 = serializationSchema2.serialize(rowData); - assertEquals(expectResult2, new String(actual2)); - - // mapNullKey Mode is literal - byte[] actual3 = serializationSchema3.serialize(rowData); - assertEquals(expectResult3, new String(actual3)); - } - - @Test - public void testJsonParse() throws Exception { - for (TestSpec spec : testData) { - testIgnoreParseErrors(spec); - if (spec.errorMessage != null) { - testParseErrors(spec); - } - } - } - - private void testIgnoreParseErrors(TestSpec spec) throws Exception { - // the parsing field should be null and no exception is thrown - JsonRowDataDeserializationSchema ignoreErrorsSchema = new JsonRowDataDeserializationSchema( - spec.rowType, InternalTypeInfo.of(spec.rowType), false, true, - spec.timestampFormat); - Row expected; - if (spec.expected != null) { - expected = spec.expected; - } else { - expected = new Row(1); - } - RowData rowData = ignoreErrorsSchema.deserialize(spec.json.getBytes()); - Row actual = convertToExternal(rowData, fromLogicalToDataType(spec.rowType)); - assertEquals("Test Ignore Parse Error: " + spec.json, - expected, - actual); - } - - private void testParseErrors(TestSpec spec) throws Exception { - // expect exception if parse error is not ignored - JsonRowDataDeserializationSchema failingSchema = new JsonRowDataDeserializationSchema( - spec.rowType, InternalTypeInfo.of(spec.rowType), false, false, - spec.timestampFormat); - - try { - failingSchema.deserialize(spec.json.getBytes()); - fail("expecting exception " + spec.errorMessage); - } catch (Throwable t) { - assertEquals(t.getMessage(), spec.errorMessage); - } - } - - private static List testData = Arrays.asList( - TestSpec - .json("{\"id\": \"trueA\"}") - .rowType(ROW(FIELD("id", BOOLEAN()))) - .expect(Row.of(false)), - - TestSpec - .json("{\"id\": true}") - .rowType(ROW(FIELD("id", BOOLEAN()))) - .expect(Row.of(true)), - - TestSpec - .json("{\"id\":\"abc\"}") - .rowType(ROW(FIELD("id", INT()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'."), - - TestSpec - .json("{\"id\":112.013}") - .rowType(ROW(FIELD("id", BIGINT()))) - .expect(Row.of(112L)), - - TestSpec - .json("{\"id\":\"long\"}") - .rowType(ROW(FIELD("id", BIGINT()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'."), - - TestSpec - .json("{\"id\":\"112.013.123\"}") - .rowType(ROW(FIELD("id", FLOAT()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'."), - - TestSpec - .json("{\"id\":\"112.013.123\"}") - .rowType(ROW(FIELD("id", DOUBLE()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'."), - - TestSpec - .json("{\"id\":\"18:00:243\"}") - .rowType(ROW(FIELD("id", TIME()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'."), - - TestSpec - .json("{\"id\":\"18:00:243\"}") - .rowType(ROW(FIELD("id", TIME()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'."), - - TestSpec - .json("{\"id\":\"20191112\"}") - .rowType(ROW(FIELD("id", DATE()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'."), - - TestSpec - .json("{\"id\":\"20191112\"}") - .rowType(ROW(FIELD("id", DATE()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'."), - - TestSpec - .json("{\"id\":true}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("true")), - - TestSpec - .json("{\"id\":123.234}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("123.234")), - - TestSpec - .json("{\"id\":1234567}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("1234567")), - - TestSpec - .json("{\"id\":\"string field\"}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("string field")), - - TestSpec - .json("{\"id\":[\"array data1\",\"array data2\",123,234.345]}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("[\"array data1\",\"array data2\",123,234.345]")), - - TestSpec - .json("{\"id\":{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}")), - - TestSpec - .json("{\"id\":\"2019-11-12 18:00:12\"}") - .rowType(ROW(FIELD("id", TIMESTAMP(0)))) - .timestampFormat(TimestampFormat.ISO_8601) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'."), - - TestSpec - .json("{\"id\":\"2019-11-12T18:00:12\"}") - .rowType(ROW(FIELD("id", TIMESTAMP(0)))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."), - - TestSpec - .json("{\"id\":\"2019-11-12T18:00:12Z\"}") - .rowType(ROW(FIELD("id", TIMESTAMP(0)))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'."), - - TestSpec - .json("{\"id\":\"2019-11-12T18:00:12Z\"}") - .rowType(ROW(FIELD("id", TIMESTAMP(0)))) - .timestampFormat(TimestampFormat.ISO_8601) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'."), - - TestSpec - .json("{\"id\":\"abc\"}") - .rowType(ROW(FIELD("id", DECIMAL(10, 3)))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'."), - - TestSpec - .json("{\"row\":{\"id\":\"abc\"}}") - .rowType(ROW(FIELD("row", ROW(FIELD("id", BOOLEAN()))))) - .expect(Row.of(Row.of(false))), - - TestSpec - .json("{\"array\":[123, \"abc\"]}") - .rowType(ROW(FIELD("array", ARRAY(INT())))) - .expect(Row.of((Object) new Integer[]{123, null})) - .expectErrorMessage("Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'."), - - TestSpec - .json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") - .rowType(ROW(FIELD("map", MAP(STRING(), INT())))) - .expect(Row.of(createHashMap("key1", 123, "key2", null))) - .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'."), - - TestSpec - .json("{\"id\":\"2019-11-12T18:00:12\"}") - .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0)))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."), - - TestSpec - .json("{\"id\":\"2019-11-12T18:00:12+0800\"}") - .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0)))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12+0800\"}'."), - - TestSpec - .json("{\"id\":1,\"factor\":799.929496989092949698}") - .rowType(ROW(FIELD("id", INT()), FIELD("factor", DECIMAL(38, 18)))) - .expect(Row.of(1, new BigDecimal("799.929496989092949698"))) - ); - - private static Map createHashMap(String k1, Integer v1, String k2, Integer v2) { - Map map = new HashMap<>(); - map.put(k1, v1); - map.put(k2, v2); - return map; - } - - @SuppressWarnings("unchecked") - private static Row convertToExternal(RowData rowData, DataType dataType) { - return (Row) DataFormatConverters.getConverterForDataType(dataType).toExternal(rowData); - } - - private static class TestSpec { - private final String json; - private RowType rowType; - private TimestampFormat timestampFormat = TimestampFormat.SQL; - private Row expected; - private String errorMessage; - - private TestSpec(String json) { - this.json = json; - } - - public static TestSpec json(String json) { - return new TestSpec(json); - } - - TestSpec expect(Row row) { - this.expected = row; - return this; - } - - TestSpec rowType(DataType rowType) { - this.rowType = (RowType) rowType.getLogicalType(); - return this; - } - - TestSpec expectErrorMessage(String errorMessage) { - this.errorMessage = errorMessage; - return this; - } - - TestSpec timestampFormat(TimestampFormat timestampFormat){ - this.timestampFormat = timestampFormat; - return this; - } - } + @Test + public void testSerDe() throws Exception { + byte tinyint = 'c'; + short smallint = 128; + int intValue = 45536; + float floatValue = 33.333F; + long bigint = 1238123899121L; + String name = "asdlkjasjkdla998y1122"; + byte[] bytes = new byte[1024]; + ThreadLocalRandom.current().nextBytes(bytes); + BigDecimal decimal = new BigDecimal("123.456789"); + Double[] doubles = new Double[] {1.1, 2.2, 3.3}; + LocalDate date = LocalDate.parse("1990-10-14"); + LocalTime time = LocalTime.parse("12:12:43"); + Timestamp timestamp3 = Timestamp.valueOf("1990-10-14 12:12:43.123"); + Timestamp timestamp9 = Timestamp.valueOf("1990-10-14 12:12:43.123456789"); + Instant timestampWithLocalZone = + LocalDateTime.of(1990, 10, 14, 12, 12, 43, 123456789) + .atOffset(ZoneOffset.of("Z")) + .toInstant(); + + Map map = new HashMap<>(); + map.put("flink", 123L); + + Map multiSet = new HashMap<>(); + multiSet.put("blink", 2); + + Map> nestedMap = new HashMap<>(); + Map innerMap = new HashMap<>(); + innerMap.put("key", 234); + nestedMap.put("inner_map", innerMap); + + ObjectMapper objectMapper = new ObjectMapper(); + ArrayNode doubleNode = objectMapper.createArrayNode().add(1.1D).add(2.2D).add(3.3D); + + // Root + ObjectNode root = objectMapper.createObjectNode(); + root.put("bool", true); + root.put("tinyint", tinyint); + root.put("smallint", smallint); + root.put("int", intValue); + root.put("bigint", bigint); + root.put("float", floatValue); + root.put("name", name); + root.put("bytes", bytes); + root.put("decimal", decimal); + root.set("doubles", doubleNode); + root.put("date", "1990-10-14"); + root.put("time", "12:12:43"); + root.put("timestamp3", "1990-10-14T12:12:43.123"); + root.put("timestamp9", "1990-10-14T12:12:43.123456789"); + root.put("timestampWithLocalZone", "1990-10-14T12:12:43.123456789Z"); + root.putObject("map").put("flink", 123); + root.putObject("multiSet").put("blink", 2); + root.putObject("map2map").putObject("inner_map").put("key", 234); + + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + DataType dataType = + ROW( + FIELD("bool", BOOLEAN()), + FIELD("tinyint", TINYINT()), + FIELD("smallint", SMALLINT()), + FIELD("int", INT()), + FIELD("bigint", BIGINT()), + FIELD("float", FLOAT()), + FIELD("name", STRING()), + FIELD("bytes", BYTES()), + FIELD("decimal", DECIMAL(9, 6)), + FIELD("doubles", ARRAY(DOUBLE())), + FIELD("date", DATE()), + FIELD("time", TIME(0)), + FIELD("timestamp3", TIMESTAMP(3)), + FIELD("timestamp9", TIMESTAMP(9)), + FIELD("timestampWithLocalZone", TIMESTAMP_WITH_LOCAL_TIME_ZONE(9)), + FIELD("map", MAP(STRING(), BIGINT())), + FIELD("multiSet", MULTISET(STRING())), + FIELD("map2map", MAP(STRING(), MAP(STRING(), INT())))); + RowType schema = (RowType) dataType.getLogicalType(); + TypeInformation resultTypeInfo = InternalTypeInfo.of(schema); + + JsonRowDataDeserializationSchema deserializationSchema = + new JsonRowDataDeserializationSchema( + schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); + + Row expected = new Row(18); + expected.setField(0, true); + expected.setField(1, tinyint); + expected.setField(2, smallint); + expected.setField(3, intValue); + expected.setField(4, bigint); + expected.setField(5, floatValue); + expected.setField(6, name); + expected.setField(7, bytes); + expected.setField(8, decimal); + expected.setField(9, doubles); + expected.setField(10, date); + expected.setField(11, time); + expected.setField(12, timestamp3.toLocalDateTime()); + expected.setField(13, timestamp9.toLocalDateTime()); + expected.setField(14, timestampWithLocalZone); + expected.setField(15, map); + expected.setField(16, multiSet); + expected.setField(17, nestedMap); + + RowData rowData = deserializationSchema.deserialize(serializedJson); + Row actual = convertToExternal(rowData, dataType); + assertEquals(expected, actual); + + // test serialization + JsonRowDataSerializationSchema serializationSchema = + new JsonRowDataSerializationSchema( + schema, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); + + byte[] actualBytes = serializationSchema.serialize(rowData); + assertEquals(new String(serializedJson), new String(actualBytes)); + } + + /** + * Tests the deserialization slow path, e.g. convert into string and use {@link + * Double#parseDouble(String)}. + */ + @Test + public void testSlowDeserialization() throws Exception { + Random random = new Random(); + boolean bool = random.nextBoolean(); + int integer = random.nextInt(); + long bigint = random.nextLong(); + double doubleValue = random.nextDouble(); + float floatValue = random.nextFloat(); + + ObjectMapper objectMapper = new ObjectMapper(); + ObjectNode root = objectMapper.createObjectNode(); + root.put("bool", String.valueOf(bool)); + root.put("int", String.valueOf(integer)); + root.put("bigint", String.valueOf(bigint)); + root.put("double1", String.valueOf(doubleValue)); + root.put("double2", new BigDecimal(doubleValue)); + root.put("float1", String.valueOf(floatValue)); + root.put("float2", new BigDecimal(floatValue)); + + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + DataType dataType = + ROW( + FIELD("bool", BOOLEAN()), + FIELD("int", INT()), + FIELD("bigint", BIGINT()), + FIELD("double1", DOUBLE()), + FIELD("double2", DOUBLE()), + FIELD("float1", FLOAT()), + FIELD("float2", FLOAT())); + RowType rowType = (RowType) dataType.getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = + new JsonRowDataDeserializationSchema( + rowType, + InternalTypeInfo.of(rowType), + false, + false, + TimestampFormat.ISO_8601); + + Row expected = new Row(7); + expected.setField(0, bool); + expected.setField(1, integer); + expected.setField(2, bigint); + expected.setField(3, doubleValue); + expected.setField(4, doubleValue); + expected.setField(5, floatValue); + expected.setField(6, floatValue); + + RowData rowData = deserializationSchema.deserialize(serializedJson); + Row actual = convertToExternal(rowData, dataType); + assertEquals(expected, actual); + } + + @Test + public void testSerDeMultiRows() throws Exception { + RowType rowType = + (RowType) + ROW( + FIELD("f1", INT()), + FIELD("f2", BOOLEAN()), + FIELD("f3", STRING()), + FIELD("f4", MAP(STRING(), STRING())), + FIELD("f5", ARRAY(STRING())), + FIELD("f6", ROW(FIELD("f1", STRING()), FIELD("f2", INT())))) + .getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = + new JsonRowDataDeserializationSchema( + rowType, + InternalTypeInfo.of(rowType), + false, + false, + TimestampFormat.ISO_8601); + JsonRowDataSerializationSchema serializationSchema = + new JsonRowDataSerializationSchema( + rowType, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); + + ObjectMapper objectMapper = new ObjectMapper(); + + // the first row + { + ObjectNode root = objectMapper.createObjectNode(); + root.put("f1", 1); + root.put("f2", true); + root.put("f3", "str"); + ObjectNode map = root.putObject("f4"); + map.put("hello1", "flink"); + ArrayNode array = root.putArray("f5"); + array.add("element1"); + array.add("element2"); + ObjectNode row = root.putObject("f6"); + row.put("f1", "this is row1"); + row.put("f2", 12); + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + RowData rowData = deserializationSchema.deserialize(serializedJson); + byte[] actual = serializationSchema.serialize(rowData); + assertEquals(new String(serializedJson), new String(actual)); + } + + // the second row + { + ObjectNode root = objectMapper.createObjectNode(); + root.put("f1", 10); + root.put("f2", false); + root.put("f3", "newStr"); + ObjectNode map = root.putObject("f4"); + map.put("hello2", "json"); + ArrayNode array = root.putArray("f5"); + array.add("element3"); + array.add("element4"); + ObjectNode row = root.putObject("f6"); + row.put("f1", "this is row2"); + row.putNull("f2"); + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + RowData rowData = deserializationSchema.deserialize(serializedJson); + byte[] actual = serializationSchema.serialize(rowData); + assertEquals(new String(serializedJson), new String(actual)); + } + } + + @Test + public void testSerDeMultiRowsWithNullValues() throws Exception { + String[] jsons = + new String[] { + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{\"k1\":10.01,\"k2\":\"invalid\"}}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, " + + "\"ids\":[1, 2, 3]}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{}}", + }; + + String[] expected = + new String[] { + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null,\"metrics\":{\"k1\":10.01,\"k2\":null}}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}," + + "\"ids\":[1,2,3],\"metrics\":null}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null,\"metrics\":{}}", + }; + + RowType rowType = + (RowType) + ROW( + FIELD("svt", STRING()), + FIELD("ops", ROW(FIELD("id", STRING()))), + FIELD("ids", ARRAY(INT())), + FIELD("metrics", MAP(STRING(), DOUBLE()))) + .getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = + new JsonRowDataDeserializationSchema( + rowType, + InternalTypeInfo.of(rowType), + false, + true, + TimestampFormat.ISO_8601); + JsonRowDataSerializationSchema serializationSchema = + new JsonRowDataSerializationSchema( + rowType, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); + + for (int i = 0; i < jsons.length; i++) { + String json = jsons[i]; + RowData row = deserializationSchema.deserialize(json.getBytes()); + String result = new String(serializationSchema.serialize(row)); + assertEquals(expected[i], result); + } + } + + @Test + public void testDeserializationMissingNode() throws Exception { + DataType dataType = ROW(FIELD("name", STRING())); + RowType schema = (RowType) dataType.getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = + new JsonRowDataDeserializationSchema( + schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); + RowData rowData = deserializationSchema.deserialize("".getBytes()); + assertEquals(null, rowData); + } + + @Test + public void testDeserializationMissingField() throws Exception { + ObjectMapper objectMapper = new ObjectMapper(); + + // Root + ObjectNode root = objectMapper.createObjectNode(); + root.put("id", 123123123); + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + DataType dataType = ROW(FIELD("name", STRING())); + RowType schema = (RowType) dataType.getLogicalType(); + + // pass on missing field + JsonRowDataDeserializationSchema deserializationSchema = + new JsonRowDataDeserializationSchema( + schema, + InternalTypeInfo.of(schema), + false, + false, + TimestampFormat.ISO_8601); + + Row expected = new Row(1); + Row actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); + assertEquals(expected, actual); + + // fail on missing field + deserializationSchema = + deserializationSchema = + new JsonRowDataDeserializationSchema( + schema, + InternalTypeInfo.of(schema), + true, + false, + TimestampFormat.ISO_8601); + + String errorMessage = "Failed to deserialize JSON '{\"id\":123123123}'."; + try { + deserializationSchema.deserialize(serializedJson); + fail("expecting exception message: " + errorMessage); + } catch (Throwable t) { + assertEquals(errorMessage, t.getMessage()); + } + + // ignore on parse error + deserializationSchema = + new JsonRowDataDeserializationSchema( + schema, InternalTypeInfo.of(schema), false, true, TimestampFormat.ISO_8601); + actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); + assertEquals(expected, actual); + + errorMessage = + "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."; + try { + // failOnMissingField and ignoreParseErrors both enabled + new JsonRowDataDeserializationSchema( + schema, InternalTypeInfo.of(schema), true, true, TimestampFormat.ISO_8601); + Assert.fail("expecting exception message: " + errorMessage); + } catch (Throwable t) { + assertEquals(errorMessage, t.getMessage()); + } + } + + @Test + public void testSerDeSQLTimestampFormat() throws Exception { + RowType rowType = + (RowType) + ROW( + FIELD("timestamp3", TIMESTAMP(3)), + FIELD("timestamp9", TIMESTAMP(9)), + FIELD( + "timestamp_with_local_timezone3", + TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), + FIELD( + "timestamp_with_local_timezone9", + TIMESTAMP_WITH_LOCAL_TIME_ZONE(9))) + .getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = + new JsonRowDataDeserializationSchema( + rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); + JsonRowDataSerializationSchema serializationSchema = + new JsonRowDataSerializationSchema( + rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, "null"); + + ObjectMapper objectMapper = new ObjectMapper(); + + ObjectNode root = objectMapper.createObjectNode(); + root.put("timestamp3", "1990-10-14 12:12:43.123"); + root.put("timestamp9", "1990-10-14 12:12:43.123456789"); + root.put("timestamp_with_local_timezone3", "1990-10-14 12:12:43.123Z"); + root.put("timestamp_with_local_timezone9", "1990-10-14 12:12:43.123456789Z"); + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + RowData rowData = deserializationSchema.deserialize(serializedJson); + byte[] actual = serializationSchema.serialize(rowData); + assertEquals(new String(serializedJson), new String(actual)); + } + + @Test + public void testSerializationMapNullKey() throws Exception { + RowType rowType = + (RowType) + ROW(FIELD("nestedMap", MAP(STRING(), MAP(STRING(), INT())))) + .getLogicalType(); + + // test data + // use LinkedHashMap to make sure entries order + Map map = new LinkedHashMap<>(); + map.put(StringData.fromString("no-null key"), 1); + map.put(StringData.fromString(null), 2); + GenericMapData mapData = new GenericMapData(map); + + Map nestedMap = new LinkedHashMap<>(); + nestedMap.put(StringData.fromString("no-null key"), mapData); + nestedMap.put(StringData.fromString(null), mapData); + + GenericMapData nestedMapData = new GenericMapData(nestedMap); + GenericRowData rowData = new GenericRowData(1); + rowData.setField(0, nestedMapData); + + JsonRowDataSerializationSchema serializationSchema1 = + new JsonRowDataSerializationSchema( + rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.FAIL, "null"); + // expect message for serializationSchema1 + String errorMessage1 = + "JSON format doesn't support to serialize map data with null keys." + + " You can drop null key entries or encode null in literals by specifying map-null-key.mode option."; + + JsonRowDataSerializationSchema serializationSchema2 = + new JsonRowDataSerializationSchema( + rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.DROP, "null"); + // expect result for serializationSchema2 + String expectResult2 = "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1}}}"; + + JsonRowDataSerializationSchema serializationSchema3 = + new JsonRowDataSerializationSchema( + rowType, + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.LITERAL, + "nullKey"); + // expect result for serializationSchema3 + String expectResult3 = + "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1,\"nullKey\":2},\"nullKey\":{\"no-null key\":1,\"nullKey\":2}}}"; + + try { + // throw exception when mapNullKey Mode is fail + serializationSchema1.serialize(rowData); + Assert.fail("expecting exception message: " + errorMessage1); + } catch (Throwable t) { + assertEquals(errorMessage1, t.getCause().getMessage()); + } + + // mapNullKey Mode is drop + byte[] actual2 = serializationSchema2.serialize(rowData); + assertEquals(expectResult2, new String(actual2)); + + // mapNullKey Mode is literal + byte[] actual3 = serializationSchema3.serialize(rowData); + assertEquals(expectResult3, new String(actual3)); + } + + @Test + public void testJsonParse() throws Exception { + for (TestSpec spec : testData) { + testIgnoreParseErrors(spec); + if (spec.errorMessage != null) { + testParseErrors(spec); + } + } + } + + private void testIgnoreParseErrors(TestSpec spec) throws Exception { + // the parsing field should be null and no exception is thrown + JsonRowDataDeserializationSchema ignoreErrorsSchema = + new JsonRowDataDeserializationSchema( + spec.rowType, + InternalTypeInfo.of(spec.rowType), + false, + true, + spec.timestampFormat); + Row expected; + if (spec.expected != null) { + expected = spec.expected; + } else { + expected = new Row(1); + } + RowData rowData = ignoreErrorsSchema.deserialize(spec.json.getBytes()); + Row actual = convertToExternal(rowData, fromLogicalToDataType(spec.rowType)); + assertEquals("Test Ignore Parse Error: " + spec.json, expected, actual); + } + + private void testParseErrors(TestSpec spec) throws Exception { + // expect exception if parse error is not ignored + JsonRowDataDeserializationSchema failingSchema = + new JsonRowDataDeserializationSchema( + spec.rowType, + InternalTypeInfo.of(spec.rowType), + false, + false, + spec.timestampFormat); + + try { + failingSchema.deserialize(spec.json.getBytes()); + fail("expecting exception " + spec.errorMessage); + } catch (Throwable t) { + assertEquals(t.getMessage(), spec.errorMessage); + } + } + + private static List testData = + Arrays.asList( + TestSpec.json("{\"id\": \"trueA\"}") + .rowType(ROW(FIELD("id", BOOLEAN()))) + .expect(Row.of(false)), + TestSpec.json("{\"id\": true}") + .rowType(ROW(FIELD("id", BOOLEAN()))) + .expect(Row.of(true)), + TestSpec.json("{\"id\":\"abc\"}") + .rowType(ROW(FIELD("id", INT()))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'."), + TestSpec.json("{\"id\":112.013}") + .rowType(ROW(FIELD("id", BIGINT()))) + .expect(Row.of(112L)), + TestSpec.json("{\"id\":\"long\"}") + .rowType(ROW(FIELD("id", BIGINT()))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'."), + TestSpec.json("{\"id\":\"112.013.123\"}") + .rowType(ROW(FIELD("id", FLOAT()))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'."), + TestSpec.json("{\"id\":\"112.013.123\"}") + .rowType(ROW(FIELD("id", DOUBLE()))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'."), + TestSpec.json("{\"id\":\"18:00:243\"}") + .rowType(ROW(FIELD("id", TIME()))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"18:00:243\"}'."), + TestSpec.json("{\"id\":\"18:00:243\"}") + .rowType(ROW(FIELD("id", TIME()))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"18:00:243\"}'."), + TestSpec.json("{\"id\":\"20191112\"}") + .rowType(ROW(FIELD("id", DATE()))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"20191112\"}'."), + TestSpec.json("{\"id\":\"20191112\"}") + .rowType(ROW(FIELD("id", DATE()))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"20191112\"}'."), + TestSpec.json("{\"id\":true}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("true")), + TestSpec.json("{\"id\":123.234}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("123.234")), + TestSpec.json("{\"id\":1234567}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("1234567")), + TestSpec.json("{\"id\":\"string field\"}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("string field")), + TestSpec.json("{\"id\":[\"array data1\",\"array data2\",123,234.345]}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("[\"array data1\",\"array data2\",123,234.345]")), + TestSpec.json("{\"id\":{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}}") + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}")), + TestSpec.json("{\"id\":\"2019-11-12 18:00:12\"}") + .rowType(ROW(FIELD("id", TIMESTAMP(0)))) + .timestampFormat(TimestampFormat.ISO_8601) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'."), + TestSpec.json("{\"id\":\"2019-11-12T18:00:12\"}") + .rowType(ROW(FIELD("id", TIMESTAMP(0)))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."), + TestSpec.json("{\"id\":\"2019-11-12T18:00:12Z\"}") + .rowType(ROW(FIELD("id", TIMESTAMP(0)))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'."), + TestSpec.json("{\"id\":\"2019-11-12T18:00:12Z\"}") + .rowType(ROW(FIELD("id", TIMESTAMP(0)))) + .timestampFormat(TimestampFormat.ISO_8601) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'."), + TestSpec.json("{\"id\":\"abc\"}") + .rowType(ROW(FIELD("id", DECIMAL(10, 3)))) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'."), + TestSpec.json("{\"row\":{\"id\":\"abc\"}}") + .rowType(ROW(FIELD("row", ROW(FIELD("id", BOOLEAN()))))) + .expect(Row.of(Row.of(false))), + TestSpec.json("{\"array\":[123, \"abc\"]}") + .rowType(ROW(FIELD("array", ARRAY(INT())))) + .expect(Row.of((Object) new Integer[] {123, null})) + .expectErrorMessage( + "Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'."), + TestSpec.json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") + .rowType(ROW(FIELD("map", MAP(STRING(), INT())))) + .expect(Row.of(createHashMap("key1", 123, "key2", null))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'."), + TestSpec.json("{\"id\":\"2019-11-12T18:00:12\"}") + .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0)))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."), + TestSpec.json("{\"id\":\"2019-11-12T18:00:12+0800\"}") + .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0)))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12+0800\"}'."), + TestSpec.json("{\"id\":1,\"factor\":799.929496989092949698}") + .rowType(ROW(FIELD("id", INT()), FIELD("factor", DECIMAL(38, 18)))) + .expect(Row.of(1, new BigDecimal("799.929496989092949698")))); + + private static Map createHashMap( + String k1, Integer v1, String k2, Integer v2) { + Map map = new HashMap<>(); + map.put(k1, v1); + map.put(k2, v2); + return map; + } + + @SuppressWarnings("unchecked") + private static Row convertToExternal(RowData rowData, DataType dataType) { + return (Row) DataFormatConverters.getConverterForDataType(dataType).toExternal(rowData); + } + + private static class TestSpec { + private final String json; + private RowType rowType; + private TimestampFormat timestampFormat = TimestampFormat.SQL; + private Row expected; + private String errorMessage; + + private TestSpec(String json) { + this.json = json; + } + + public static TestSpec json(String json) { + return new TestSpec(json); + } + + TestSpec expect(Row row) { + this.expected = row; + return this; + } + + TestSpec rowType(DataType rowType) { + this.rowType = (RowType) rowType.getLogicalType(); + return this; + } + + TestSpec expectErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + return this; + } + + TestSpec timestampFormat(TimestampFormat timestampFormat) { + this.timestampFormat = timestampFormat; + return this; + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index 88512906c..deee53ae0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -49,393 +49,387 @@ import static org.junit.internal.matchers.ThrowableCauseMatcher.hasCause; import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; -/** - * Tests for the {@link JsonRowDeserializationSchema}. - */ +/** Tests for the {@link JsonRowDeserializationSchema}. */ public class JsonRowDeserializationSchemaTest { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - /** - * Tests simple deserialization using type information. - */ - @Test - public void testTypeInfoDeserialization() throws Exception { - long id = 1238123899121L; - String name = "asdlkjasjkdla998y1122"; - byte[] bytes = new byte[1024]; - ThreadLocalRandom.current().nextBytes(bytes); - Timestamp timestamp = Timestamp.valueOf("1990-10-14 12:12:43"); - Date date = Date.valueOf("1990-10-14"); - Time time = Time.valueOf("12:12:43"); - - Map map = new HashMap<>(); - map.put("flink", 123L); - - Map> nestedMap = new HashMap<>(); - Map innerMap = new HashMap<>(); - innerMap.put("key", 234); - nestedMap.put("inner_map", innerMap); - - ObjectMapper objectMapper = new ObjectMapper(); - - // Root - ObjectNode root = objectMapper.createObjectNode(); - root.put("id", id); - root.put("name", name); - root.put("bytes", bytes); - root.put("date1", "1990-10-14"); - root.put("date2", "1990-10-14"); - root.put("time1", "12:12:43Z"); - root.put("time2", "12:12:43Z"); - root.put("timestamp1", "1990-10-14T12:12:43Z"); - root.put("timestamp2", "1990-10-14T12:12:43Z"); - root.putObject("map").put("flink", 123); - root.putObject("map2map").putObject("inner_map").put("key", 234); - - byte[] serializedJson = objectMapper.writeValueAsBytes(root); - - JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder( - Types.ROW_NAMED( - new String[]{"id", "name", "bytes", "date1", "date2", - "time1", "time2", "timestamp1", "timestamp2", "map", "map2map"}, - Types.LONG, Types.STRING, Types.PRIMITIVE_ARRAY(Types.BYTE), - Types.SQL_DATE, Types.LOCAL_DATE, Types.SQL_TIME, Types.LOCAL_TIME, - Types.SQL_TIMESTAMP, Types.LOCAL_DATE_TIME, - Types.MAP(Types.STRING, Types.LONG), - Types.MAP(Types.STRING, Types.MAP(Types.STRING, Types.INT))) - ).build(); - - Row row = new Row(11); - row.setField(0, id); - row.setField(1, name); - row.setField(2, bytes); - row.setField(3, date); - row.setField(4, date.toLocalDate()); - row.setField(5, time); - row.setField(6, time.toLocalTime()); - row.setField(7, timestamp); - row.setField(8, timestamp.toLocalDateTime()); - row.setField(9, map); - row.setField(10, nestedMap); - - assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); - } - - @Test - public void testSchemaDeserialization() throws Exception { - final BigDecimal id = BigDecimal.valueOf(1238123899121L); - final String name = "asdlkjasjkdla998y1122"; - final byte[] bytes = new byte[1024]; - ThreadLocalRandom.current().nextBytes(bytes); - final BigDecimal[] numbers = new BigDecimal[] { - BigDecimal.valueOf(1), BigDecimal.valueOf(2), BigDecimal.valueOf(3)}; - final String[] strings = new String[] {"one", "two", "three"}; - - final ObjectMapper objectMapper = new ObjectMapper(); - - // Root - ObjectNode root = objectMapper.createObjectNode(); - root.put("id", id.longValue()); - root.putNull("idOrNull"); - root.put("name", name); - root.put("date", "1990-10-14"); - root.put("time", "12:12:43Z"); - root.put("timestamp", "1990-10-14T12:12:43Z"); - root.put("bytes", bytes); - root.putArray("numbers").add(1).add(2).add(3); - root.putArray("strings").add("one").add("two").add("three"); - root.putObject("nested").put("booleanField", true).put("decimalField", 12); - - final byte[] serializedJson = objectMapper.writeValueAsBytes(root); - - JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder( - "{" + - " type: 'object'," + - " properties: {" + - " id: { type: 'integer' }," + - " idOrNull: { type: ['integer', 'null'] }," + - " name: { type: 'string' }," + - " date: { type: 'string', format: 'date' }," + - " time: { type: 'string', format: 'time' }," + - " timestamp: { type: 'string', format: 'date-time' }," + - " bytes: { type: 'string', contentEncoding: 'base64' }," + - " numbers: { type: 'array', items: { type: 'integer' } }," + - " strings: { type: 'array', items: { type: 'string' } }," + - " nested: { " + - " type: 'object'," + - " properties: { " + - " booleanField: { type: 'boolean' }," + - " decimalField: { type: 'number' }" + - " }" + - " }" + - " }" + - "}").build(); - - final Row expected = new Row(10); - expected.setField(0, id); - expected.setField(1, null); - expected.setField(2, name); - expected.setField(3, Date.valueOf("1990-10-14")); - expected.setField(4, Time.valueOf("12:12:43")); - expected.setField(5, Timestamp.valueOf("1990-10-14 12:12:43")); - expected.setField(6, bytes); - expected.setField(7, numbers); - expected.setField(8, strings); - final Row nestedRow = new Row(2); - nestedRow.setField(0, true); - nestedRow.setField(1, BigDecimal.valueOf(12)); - expected.setField(9, nestedRow); - - assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(expected)); - } - - /** - * Tests deserialization with non-existing field name. - */ - @Test - public void testMissingNode() throws Exception { - ObjectMapper objectMapper = new ObjectMapper(); - - // Root - ObjectNode root = objectMapper.createObjectNode(); - root.put("id", 123123123); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); - - TypeInformation rowTypeInformation = Types.ROW_NAMED( - new String[]{"name"}, - Types.STRING); - - JsonRowDeserializationSchema deserializationSchema = - new JsonRowDeserializationSchema.Builder(rowTypeInformation) - .build(); - - Row row = new Row(1); - assertThat(serializedJson, - whenDeserializedWith(deserializationSchema).equalsTo(row)); - - deserializationSchema = new JsonRowDeserializationSchema.Builder(rowTypeInformation) - .failOnMissingField() - .build(); - - assertThat(serializedJson, - whenDeserializedWith(deserializationSchema) - .failsWithException(hasCause(instanceOf(IllegalStateException.class)))); - - // ignore-parse-errors ignores missing field exception too - deserializationSchema = new JsonRowDeserializationSchema.Builder(rowTypeInformation) - .ignoreParseErrors() - .build(); - assertThat(serializedJson, - whenDeserializedWith(deserializationSchema).equalsTo(row)); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("JSON format doesn't support failOnMissingField and ignoreParseErrors are both true"); - new JsonRowDeserializationSchema.Builder(rowTypeInformation) - .failOnMissingField() - .ignoreParseErrors() - .build(); - } - - /** - * Tests that number of field names and types has to match. - */ - @Test - public void testNumberOfFieldNamesAndTypesMismatch() { - try { - new JsonRowDeserializationSchema.Builder( - Types.ROW_NAMED( - new String[]{"one", "two", "three"}, - Types.LONG)).build(); - Assert.fail("Did not throw expected Exception"); - } catch (IllegalArgumentException ignored) { - // Expected - } - } - - @Test - public void testJsonParse() { - for (TestSpec spec : testData) { - testIgnoreParseErrors(spec); - if (spec.errorMessage != null) { - testParseErrors(spec); - } - } - } - - private void testIgnoreParseErrors(TestSpec spec) { - // the parsing field should be null and no exception is thrown - JsonRowDeserializationSchema ignoreErrorsSchema = - new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation) - .ignoreParseErrors() - .build(); - Row expected; - if (spec.expected != null) { - expected = spec.expected; - } else { - expected = new Row(1); - } - assertThat("Test Ignore Parse Error: " + spec.json, - spec.json.getBytes(), - whenDeserializedWith(ignoreErrorsSchema).equalsTo(expected)); - } - - private void testParseErrors(TestSpec spec) { - // expect exception if parse error is not ignored - JsonRowDeserializationSchema failingSchema = - new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation) - .build(); - assertThat("Test Parse Error: " + spec.json, - spec.json.getBytes(), - whenDeserializedWith(failingSchema) - .failsWithException(hasMessage(containsString(spec.errorMessage)))); - } - - private static List testData = Arrays.asList( - TestSpec - .json("{\"id\": \"trueA\"}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.BOOLEAN)) - .expect(Row.of(false)), - - TestSpec - .json("{\"id\": true}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.BOOLEAN)) - .expect(Row.of(true)), - - TestSpec - .json("{\"id\":\"abc\"}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.INT)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), - - TestSpec - .json("{\"id\":112.013}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.LONG)) - .expect(Row.of(112L)), - - TestSpec - .json("{\"id\":true}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) - .expect(Row.of("true")), - - TestSpec - .json("{\"id\":123.234}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) - .expect(Row.of("123.234")), - - TestSpec - .json("{\"id\":1234567}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) - .expect(Row.of("1234567")), - - TestSpec - .json("{\"id\":\"string field\"}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) - .expect(Row.of("string field")), - - TestSpec - .json("{\"id\":[\"array data1\",\"array data2\",123,234.345]}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) - .expect(Row.of("[\"array data1\",\"array data2\",123,234.345]")), - - TestSpec - .json("{\"id\":{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.STRING)) - .expect(Row.of("{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}")), - - TestSpec - .json("{\"id\":\"long\"}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.LONG)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'"), - - TestSpec - .json("{\"id\":\"112.013.123\"}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.FLOAT)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), - - TestSpec - .json("{\"id\":\"112.013.123\"}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.DOUBLE)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), - - TestSpec - .json("{\"id\":\"18:00:243\"}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.SQL_TIME)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"), - - TestSpec - .json("{\"id\":\"20191112\"}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.SQL_DATE)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'"), - - TestSpec - .json("{\"id\":\"2019-11-12 18:00:12\"}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.SQL_TIMESTAMP)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'"), - - TestSpec - .json("{\"id\":\"abc\"}") - .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.BIG_DEC)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), - - TestSpec - .json("{\"row\":{\"id\":\"abc\"}}") - .typeInfo(Types.ROW_NAMED(new String[]{"row"}, Types.ROW_NAMED(new String[]{"id"}, Types.INT))) - .expect(Row.of(new Row(1))) - .expectErrorMessage("Failed to deserialize JSON '{\"row\":{\"id\":\"abc\"}}'"), - - TestSpec - .json("{\"array\":[123, \"abc\"]}") - .typeInfo(Types.ROW_NAMED(new String[]{"array"}, Types.OBJECT_ARRAY(Types.INT))) - .expect(Row.of((Object) new Integer[]{123, null})) - .expectErrorMessage("Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'"), - - TestSpec - .json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") - .typeInfo(Types.ROW_NAMED(new String[]{"map"}, Types.MAP(Types.STRING, Types.INT))) - .expect(Row.of(createHashMap("key1", 123, "key2", null))) - .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'"), - - TestSpec - .json("{\"id\":1,\"factor\":799.929496989092949698}") - .typeInfo(Types.ROW_NAMED(new String[]{"id", "factor"}, Types.INT, Types.BIG_DEC)) - .expect(Row.of(1, new BigDecimal("799.929496989092949698"))) - ); - - private static Map createHashMap(String k1, Integer v1, String k2, Integer v2) { - Map map = new HashMap<>(); - map.put(k1, v1); - map.put(k2, v2); - return map; - } - - private static class TestSpec { - private final String json; - private @Nullable TypeInformation rowTypeInformation; - private @Nullable Row expected; - private @Nullable String errorMessage; - - private TestSpec(String json) { - this.json = json; - } - - public static TestSpec json(String json) { - return new TestSpec(json); - } - - TestSpec expect(Row row) { - this.expected = row; - return this; - } - - TestSpec typeInfo(TypeInformation rowTypeInformation) { - this.rowTypeInformation = rowTypeInformation; - return this; - } - - TestSpec expectErrorMessage(String errorMessage) { - this.errorMessage = errorMessage; - return this; - } - } + @Rule public ExpectedException thrown = ExpectedException.none(); + + /** Tests simple deserialization using type information. */ + @Test + public void testTypeInfoDeserialization() throws Exception { + long id = 1238123899121L; + String name = "asdlkjasjkdla998y1122"; + byte[] bytes = new byte[1024]; + ThreadLocalRandom.current().nextBytes(bytes); + Timestamp timestamp = Timestamp.valueOf("1990-10-14 12:12:43"); + Date date = Date.valueOf("1990-10-14"); + Time time = Time.valueOf("12:12:43"); + + Map map = new HashMap<>(); + map.put("flink", 123L); + + Map> nestedMap = new HashMap<>(); + Map innerMap = new HashMap<>(); + innerMap.put("key", 234); + nestedMap.put("inner_map", innerMap); + + ObjectMapper objectMapper = new ObjectMapper(); + + // Root + ObjectNode root = objectMapper.createObjectNode(); + root.put("id", id); + root.put("name", name); + root.put("bytes", bytes); + root.put("date1", "1990-10-14"); + root.put("date2", "1990-10-14"); + root.put("time1", "12:12:43Z"); + root.put("time2", "12:12:43Z"); + root.put("timestamp1", "1990-10-14T12:12:43Z"); + root.put("timestamp2", "1990-10-14T12:12:43Z"); + root.putObject("map").put("flink", 123); + root.putObject("map2map").putObject("inner_map").put("key", 234); + + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + JsonRowDeserializationSchema deserializationSchema = + new JsonRowDeserializationSchema.Builder( + Types.ROW_NAMED( + new String[] { + "id", + "name", + "bytes", + "date1", + "date2", + "time1", + "time2", + "timestamp1", + "timestamp2", + "map", + "map2map" + }, + Types.LONG, + Types.STRING, + Types.PRIMITIVE_ARRAY(Types.BYTE), + Types.SQL_DATE, + Types.LOCAL_DATE, + Types.SQL_TIME, + Types.LOCAL_TIME, + Types.SQL_TIMESTAMP, + Types.LOCAL_DATE_TIME, + Types.MAP(Types.STRING, Types.LONG), + Types.MAP( + Types.STRING, Types.MAP(Types.STRING, Types.INT)))) + .build(); + + Row row = new Row(11); + row.setField(0, id); + row.setField(1, name); + row.setField(2, bytes); + row.setField(3, date); + row.setField(4, date.toLocalDate()); + row.setField(5, time); + row.setField(6, time.toLocalTime()); + row.setField(7, timestamp); + row.setField(8, timestamp.toLocalDateTime()); + row.setField(9, map); + row.setField(10, nestedMap); + + assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); + } + + @Test + public void testSchemaDeserialization() throws Exception { + final BigDecimal id = BigDecimal.valueOf(1238123899121L); + final String name = "asdlkjasjkdla998y1122"; + final byte[] bytes = new byte[1024]; + ThreadLocalRandom.current().nextBytes(bytes); + final BigDecimal[] numbers = + new BigDecimal[] { + BigDecimal.valueOf(1), BigDecimal.valueOf(2), BigDecimal.valueOf(3) + }; + final String[] strings = new String[] {"one", "two", "three"}; + + final ObjectMapper objectMapper = new ObjectMapper(); + + // Root + ObjectNode root = objectMapper.createObjectNode(); + root.put("id", id.longValue()); + root.putNull("idOrNull"); + root.put("name", name); + root.put("date", "1990-10-14"); + root.put("time", "12:12:43Z"); + root.put("timestamp", "1990-10-14T12:12:43Z"); + root.put("bytes", bytes); + root.putArray("numbers").add(1).add(2).add(3); + root.putArray("strings").add("one").add("two").add("three"); + root.putObject("nested").put("booleanField", true).put("decimalField", 12); + + final byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + JsonRowDeserializationSchema deserializationSchema = + new JsonRowDeserializationSchema.Builder( + "{" + + " type: 'object'," + + " properties: {" + + " id: { type: 'integer' }," + + " idOrNull: { type: ['integer', 'null'] }," + + " name: { type: 'string' }," + + " date: { type: 'string', format: 'date' }," + + " time: { type: 'string', format: 'time' }," + + " timestamp: { type: 'string', format: 'date-time' }," + + " bytes: { type: 'string', contentEncoding: 'base64' }," + + " numbers: { type: 'array', items: { type: 'integer' } }," + + " strings: { type: 'array', items: { type: 'string' } }," + + " nested: { " + + " type: 'object'," + + " properties: { " + + " booleanField: { type: 'boolean' }," + + " decimalField: { type: 'number' }" + + " }" + + " }" + + " }" + + "}") + .build(); + + final Row expected = new Row(10); + expected.setField(0, id); + expected.setField(1, null); + expected.setField(2, name); + expected.setField(3, Date.valueOf("1990-10-14")); + expected.setField(4, Time.valueOf("12:12:43")); + expected.setField(5, Timestamp.valueOf("1990-10-14 12:12:43")); + expected.setField(6, bytes); + expected.setField(7, numbers); + expected.setField(8, strings); + final Row nestedRow = new Row(2); + nestedRow.setField(0, true); + nestedRow.setField(1, BigDecimal.valueOf(12)); + expected.setField(9, nestedRow); + + assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(expected)); + } + + /** Tests deserialization with non-existing field name. */ + @Test + public void testMissingNode() throws Exception { + ObjectMapper objectMapper = new ObjectMapper(); + + // Root + ObjectNode root = objectMapper.createObjectNode(); + root.put("id", 123123123); + byte[] serializedJson = objectMapper.writeValueAsBytes(root); + + TypeInformation rowTypeInformation = + Types.ROW_NAMED(new String[] {"name"}, Types.STRING); + + JsonRowDeserializationSchema deserializationSchema = + new JsonRowDeserializationSchema.Builder(rowTypeInformation).build(); + + Row row = new Row(1); + assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); + + deserializationSchema = + new JsonRowDeserializationSchema.Builder(rowTypeInformation) + .failOnMissingField() + .build(); + + assertThat( + serializedJson, + whenDeserializedWith(deserializationSchema) + .failsWithException(hasCause(instanceOf(IllegalStateException.class)))); + + // ignore-parse-errors ignores missing field exception too + deserializationSchema = + new JsonRowDeserializationSchema.Builder(rowTypeInformation) + .ignoreParseErrors() + .build(); + assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "JSON format doesn't support failOnMissingField and ignoreParseErrors are both true"); + new JsonRowDeserializationSchema.Builder(rowTypeInformation) + .failOnMissingField() + .ignoreParseErrors() + .build(); + } + + /** Tests that number of field names and types has to match. */ + @Test + public void testNumberOfFieldNamesAndTypesMismatch() { + try { + new JsonRowDeserializationSchema.Builder( + Types.ROW_NAMED(new String[] {"one", "two", "three"}, Types.LONG)) + .build(); + Assert.fail("Did not throw expected Exception"); + } catch (IllegalArgumentException ignored) { + // Expected + } + } + + @Test + public void testJsonParse() { + for (TestSpec spec : testData) { + testIgnoreParseErrors(spec); + if (spec.errorMessage != null) { + testParseErrors(spec); + } + } + } + + private void testIgnoreParseErrors(TestSpec spec) { + // the parsing field should be null and no exception is thrown + JsonRowDeserializationSchema ignoreErrorsSchema = + new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation) + .ignoreParseErrors() + .build(); + Row expected; + if (spec.expected != null) { + expected = spec.expected; + } else { + expected = new Row(1); + } + assertThat( + "Test Ignore Parse Error: " + spec.json, + spec.json.getBytes(), + whenDeserializedWith(ignoreErrorsSchema).equalsTo(expected)); + } + + private void testParseErrors(TestSpec spec) { + // expect exception if parse error is not ignored + JsonRowDeserializationSchema failingSchema = + new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation).build(); + assertThat( + "Test Parse Error: " + spec.json, + spec.json.getBytes(), + whenDeserializedWith(failingSchema) + .failsWithException(hasMessage(containsString(spec.errorMessage)))); + } + + private static List testData = + Arrays.asList( + TestSpec.json("{\"id\": \"trueA\"}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.BOOLEAN)) + .expect(Row.of(false)), + TestSpec.json("{\"id\": true}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.BOOLEAN)) + .expect(Row.of(true)), + TestSpec.json("{\"id\":\"abc\"}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.INT)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), + TestSpec.json("{\"id\":112.013}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.LONG)) + .expect(Row.of(112L)), + TestSpec.json("{\"id\":true}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) + .expect(Row.of("true")), + TestSpec.json("{\"id\":123.234}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) + .expect(Row.of("123.234")), + TestSpec.json("{\"id\":1234567}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) + .expect(Row.of("1234567")), + TestSpec.json("{\"id\":\"string field\"}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) + .expect(Row.of("string field")), + TestSpec.json("{\"id\":[\"array data1\",\"array data2\",123,234.345]}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) + .expect(Row.of("[\"array data1\",\"array data2\",123,234.345]")), + TestSpec.json("{\"id\":{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) + .expect(Row.of("{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}")), + TestSpec.json("{\"id\":\"long\"}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.LONG)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'"), + TestSpec.json("{\"id\":\"112.013.123\"}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.FLOAT)) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), + TestSpec.json("{\"id\":\"112.013.123\"}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.DOUBLE)) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), + TestSpec.json("{\"id\":\"18:00:243\"}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.SQL_TIME)) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"), + TestSpec.json("{\"id\":\"20191112\"}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.SQL_DATE)) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"20191112\"}'"), + TestSpec.json("{\"id\":\"2019-11-12 18:00:12\"}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.SQL_TIMESTAMP)) + .expectErrorMessage( + "Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'"), + TestSpec.json("{\"id\":\"abc\"}") + .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.BIG_DEC)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), + TestSpec.json("{\"row\":{\"id\":\"abc\"}}") + .typeInfo( + Types.ROW_NAMED( + new String[] {"row"}, + Types.ROW_NAMED(new String[] {"id"}, Types.INT))) + .expect(Row.of(new Row(1))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"row\":{\"id\":\"abc\"}}'"), + TestSpec.json("{\"array\":[123, \"abc\"]}") + .typeInfo( + Types.ROW_NAMED( + new String[] {"array"}, Types.OBJECT_ARRAY(Types.INT))) + .expect(Row.of((Object) new Integer[] {123, null})) + .expectErrorMessage( + "Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'"), + TestSpec.json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") + .typeInfo( + Types.ROW_NAMED( + new String[] {"map"}, + Types.MAP(Types.STRING, Types.INT))) + .expect(Row.of(createHashMap("key1", 123, "key2", null))) + .expectErrorMessage( + "Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'"), + TestSpec.json("{\"id\":1,\"factor\":799.929496989092949698}") + .typeInfo( + Types.ROW_NAMED( + new String[] {"id", "factor"}, + Types.INT, + Types.BIG_DEC)) + .expect(Row.of(1, new BigDecimal("799.929496989092949698")))); + + private static Map createHashMap( + String k1, Integer v1, String k2, Integer v2) { + Map map = new HashMap<>(); + map.put(k1, v1); + map.put(k2, v2); + return map; + } + + private static class TestSpec { + private final String json; + private @Nullable TypeInformation rowTypeInformation; + private @Nullable Row expected; + private @Nullable String errorMessage; + + private TestSpec(String json) { + this.json = json; + } + + public static TestSpec json(String json) { + return new TestSpec(json); + } + + TestSpec expect(Row row) { + this.expected = row; + return this; + } + + TestSpec typeInfo(TypeInformation rowTypeInformation) { + this.rowTypeInformation = rowTypeInformation; + return this; + } + + TestSpec expectErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + return this; + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java index 28beacbb8..5c0ecf640 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -38,145 +38,144 @@ import static org.junit.Assert.assertEquals; -/** - * Tests for the {@link JsonRowFormatFactory}. - */ +/** Tests for the {@link JsonRowFormatFactory}. */ public class JsonRowFormatFactoryTest extends TestLogger { - private static final String JSON_SCHEMA = - "{" + - " 'title': 'Fruit'," + - " 'type': 'object'," + - " 'properties': {" + - " 'name': {" + - " 'type': 'string'" + - " }," + - " 'count': {" + - " 'type': 'integer'" + - " }," + - " 'time': {" + - " 'description': 'row time'," + - " 'type': 'string'," + - " 'format': 'date-time'" + - " }" + - " }," + - " 'required': ['name', 'count', 'time']" + - "}"; - - private static final TypeInformation SCHEMA = Types.ROW( - new String[]{"field1", "field2"}, - new TypeInformation[]{Types.BOOLEAN(), Types.INT()}); - - @Test - public void testSchema() { - final Map properties = toMap( - new Json() - .schema(SCHEMA) - .failOnMissingField(false)); - - testSchemaSerializationSchema(properties); - - testSchemaDeserializationSchema(properties); - } - - @Test - public void testSchemaIgnoreParseErrors() { - final Map properties = toMap( - new Json() - .schema(SCHEMA) - .ignoreParseErrors(true)); - - testSchemaSerializationSchema(properties); - - final DeserializationSchema actual2 = TableFactoryService - .find(DeserializationSchemaFactory.class, properties) - .createDeserializationSchema(properties); - final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema.Builder(SCHEMA).ignoreParseErrors().build(); - assertEquals(expected2, actual2); - } - - @Test - public void testJsonSchema() { - final Map properties = toMap( - new Json() - .jsonSchema(JSON_SCHEMA) - .failOnMissingField(true)); - - testJsonSchemaSerializationSchema(properties); - - testJsonSchemaDeserializationSchema(properties); - } - - @Test - public void testSchemaDerivation() { - final Map properties = toMap( - new Schema() - .field("field1", Types.BOOLEAN()) - .field("field2", Types.INT()) - .field("proctime", Types.SQL_TIMESTAMP()).proctime(), - new Json() - .deriveSchema()); - - testSchemaSerializationSchema(properties); - - testSchemaDeserializationSchema(properties); - } - - @Test - public void testSchemaDerivationByDefault() { - final Map properties = toMap( - new Schema() - .field("field1", Types.BOOLEAN()) - .field("field2", Types.INT()) - .field("proctime", Types.SQL_TIMESTAMP()).proctime(), - new Json()); - - testSchemaSerializationSchema(properties); - - testSchemaDeserializationSchema(properties); - } - - private void testSchemaDeserializationSchema(Map properties) { - final DeserializationSchema actual2 = TableFactoryService - .find(DeserializationSchemaFactory.class, properties) - .createDeserializationSchema(properties); - final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema.Builder(SCHEMA).build(); - assertEquals(expected2, actual2); - } - - private void testSchemaSerializationSchema(Map properties) { - final SerializationSchema actual1 = TableFactoryService - .find(SerializationSchemaFactory.class, properties) - .createSerializationSchema(properties); - final SerializationSchema expected1 = new JsonRowSerializationSchema.Builder(SCHEMA).build(); - assertEquals(expected1, actual1); - } - - private void testJsonSchemaDeserializationSchema(Map properties) { - final DeserializationSchema actual2 = TableFactoryService - .find(DeserializationSchemaFactory.class, properties) - .createDeserializationSchema(properties); - final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema.Builder(JSON_SCHEMA) - .failOnMissingField() - .build(); - assertEquals(expected2, actual2); - } - - private void testJsonSchemaSerializationSchema(Map properties) { - final SerializationSchema actual1 = TableFactoryService - .find(SerializationSchemaFactory.class, properties) - .createSerializationSchema(properties); - final SerializationSchema expected1 = JsonRowSerializationSchema.builder() - .withTypeInfo(JsonRowSchemaConverter.convert(JSON_SCHEMA)) - .build(); - assertEquals(expected1, actual1); - } - - private static Map toMap(Descriptor... desc) { - final DescriptorProperties descriptorProperties = new DescriptorProperties(); - for (Descriptor d : desc) { - descriptorProperties.putProperties(d.toProperties()); - } - return descriptorProperties.asMap(); - } + private static final String JSON_SCHEMA = + "{" + + " 'title': 'Fruit'," + + " 'type': 'object'," + + " 'properties': {" + + " 'name': {" + + " 'type': 'string'" + + " }," + + " 'count': {" + + " 'type': 'integer'" + + " }," + + " 'time': {" + + " 'description': 'row time'," + + " 'type': 'string'," + + " 'format': 'date-time'" + + " }" + + " }," + + " 'required': ['name', 'count', 'time']" + + "}"; + + private static final TypeInformation SCHEMA = + Types.ROW( + new String[] {"field1", "field2"}, + new TypeInformation[] {Types.BOOLEAN(), Types.INT()}); + + @Test + public void testSchema() { + final Map properties = + toMap(new Json().schema(SCHEMA).failOnMissingField(false)); + + testSchemaSerializationSchema(properties); + + testSchemaDeserializationSchema(properties); + } + + @Test + public void testSchemaIgnoreParseErrors() { + final Map properties = + toMap(new Json().schema(SCHEMA).ignoreParseErrors(true)); + + testSchemaSerializationSchema(properties); + + final DeserializationSchema actual2 = + TableFactoryService.find(DeserializationSchemaFactory.class, properties) + .createDeserializationSchema(properties); + final JsonRowDeserializationSchema expected2 = + new JsonRowDeserializationSchema.Builder(SCHEMA).ignoreParseErrors().build(); + assertEquals(expected2, actual2); + } + + @Test + public void testJsonSchema() { + final Map properties = + toMap(new Json().jsonSchema(JSON_SCHEMA).failOnMissingField(true)); + + testJsonSchemaSerializationSchema(properties); + + testJsonSchemaDeserializationSchema(properties); + } + + @Test + public void testSchemaDerivation() { + final Map properties = + toMap( + new Schema() + .field("field1", Types.BOOLEAN()) + .field("field2", Types.INT()) + .field("proctime", Types.SQL_TIMESTAMP()) + .proctime(), + new Json().deriveSchema()); + + testSchemaSerializationSchema(properties); + + testSchemaDeserializationSchema(properties); + } + + @Test + public void testSchemaDerivationByDefault() { + final Map properties = + toMap( + new Schema() + .field("field1", Types.BOOLEAN()) + .field("field2", Types.INT()) + .field("proctime", Types.SQL_TIMESTAMP()) + .proctime(), + new Json()); + + testSchemaSerializationSchema(properties); + + testSchemaDeserializationSchema(properties); + } + + private void testSchemaDeserializationSchema(Map properties) { + final DeserializationSchema actual2 = + TableFactoryService.find(DeserializationSchemaFactory.class, properties) + .createDeserializationSchema(properties); + final JsonRowDeserializationSchema expected2 = + new JsonRowDeserializationSchema.Builder(SCHEMA).build(); + assertEquals(expected2, actual2); + } + + private void testSchemaSerializationSchema(Map properties) { + final SerializationSchema actual1 = + TableFactoryService.find(SerializationSchemaFactory.class, properties) + .createSerializationSchema(properties); + final SerializationSchema expected1 = + new JsonRowSerializationSchema.Builder(SCHEMA).build(); + assertEquals(expected1, actual1); + } + + private void testJsonSchemaDeserializationSchema(Map properties) { + final DeserializationSchema actual2 = + TableFactoryService.find(DeserializationSchemaFactory.class, properties) + .createDeserializationSchema(properties); + final JsonRowDeserializationSchema expected2 = + new JsonRowDeserializationSchema.Builder(JSON_SCHEMA).failOnMissingField().build(); + assertEquals(expected2, actual2); + } + + private void testJsonSchemaSerializationSchema(Map properties) { + final SerializationSchema actual1 = + TableFactoryService.find(SerializationSchemaFactory.class, properties) + .createSerializationSchema(properties); + final SerializationSchema expected1 = + JsonRowSerializationSchema.builder() + .withTypeInfo(JsonRowSchemaConverter.convert(JSON_SCHEMA)) + .build(); + assertEquals(expected1, actual1); + } + + private static Map toMap(Descriptor... desc) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(); + for (Descriptor d : desc) { + descriptorProperties.putProperties(d.toProperties()); + } + return descriptorProperties.asMap(); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java index 1af45f4e8..74d4d03da 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java @@ -30,85 +30,117 @@ import static org.junit.Assert.assertEquals; -/** - * Tests for {@link JsonRowSchemaConverter}. - */ +/** Tests for {@link JsonRowSchemaConverter}. */ public class JsonRowSchemaConverterTest { - @Test - public void testComplexSchema() throws Exception { - final URL url = getClass().getClassLoader().getResource("complex-schema.json"); - Objects.requireNonNull(url); - final String schema = FileUtils.readFileUtf8(new File(url.getFile())); - final TypeInformation result = JsonRowSchemaConverter.convert(schema); - - final TypeInformation expected = Types.ROW_NAMED( - new String[] {"fn", "familyName", "additionalName", "tuples", "honorificPrefix", "url", - "email", "tel", "sound", "org"}, - Types.STRING, Types.STRING, Types.BOOLEAN, Types.ROW(Types.BIG_DEC, Types.STRING, Types.STRING, Types.STRING), - Types.OBJECT_ARRAY(Types.STRING), Types.STRING, Types.ROW_NAMED(new String[] {"type", "value"}, Types.STRING, Types.STRING), - Types.ROW_NAMED(new String[] {"type", "value"}, Types.BIG_DEC, Types.STRING), Types.VOID, - Types.ROW_NAMED(new String[] {"organizationUnit"}, Types.ROW())); - - assertEquals(expected, result); - } - - @Test - public void testReferenceSchema() throws Exception { - final URL url = getClass().getClassLoader().getResource("reference-schema.json"); - Objects.requireNonNull(url); - final String schema = FileUtils.readFileUtf8(new File(url.getFile())); - final TypeInformation result = JsonRowSchemaConverter.convert(schema); - - final TypeInformation expected = Types.ROW_NAMED( - new String[] {"billing_address", "shipping_address", "optional_address"}, - Types.ROW_NAMED(new String[] {"street_address", "city", "state"}, Types.STRING, Types.STRING, Types.STRING), - Types.ROW_NAMED(new String[] {"street_address", "city", "state"}, Types.STRING, Types.STRING, Types.STRING), - Types.ROW_NAMED(new String[] {"street_address", "city", "state"}, Types.STRING, Types.STRING, Types.STRING)); - - assertEquals(expected, result); - } - - @Test - public void testAtomicType() { - final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'number' }"); - - assertEquals(Types.BIG_DEC, result); - } - - @Test(expected = IllegalArgumentException.class) - public void testMissingType() { - JsonRowSchemaConverter.convert("{ }"); - } - - @Test(expected = IllegalArgumentException.class) - public void testWrongType() { - JsonRowSchemaConverter.convert("{ type: 'whatever' }"); - } - - @Test(expected = IllegalArgumentException.class) - public void testArrayWithAdditionalItems() { - JsonRowSchemaConverter.convert("{ type: 'array', items: [{type: 'integer'}], additionalItems: true }"); - } - - @Test - public void testMissingProperties() { - final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'object' }"); - - assertEquals(Types.ROW(), result); - } - - @Test - public void testNullUnionTypes() { - final TypeInformation result = JsonRowSchemaConverter.convert("{ type: ['string', 'null'] }"); - - assertEquals(Types.STRING, result); - } - - @Test - public void testTimestamp() { - final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'string', format: 'date-time' }"); - - assertEquals(Types.SQL_TIMESTAMP, result); - } + @Test + public void testComplexSchema() throws Exception { + final URL url = getClass().getClassLoader().getResource("complex-schema.json"); + Objects.requireNonNull(url); + final String schema = FileUtils.readFileUtf8(new File(url.getFile())); + final TypeInformation result = JsonRowSchemaConverter.convert(schema); + + final TypeInformation expected = + Types.ROW_NAMED( + new String[] { + "fn", + "familyName", + "additionalName", + "tuples", + "honorificPrefix", + "url", + "email", + "tel", + "sound", + "org" + }, + Types.STRING, + Types.STRING, + Types.BOOLEAN, + Types.ROW(Types.BIG_DEC, Types.STRING, Types.STRING, Types.STRING), + Types.OBJECT_ARRAY(Types.STRING), + Types.STRING, + Types.ROW_NAMED(new String[] {"type", "value"}, Types.STRING, Types.STRING), + Types.ROW_NAMED( + new String[] {"type", "value"}, Types.BIG_DEC, Types.STRING), + Types.VOID, + Types.ROW_NAMED(new String[] {"organizationUnit"}, Types.ROW())); + + assertEquals(expected, result); + } + + @Test + public void testReferenceSchema() throws Exception { + final URL url = getClass().getClassLoader().getResource("reference-schema.json"); + Objects.requireNonNull(url); + final String schema = FileUtils.readFileUtf8(new File(url.getFile())); + final TypeInformation result = JsonRowSchemaConverter.convert(schema); + + final TypeInformation expected = + Types.ROW_NAMED( + new String[] {"billing_address", "shipping_address", "optional_address"}, + Types.ROW_NAMED( + new String[] {"street_address", "city", "state"}, + Types.STRING, + Types.STRING, + Types.STRING), + Types.ROW_NAMED( + new String[] {"street_address", "city", "state"}, + Types.STRING, + Types.STRING, + Types.STRING), + Types.ROW_NAMED( + new String[] {"street_address", "city", "state"}, + Types.STRING, + Types.STRING, + Types.STRING)); + + assertEquals(expected, result); + } + + @Test + public void testAtomicType() { + final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'number' }"); + + assertEquals(Types.BIG_DEC, result); + } + + @Test(expected = IllegalArgumentException.class) + public void testMissingType() { + JsonRowSchemaConverter.convert("{ }"); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongType() { + JsonRowSchemaConverter.convert("{ type: 'whatever' }"); + } + + @Test(expected = IllegalArgumentException.class) + public void testArrayWithAdditionalItems() { + JsonRowSchemaConverter.convert( + "{ type: 'array', items: [{type: 'integer'}], additionalItems: true }"); + } + + @Test + public void testMissingProperties() { + final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'object' }"); + + assertEquals(Types.ROW(), result); + } + + @Test + public void testNullUnionTypes() { + final TypeInformation result = + JsonRowSchemaConverter.convert("{ type: ['string', 'null'] }"); + + assertEquals(Types.STRING, result); + } + + @Test + public void testTimestamp() { + final TypeInformation result = + JsonRowSchemaConverter.convert("{ type: 'string', format: 'date-time' }"); + + assertEquals(Types.SQL_TIMESTAMP, result); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java index 378f92b89..e9c46be1e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java @@ -35,191 +35,210 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; -/** - * Tests for the {@link JsonRowSerializationSchema}. - */ +/** Tests for the {@link JsonRowSerializationSchema}. */ public class JsonRowSerializationSchemaTest { - @Test - public void testRowSerialization() { - final TypeInformation rowSchema = Types.ROW_NAMED( - new String[] {"f1", "f2", "f3", "f4", "f5"}, - Types.INT, Types.BOOLEAN, Types.STRING, Types.SQL_TIMESTAMP, Types.LOCAL_DATE_TIME); - - final Row row = new Row(5); - row.setField(0, 1); - row.setField(1, true); - row.setField(2, "str"); - row.setField(3, Timestamp.valueOf("1990-10-14 12:12:43")); - row.setField(4, Timestamp.valueOf("1990-10-14 12:12:43").toLocalDateTime()); - - final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) - .build(); - final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(rowSchema) - .build(); - - assertThat(row, whenSerializedWith(serializationSchema) - .andDeserializedWith(deserializationSchema) - .equalsTo(row)); - } - - @Test - public void testSerializationOfTwoRows() throws IOException { - final TypeInformation rowSchema = Types.ROW_NAMED( - new String[] {"f1", "f2", "f3"}, - Types.INT, Types.BOOLEAN, Types.STRING); - - final Row row1 = new Row(3); - row1.setField(0, 1); - row1.setField(1, true); - row1.setField(2, "str"); - - final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) - .build(); - final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(rowSchema) - .build(); - - byte[] bytes = serializationSchema.serialize(row1); - assertEquals(row1, deserializationSchema.deserialize(bytes)); - - final Row row2 = new Row(3); - row2.setField(0, 10); - row2.setField(1, false); - row2.setField(2, "newStr"); - - bytes = serializationSchema.serialize(row2); - assertEquals(row2, deserializationSchema.deserialize(bytes)); - } - - @Test - public void testMultiRowsWithNullValues() throws IOException { - String[] jsons = new String[] { - "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, " + - "\"ids\":[1, 2, 3]}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}", - }; - - String[] expected = new String[] { - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}," + - "\"ids\":[1,2,3]}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}", - }; - - TypeInformation schema = Types.ROW_NAMED( - new String[]{"svt", "ops", "ids"}, - Types.STRING, - Types.ROW_NAMED(new String[]{"id"}, Types.STRING), - Types.PRIMITIVE_ARRAY(Types.INT)); - JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(schema) - .build(); - JsonRowSerializationSchema serializationSchema = JsonRowSerializationSchema.builder() - .withTypeInfo(schema) - .build(); - - for (int i = 0; i < jsons.length; i++) { - String json = jsons[i]; - Row row = deserializationSchema.deserialize(json.getBytes()); - String result = new String(serializationSchema.serialize(row)); - assertEquals(expected[i], result); - } - } - - @Test - public void testNestedSchema() { - final TypeInformation rowSchema = Types.ROW_NAMED( - new String[] {"f1", "f2", "f3"}, - Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.DOUBLE)); - - final Row row = new Row(3); - row.setField(0, 42); - row.setField(1, false); - final Row nested = new Row(2); - nested.setField(0, 22); - nested.setField(1, 2.3); - row.setField(2, nested); - - final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) - .build(); - final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(rowSchema) - .build(); - - assertThat(row, whenSerializedWith(serializationSchema) - .andDeserializedWith(deserializationSchema) - .equalsTo(row)); - } - - @Test - public void testSerializeRowWithInvalidNumberOfFields() { - final TypeInformation rowSchema = Types.ROW_NAMED( - new String[]{"f1", "f2", "f3"}, - Types.INT, Types.BOOLEAN, Types.STRING); - - final Row row = new Row(1); - row.setField(0, 1); - - final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) - .build(); - assertThat(row, whenSerializedWith(serializationSchema).failsWithException(instanceOf(RuntimeException.class))); - } - - @Test - public void testSchema() { - final TypeInformation rowSchema = JsonRowSchemaConverter.convert( - "{" + - " type: 'object'," + - " properties: {" + - " id: { type: 'integer' }," + - " idNumber: { type: 'number' }," + - " idOrNull: { type: ['integer', 'null'] }," + - " name: { type: 'string' }," + - " date: { type: 'string', format: 'date' }," + - " time: { type: 'string', format: 'time' }," + - " timestamp: { type: 'string', format: 'date-time' }," + - " bytes: { type: 'string', contentEncoding: 'base64' }," + - " numbers: { type: 'array', items: { type: 'integer' } }," + - " strings: { type: 'array', items: { type: 'string' } }," + - " nested: { " + - " type: 'object'," + - " properties: { " + - " booleanField: { type: 'boolean' }," + - " decimalField: { type: 'number' }" + - " }" + - " }" + - " }" + - "}"); - - final Row row = new Row(11); - row.setField(0, BigDecimal.valueOf(-333)); - row.setField(1, BigDecimal.valueOf(12.2222)); - row.setField(2, null); - row.setField(3, ""); - row.setField(4, Date.valueOf("1990-10-14")); - row.setField(5, Time.valueOf("12:12:43")); - row.setField(6, Timestamp.valueOf("1990-10-14 12:12:43")); - - final byte[] bytes = new byte[1024]; - ThreadLocalRandom.current().nextBytes(bytes); - row.setField(7, bytes); - final BigDecimal[] numbers = new BigDecimal[] { - BigDecimal.valueOf(1), BigDecimal.valueOf(2), BigDecimal.valueOf(3)}; - row.setField(8, numbers); - final String[] strings = new String[] {"one", "two", "three"}; - row.setField(9, strings); - final Row nestedRow = new Row(2); - nestedRow.setField(0, true); - nestedRow.setField(1, BigDecimal.valueOf(12)); - row.setField(10, nestedRow); - - final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema) - .build(); - final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(rowSchema) - .build(); - - assertThat(row, whenSerializedWith(serializationSchema) - .andDeserializedWith(deserializationSchema) - .equalsTo(row)); - } - + @Test + public void testRowSerialization() { + final TypeInformation rowSchema = + Types.ROW_NAMED( + new String[] {"f1", "f2", "f3", "f4", "f5"}, + Types.INT, + Types.BOOLEAN, + Types.STRING, + Types.SQL_TIMESTAMP, + Types.LOCAL_DATE_TIME); + + final Row row = new Row(5); + row.setField(0, 1); + row.setField(1, true); + row.setField(2, "str"); + row.setField(3, Timestamp.valueOf("1990-10-14 12:12:43")); + row.setField(4, Timestamp.valueOf("1990-10-14 12:12:43").toLocalDateTime()); + + final JsonRowSerializationSchema serializationSchema = + new JsonRowSerializationSchema.Builder(rowSchema).build(); + final JsonRowDeserializationSchema deserializationSchema = + new JsonRowDeserializationSchema.Builder(rowSchema).build(); + + assertThat( + row, + whenSerializedWith(serializationSchema) + .andDeserializedWith(deserializationSchema) + .equalsTo(row)); + } + + @Test + public void testSerializationOfTwoRows() throws IOException { + final TypeInformation rowSchema = + Types.ROW_NAMED( + new String[] {"f1", "f2", "f3"}, Types.INT, Types.BOOLEAN, Types.STRING); + + final Row row1 = new Row(3); + row1.setField(0, 1); + row1.setField(1, true); + row1.setField(2, "str"); + + final JsonRowSerializationSchema serializationSchema = + new JsonRowSerializationSchema.Builder(rowSchema).build(); + final JsonRowDeserializationSchema deserializationSchema = + new JsonRowDeserializationSchema.Builder(rowSchema).build(); + + byte[] bytes = serializationSchema.serialize(row1); + assertEquals(row1, deserializationSchema.deserialize(bytes)); + + final Row row2 = new Row(3); + row2.setField(0, 10); + row2.setField(1, false); + row2.setField(2, "newStr"); + + bytes = serializationSchema.serialize(row2); + assertEquals(row2, deserializationSchema.deserialize(bytes)); + } + + @Test + public void testMultiRowsWithNullValues() throws IOException { + String[] jsons = + new String[] { + "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, " + + "\"ids\":[1, 2, 3]}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}", + }; + + String[] expected = + new String[] { + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}," + + "\"ids\":[1,2,3]}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}", + }; + + TypeInformation schema = + Types.ROW_NAMED( + new String[] {"svt", "ops", "ids"}, + Types.STRING, + Types.ROW_NAMED(new String[] {"id"}, Types.STRING), + Types.PRIMITIVE_ARRAY(Types.INT)); + JsonRowDeserializationSchema deserializationSchema = + new JsonRowDeserializationSchema.Builder(schema).build(); + JsonRowSerializationSchema serializationSchema = + JsonRowSerializationSchema.builder().withTypeInfo(schema).build(); + + for (int i = 0; i < jsons.length; i++) { + String json = jsons[i]; + Row row = deserializationSchema.deserialize(json.getBytes()); + String result = new String(serializationSchema.serialize(row)); + assertEquals(expected[i], result); + } + } + + @Test + public void testNestedSchema() { + final TypeInformation rowSchema = + Types.ROW_NAMED( + new String[] {"f1", "f2", "f3"}, + Types.INT, + Types.BOOLEAN, + Types.ROW(Types.INT, Types.DOUBLE)); + + final Row row = new Row(3); + row.setField(0, 42); + row.setField(1, false); + final Row nested = new Row(2); + nested.setField(0, 22); + nested.setField(1, 2.3); + row.setField(2, nested); + + final JsonRowSerializationSchema serializationSchema = + new JsonRowSerializationSchema.Builder(rowSchema).build(); + final JsonRowDeserializationSchema deserializationSchema = + new JsonRowDeserializationSchema.Builder(rowSchema).build(); + + assertThat( + row, + whenSerializedWith(serializationSchema) + .andDeserializedWith(deserializationSchema) + .equalsTo(row)); + } + + @Test + public void testSerializeRowWithInvalidNumberOfFields() { + final TypeInformation rowSchema = + Types.ROW_NAMED( + new String[] {"f1", "f2", "f3"}, Types.INT, Types.BOOLEAN, Types.STRING); + + final Row row = new Row(1); + row.setField(0, 1); + + final JsonRowSerializationSchema serializationSchema = + new JsonRowSerializationSchema.Builder(rowSchema).build(); + assertThat( + row, + whenSerializedWith(serializationSchema) + .failsWithException(instanceOf(RuntimeException.class))); + } + + @Test + public void testSchema() { + final TypeInformation rowSchema = + JsonRowSchemaConverter.convert( + "{" + + " type: 'object'," + + " properties: {" + + " id: { type: 'integer' }," + + " idNumber: { type: 'number' }," + + " idOrNull: { type: ['integer', 'null'] }," + + " name: { type: 'string' }," + + " date: { type: 'string', format: 'date' }," + + " time: { type: 'string', format: 'time' }," + + " timestamp: { type: 'string', format: 'date-time' }," + + " bytes: { type: 'string', contentEncoding: 'base64' }," + + " numbers: { type: 'array', items: { type: 'integer' } }," + + " strings: { type: 'array', items: { type: 'string' } }," + + " nested: { " + + " type: 'object'," + + " properties: { " + + " booleanField: { type: 'boolean' }," + + " decimalField: { type: 'number' }" + + " }" + + " }" + + " }" + + "}"); + + final Row row = new Row(11); + row.setField(0, BigDecimal.valueOf(-333)); + row.setField(1, BigDecimal.valueOf(12.2222)); + row.setField(2, null); + row.setField(3, ""); + row.setField(4, Date.valueOf("1990-10-14")); + row.setField(5, Time.valueOf("12:12:43")); + row.setField(6, Timestamp.valueOf("1990-10-14 12:12:43")); + + final byte[] bytes = new byte[1024]; + ThreadLocalRandom.current().nextBytes(bytes); + row.setField(7, bytes); + final BigDecimal[] numbers = + new BigDecimal[] { + BigDecimal.valueOf(1), BigDecimal.valueOf(2), BigDecimal.valueOf(3) + }; + row.setField(8, numbers); + final String[] strings = new String[] {"one", "two", "three"}; + row.setField(9, strings); + final Row nestedRow = new Row(2); + nestedRow.setField(0, true); + nestedRow.setField(1, BigDecimal.valueOf(12)); + row.setField(10, nestedRow); + + final JsonRowSerializationSchema serializationSchema = + new JsonRowSerializationSchema.Builder(rowSchema).build(); + final JsonRowDeserializationSchema deserializationSchema = + new JsonRowDeserializationSchema.Builder(rowSchema).build(); + + assertThat( + row, + whenSerializedWith(serializationSchema) + .andDeserializedWith(deserializationSchema) + .equalsTo(row)); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index a914098d9..633fa1a18 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -50,165 +50,173 @@ import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.junit.Assert.assertEquals; -/** - * Tests for {@link CanalJsonFormatFactory}. - */ +/** Tests for {@link CanalJsonFormatFactory}. */ public class CanalJsonFormatFactoryTest extends TestLogger { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - private static final TableSchema SCHEMA = TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); - - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); - - private static final InternalTypeInfo ROW_TYPE_INFO = InternalTypeInfo.of(ROW_TYPE); - - @Test - public void testDefaultOptions() { - Map options = getAllOptions(); - - // test Deser - CanalJsonDeserializationSchema expectedDeser = CanalJsonDeserializationSchema - .builder(ROW_TYPE, ROW_TYPE_INFO) - .setIgnoreParseErrors(false) - .setTimestampFormat(TimestampFormat.SQL) - .build(); - DeserializationSchema actualDeser = createDeserializationSchema(options); - assertEquals(expectedDeser, actualDeser); - - // test Ser - CanalJsonSerializationSchema expectedSer = new CanalJsonSerializationSchema( - ROW_TYPE, - TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.FAIL, - "null"); - SerializationSchema actualSer = createSerializationSchema(options); - assertEquals(expectedSer, actualSer); - } - - @Test - public void testUserDefinedOptions() { - Map options = getAllOptions(); - options.put("canal-json.ignore-parse-errors", "true"); - options.put("canal-json.timestamp-format.standard", "ISO-8601"); - options.put("canal-json.database.include", "mydb"); - options.put("canal-json.table.include", "mytable"); - options.put("canal-json.map-null-key.mode", "LITERAL"); - options.put("canal-json.map-null-key.literal", "nullKey"); - - // test Deser - CanalJsonDeserializationSchema expectedDeser = CanalJsonDeserializationSchema - .builder(ROW_TYPE, ROW_TYPE_INFO) - .setIgnoreParseErrors(true) - .setTimestampFormat(TimestampFormat.ISO_8601) - .setDatabase("mydb") - .setTable("mytable") - .build(); - DeserializationSchema actualDeser = createDeserializationSchema(options); - assertEquals(expectedDeser, actualDeser); - - // test Ser - CanalJsonSerializationSchema expectedSer = new CanalJsonSerializationSchema( - ROW_TYPE, - TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, - "nullKey"); - SerializationSchema actualSer = createSerializationSchema(options); - assertEquals(expectedSer, actualSer); - } - - @Test - public void testInvalidIgnoreParseError() { - thrown.expect(containsCause(new IllegalArgumentException( - "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); - - final Map options = - getModifiedOptions(opts -> opts.put("canal-json.ignore-parse-errors", "abc")); - - createDeserializationSchema(options); - } - - @Test - public void testInvalidOptionForTimestampFormat() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("canal-json.timestamp-format.standard", "test")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new ValidationException("Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - createDeserializationSchema(tableOptions); - } - - @Test - public void testInvalidOptionForMapNullKeyMode() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("canal-json.map-null-key.mode", "invalid")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new ValidationException("Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - createSerializationSchema(tableOptions); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifiedOptions(Consumer> optionModifier) { - Map options = getAllOptions(); - optionModifier.accept(options); - return options; - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - options.put("format", "canal-json"); - return options; - } - - private static DeserializationSchema createDeserializationSchema(Map options) { - DynamicTableSource source = FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - CanalJsonFormatFactoryTest.class.getClassLoader(), - false); - - assert source instanceof TestDynamicTableFactory.DynamicTableSourceMock; - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) source; - - return scanSourceMock.valueFormat - .createRuntimeDecoder(ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); - } - - private static SerializationSchema createSerializationSchema(Map options) { - DynamicTableSink sink = FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - CanalJsonFormatFactoryTest.class.getClassLoader(), - false); - - assert sink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) sink; - - return sinkMock.valueFormat - .createRuntimeEncoder( - new SinkRuntimeProviderContext(false), - SCHEMA.toRowDataType()); - } + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = + TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + private static final InternalTypeInfo ROW_TYPE_INFO = InternalTypeInfo.of(ROW_TYPE); + + @Test + public void testDefaultOptions() { + Map options = getAllOptions(); + + // test Deser + CanalJsonDeserializationSchema expectedDeser = + CanalJsonDeserializationSchema.builder(ROW_TYPE, ROW_TYPE_INFO) + .setIgnoreParseErrors(false) + .setTimestampFormat(TimestampFormat.SQL) + .build(); + DeserializationSchema actualDeser = createDeserializationSchema(options); + assertEquals(expectedDeser, actualDeser); + + // test Ser + CanalJsonSerializationSchema expectedSer = + new CanalJsonSerializationSchema( + ROW_TYPE, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.FAIL, "null"); + SerializationSchema actualSer = createSerializationSchema(options); + assertEquals(expectedSer, actualSer); + } + + @Test + public void testUserDefinedOptions() { + Map options = getAllOptions(); + options.put("canal-json.ignore-parse-errors", "true"); + options.put("canal-json.timestamp-format.standard", "ISO-8601"); + options.put("canal-json.database.include", "mydb"); + options.put("canal-json.table.include", "mytable"); + options.put("canal-json.map-null-key.mode", "LITERAL"); + options.put("canal-json.map-null-key.literal", "nullKey"); + + // test Deser + CanalJsonDeserializationSchema expectedDeser = + CanalJsonDeserializationSchema.builder(ROW_TYPE, ROW_TYPE_INFO) + .setIgnoreParseErrors(true) + .setTimestampFormat(TimestampFormat.ISO_8601) + .setDatabase("mydb") + .setTable("mytable") + .build(); + DeserializationSchema actualDeser = createDeserializationSchema(options); + assertEquals(expectedDeser, actualDeser); + + // test Ser + CanalJsonSerializationSchema expectedSer = + new CanalJsonSerializationSchema( + ROW_TYPE, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "nullKey"); + SerializationSchema actualSer = createSerializationSchema(options); + assertEquals(expectedSer, actualSer); + } + + @Test + public void testInvalidIgnoreParseError() { + thrown.expect( + containsCause( + new IllegalArgumentException( + "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); + + final Map options = + getModifiedOptions(opts -> opts.put("canal-json.ignore-parse-errors", "abc")); + + createDeserializationSchema(options); + } + + @Test + public void testInvalidOptionForTimestampFormat() { + final Map tableOptions = + getModifiedOptions( + opts -> opts.put("canal-json.timestamp-format.standard", "test")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + createDeserializationSchema(tableOptions); + } + + @Test + public void testInvalidOptionForMapNullKeyMode() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("canal-json.map-null-key.mode", "invalid")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); + createSerializationSchema(tableOptions); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifiedOptions(Consumer> optionModifier) { + Map options = getAllOptions(); + optionModifier.accept(options); + return options; + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + options.put("format", "canal-json"); + return options; + } + + private static DeserializationSchema createDeserializationSchema( + Map options) { + DynamicTableSource source = + FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + CanalJsonFormatFactoryTest.class.getClassLoader(), + false); + + assert source instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) source; + + return scanSourceMock.valueFormat.createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); + } + + private static SerializationSchema createSerializationSchema( + Map options) { + DynamicTableSink sink = + FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + CanalJsonFormatFactoryTest.class.getClassLoader(), + false); + + assert sink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) sink; + + return sinkMock.valueFormat.createRuntimeEncoder( + new SinkRuntimeProviderContext(false), SCHEMA.toRowDataType()); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index 6d3b76cda..e390aad61 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -47,177 +47,181 @@ import static org.apache.flink.table.api.DataTypes.STRING; import static org.junit.Assert.assertEquals; -/** - * Tests for {@link CanalJsonSerializationSchema} and {@link CanalJsonDeserializationSchema}. - */ +/** Tests for {@link CanalJsonSerializationSchema} and {@link CanalJsonDeserializationSchema}. */ public class CanalJsonSerDeSchemaTest { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - private static final RowType SCHEMA = (RowType) ROW( - FIELD("id", INT().notNull()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", FLOAT()) - ).getLogicalType(); - - @Test - public void testFilteringTables() throws Exception { - List lines = readLines("canal-data-filter-table.txt"); - CanalJsonDeserializationSchema deserializationSchema = CanalJsonDeserializationSchema - .builder(SCHEMA, InternalTypeInfo.of(SCHEMA)) - .setDatabase("mydb") - .setTable("product") - .build(); - runTest(lines, deserializationSchema); - } - - @Test - public void testSerializationDeserialization() throws Exception { - List lines = readLines("canal-data.txt"); - CanalJsonDeserializationSchema deserializationSchema = CanalJsonDeserializationSchema - .builder(SCHEMA, InternalTypeInfo.of(SCHEMA)) - .setIgnoreParseErrors(false) - .setTimestampFormat(TimestampFormat.ISO_8601) - .build(); - runTest(lines, deserializationSchema); - } - - public void runTest(List lines, CanalJsonDeserializationSchema deserializationSchema) throws Exception { - SimpleCollector collector = new SimpleCollector(); - for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); - } - - // Canal captures change data (`canal-data.txt`) on the `product` table: - // - // CREATE TABLE product ( - // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - // name VARCHAR(255), - // description VARCHAR(512), - // weight FLOAT - // ); - // ALTER TABLE product AUTO_INCREMENT = 101; - // - // INSERT INTO product - // VALUES (default,"scooter","Small 2-wheel scooter",3.14), - // (default,"car battery","12V car battery",8.1), - // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), - // (default,"hammer","12oz carpenter's hammer",0.75), - // (default,"hammer","14oz carpenter's hammer",0.875), - // (default,"hammer","16oz carpenter's hammer",1.0), - // (default,"rocks","box of assorted rocks",5.3), - // (default,"jacket","water resistent black wind breaker",0.1), - // (default,"spare tire","24 inch spare tire",22.2); - // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; - // UPDATE product SET weight='5.1' WHERE id=107; - // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); - // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); - // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110; - // UPDATE product SET weight='5.17' WHERE id=111; - // DELETE FROM product WHERE id=111; - // UPDATE products SET weight='5.17' WHERE id=102 or id = 101; - // DELETE FROM products WHERE id=102 or id = 103; - List expected = Arrays.asList( - "+I(101,scooter,Small 2-wheel scooter,3.14)", - "+I(102,car battery,12V car battery,8.1)", - "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", - "+I(104,hammer,12oz carpenter's hammer,0.75)", - "+I(105,hammer,14oz carpenter's hammer,0.875)", - "+I(106,hammer,16oz carpenter's hammer,1.0)", - "+I(107,rocks,box of assorted rocks,5.3)", - "+I(108,jacket,water resistent black wind breaker,0.1)", - "+I(109,spare tire,24 inch spare tire,22.2)", - "-U(106,hammer,16oz carpenter's hammer,1.0)", - "+U(106,hammer,18oz carpenter hammer,1.0)", - "-U(107,rocks,box of assorted rocks,5.3)", - "+U(107,rocks,box of assorted rocks,5.1)", - "+I(110,jacket,water resistent white wind breaker,0.2)", - "+I(111,scooter,Big 2-wheel scooter ,5.18)", - "-U(110,jacket,water resistent white wind breaker,0.2)", - "+U(110,jacket,new water resistent white wind breaker,0.5)", - "-U(111,scooter,Big 2-wheel scooter ,5.18)", - "+U(111,scooter,Big 2-wheel scooter ,5.17)", - "-D(111,scooter,Big 2-wheel scooter ,5.17)", - "-U(101,scooter,Small 2-wheel scooter,3.14)", - "+U(101,scooter,Small 2-wheel scooter,5.17)", - "-U(102,car battery,12V car battery,8.1)", - "+U(102,car battery,12V car battery,5.17)", - "-D(102,car battery,12V car battery,5.17)", - "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)" - ); - List actual = collector.list.stream() - .map(Object::toString) - .collect(Collectors.toList()); - assertEquals(expected, actual); - - // test Serialization - CanalJsonSerializationSchema serializationSchema = new CanalJsonSerializationSchema( - SCHEMA, - TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, - "null"); - serializationSchema.open(null); - - List result = new ArrayList<>(); - for (RowData rowData : collector.list) { - result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); - } - - List expectedResult = Arrays.asList("{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"DELETE\"}"); - - assertEquals(expectedResult, result); - } - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - private static List readLines(String resource) throws IOException { - final URL url = CanalJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return Files.readAllLines(path); - } - - private static class SimpleCollector implements Collector { - - private List list = new ArrayList<>(); - - @Override - public void collect(RowData record) { - list.add(record); - } - - @Override - public void close() { - // do nothing - } - } + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static final RowType SCHEMA = + (RowType) + ROW( + FIELD("id", INT().notNull()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", FLOAT())) + .getLogicalType(); + + @Test + public void testFilteringTables() throws Exception { + List lines = readLines("canal-data-filter-table.txt"); + CanalJsonDeserializationSchema deserializationSchema = + CanalJsonDeserializationSchema.builder(SCHEMA, InternalTypeInfo.of(SCHEMA)) + .setDatabase("mydb") + .setTable("product") + .build(); + runTest(lines, deserializationSchema); + } + + @Test + public void testSerializationDeserialization() throws Exception { + List lines = readLines("canal-data.txt"); + CanalJsonDeserializationSchema deserializationSchema = + CanalJsonDeserializationSchema.builder(SCHEMA, InternalTypeInfo.of(SCHEMA)) + .setIgnoreParseErrors(false) + .setTimestampFormat(TimestampFormat.ISO_8601) + .build(); + runTest(lines, deserializationSchema); + } + + public void runTest(List lines, CanalJsonDeserializationSchema deserializationSchema) + throws Exception { + SimpleCollector collector = new SimpleCollector(); + for (String line : lines) { + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + } + + // Canal captures change data (`canal-data.txt`) on the `product` table: + // + // CREATE TABLE product ( + // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + // name VARCHAR(255), + // description VARCHAR(512), + // weight FLOAT + // ); + // ALTER TABLE product AUTO_INCREMENT = 101; + // + // INSERT INTO product + // VALUES (default,"scooter","Small 2-wheel scooter",3.14), + // (default,"car battery","12V car battery",8.1), + // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 + // to #3",0.8), + // (default,"hammer","12oz carpenter's hammer",0.75), + // (default,"hammer","14oz carpenter's hammer",0.875), + // (default,"hammer","16oz carpenter's hammer",1.0), + // (default,"rocks","box of assorted rocks",5.3), + // (default,"jacket","water resistent black wind breaker",0.1), + // (default,"spare tire","24 inch spare tire",22.2); + // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; + // UPDATE product SET weight='5.1' WHERE id=107; + // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); + // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); + // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' + // WHERE id=110; + // UPDATE product SET weight='5.17' WHERE id=111; + // DELETE FROM product WHERE id=111; + // UPDATE products SET weight='5.17' WHERE id=102 or id = 101; + // DELETE FROM products WHERE id=102 or id = 103; + List expected = + Arrays.asList( + "+I(101,scooter,Small 2-wheel scooter,3.14)", + "+I(102,car battery,12V car battery,8.1)", + "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", + "+I(104,hammer,12oz carpenter's hammer,0.75)", + "+I(105,hammer,14oz carpenter's hammer,0.875)", + "+I(106,hammer,16oz carpenter's hammer,1.0)", + "+I(107,rocks,box of assorted rocks,5.3)", + "+I(108,jacket,water resistent black wind breaker,0.1)", + "+I(109,spare tire,24 inch spare tire,22.2)", + "-U(106,hammer,16oz carpenter's hammer,1.0)", + "+U(106,hammer,18oz carpenter hammer,1.0)", + "-U(107,rocks,box of assorted rocks,5.3)", + "+U(107,rocks,box of assorted rocks,5.1)", + "+I(110,jacket,water resistent white wind breaker,0.2)", + "+I(111,scooter,Big 2-wheel scooter ,5.18)", + "-U(110,jacket,water resistent white wind breaker,0.2)", + "+U(110,jacket,new water resistent white wind breaker,0.5)", + "-U(111,scooter,Big 2-wheel scooter ,5.18)", + "+U(111,scooter,Big 2-wheel scooter ,5.17)", + "-D(111,scooter,Big 2-wheel scooter ,5.17)", + "-U(101,scooter,Small 2-wheel scooter,3.14)", + "+U(101,scooter,Small 2-wheel scooter,5.17)", + "-U(102,car battery,12V car battery,8.1)", + "+U(102,car battery,12V car battery,5.17)", + "-D(102,car battery,12V car battery,5.17)", + "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)"); + List actual = + collector.list.stream().map(Object::toString).collect(Collectors.toList()); + assertEquals(expected, actual); + + // test Serialization + CanalJsonSerializationSchema serializationSchema = + new CanalJsonSerializationSchema( + SCHEMA, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); + serializationSchema.open(null); + + List result = new ArrayList<>(); + for (RowData rowData : collector.list) { + result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + + List expectedResult = + Arrays.asList( + "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"DELETE\"}"); + + assertEquals(expectedResult, result); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private static List readLines(String resource) throws IOException { + final URL url = CanalJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllLines(path); + } + + private static class SimpleCollector implements Collector { + + private List list = new ArrayList<>(); + + @Override + public void collect(RowData record) { + list.add(record); + } + + @Override + public void close() { + // do nothing + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java index cf3e12c0a..529fb71ad 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java @@ -37,116 +37,121 @@ import static java.lang.String.format; -/** - * Test Filesystem connector with DebeziumJson. - */ +/** Test Filesystem connector with DebeziumJson. */ public class DebeziumJsonFileSystemITCase extends StreamingTestBase { - private static final List EXPECTED = Arrays.asList( - "+I(101,SCOOTER,Small 2-wheel scooter,3.14)", - "+I(102,CAR BATTERY,12V car battery,8.1)", - "+I(103,12-PACK DRILL BITS,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", - "+I(104,HAMMER,12oz carpenter's hammer,0.75)", - "+I(105,HAMMER,14oz carpenter's hammer,0.875)", - "+I(106,HAMMER,16oz carpenter's hammer,1.0)", - "+I(107,ROCKS,box of assorted rocks,5.3)", - "+I(108,JACKET,water resistent black wind breaker,0.1)", - "+I(109,SPARE TIRE,24 inch spare tire,22.2)", - "-D(106,HAMMER,16oz carpenter's hammer,1.0)", // -U - "+I(106,HAMMER,18oz carpenter hammer,1.0)", // +U - "-D(107,ROCKS,box of assorted rocks,5.3)", // -U - "+I(107,ROCKS,box of assorted rocks,5.1)", // +U - "+I(110,JACKET,water resistent white wind breaker,0.2)", - "+I(111,SCOOTER,Big 2-wheel scooter ,5.18)", - "-D(110,JACKET,water resistent white wind breaker,0.2)", // -U - "+I(110,JACKET,new water resistent white wind breaker,0.5)", // +U - "-D(111,SCOOTER,Big 2-wheel scooter ,5.18)", // -U - "+I(111,SCOOTER,Big 2-wheel scooter ,5.17)", // +U - "-D(111,SCOOTER,Big 2-wheel scooter ,5.17)" - ); - - private File source; - private File sink; - - private void prepareTables(boolean isPartition) throws IOException { - byte[] bytes = readBytes("debezium-data-schema-exclude.txt"); - source = TEMPORARY_FOLDER.newFolder(); - File file; - if (isPartition) { - File partition = new File(source, "p=1"); - partition.mkdirs(); - file = new File(partition, "my_file"); - } else { - file = new File(source, "my_file"); - } - file.createNewFile(); - Files.write(file.toPath(), bytes); - - sink = TEMPORARY_FOLDER.newFolder(); - - env().setParallelism(1); - } - - private void createTable(boolean isSink, String path, boolean isPartition) { - tEnv().executeSql(format("create table %s (", isSink ? "sink" : "source") + - "id int, name string," + - (isSink ? "upper_name string," : "") + - " description string, weight float" + - (isPartition ? ", p int) partitioned by (p) " : ")") + - " with (" + - "'connector'='filesystem'," + - "'format'='debezium-json'," + - format("'path'='%s'", path) + - ")"); - } - - @Test - public void testNonPartition() throws Exception { - prepareTables(false); - createTable(false, source.toURI().toString(), false); - createTable(true, sink.toURI().toString(), false); - - tEnv().executeSql("insert into sink select id,name,UPPER(name),description,weight from source").await(); - CloseableIterator iter = tEnv() - .executeSql("select id,upper_name,description,weight from sink").collect(); - - List results = CollectionUtil.iteratorToList(iter).stream() - .map(row -> row.getKind().shortString() + "(" + row.toString() + ")") - .collect(Collectors.toList()); - iter.close(); - - Assert.assertEquals(EXPECTED, results); - } - - @Test - public void testPartition() throws Exception { - prepareTables(true); - createTable(false, source.toURI().toString(), true); - createTable(true, sink.toURI().toString(), true); - - tEnv().executeSql("insert into sink select id,name,UPPER(name),description,weight,p from source").await(); - CloseableIterator iter = tEnv() - .executeSql("select id,upper_name,description,weight,p from sink").collect(); - List list = CollectionUtil.iteratorToList(iter); - iter.close(); - - List results = list.stream() - .map(row -> Row.project(row, new int[] {0, 1, 2, 3})) - .map(row -> row.getKind().shortString() + "(" + row.toString() + ")") - .collect(Collectors.toList()); - - Assert.assertEquals(EXPECTED, results); - - // check partition value - for (Row row : list) { - Assert.assertEquals(1, row.getField(4)); - } - } - - private static byte[] readBytes(String resource) throws IOException { - final URL url = DebeziumJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return Files.readAllBytes(path); - } + private static final List EXPECTED = + Arrays.asList( + "+I(101,SCOOTER,Small 2-wheel scooter,3.14)", + "+I(102,CAR BATTERY,12V car battery,8.1)", + "+I(103,12-PACK DRILL BITS,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", + "+I(104,HAMMER,12oz carpenter's hammer,0.75)", + "+I(105,HAMMER,14oz carpenter's hammer,0.875)", + "+I(106,HAMMER,16oz carpenter's hammer,1.0)", + "+I(107,ROCKS,box of assorted rocks,5.3)", + "+I(108,JACKET,water resistent black wind breaker,0.1)", + "+I(109,SPARE TIRE,24 inch spare tire,22.2)", + "-D(106,HAMMER,16oz carpenter's hammer,1.0)", // -U + "+I(106,HAMMER,18oz carpenter hammer,1.0)", // +U + "-D(107,ROCKS,box of assorted rocks,5.3)", // -U + "+I(107,ROCKS,box of assorted rocks,5.1)", // +U + "+I(110,JACKET,water resistent white wind breaker,0.2)", + "+I(111,SCOOTER,Big 2-wheel scooter ,5.18)", + "-D(110,JACKET,water resistent white wind breaker,0.2)", // -U + "+I(110,JACKET,new water resistent white wind breaker,0.5)", // +U + "-D(111,SCOOTER,Big 2-wheel scooter ,5.18)", // -U + "+I(111,SCOOTER,Big 2-wheel scooter ,5.17)", // +U + "-D(111,SCOOTER,Big 2-wheel scooter ,5.17)"); + + private File source; + private File sink; + + private void prepareTables(boolean isPartition) throws IOException { + byte[] bytes = readBytes("debezium-data-schema-exclude.txt"); + source = TEMPORARY_FOLDER.newFolder(); + File file; + if (isPartition) { + File partition = new File(source, "p=1"); + partition.mkdirs(); + file = new File(partition, "my_file"); + } else { + file = new File(source, "my_file"); + } + file.createNewFile(); + Files.write(file.toPath(), bytes); + + sink = TEMPORARY_FOLDER.newFolder(); + + env().setParallelism(1); + } + + private void createTable(boolean isSink, String path, boolean isPartition) { + tEnv().executeSql( + format("create table %s (", isSink ? "sink" : "source") + + "id int, name string," + + (isSink ? "upper_name string," : "") + + " description string, weight float" + + (isPartition ? ", p int) partitioned by (p) " : ")") + + " with (" + + "'connector'='filesystem'," + + "'format'='debezium-json'," + + format("'path'='%s'", path) + + ")"); + } + + @Test + public void testNonPartition() throws Exception { + prepareTables(false); + createTable(false, source.toURI().toString(), false); + createTable(true, sink.toURI().toString(), false); + + tEnv().executeSql( + "insert into sink select id,name,UPPER(name),description,weight from source") + .await(); + CloseableIterator iter = + tEnv().executeSql("select id,upper_name,description,weight from sink").collect(); + + List results = + CollectionUtil.iteratorToList(iter).stream() + .map(row -> row.getKind().shortString() + "(" + row.toString() + ")") + .collect(Collectors.toList()); + iter.close(); + + Assert.assertEquals(EXPECTED, results); + } + + @Test + public void testPartition() throws Exception { + prepareTables(true); + createTable(false, source.toURI().toString(), true); + createTable(true, sink.toURI().toString(), true); + + tEnv().executeSql( + "insert into sink select id,name,UPPER(name),description,weight,p from source") + .await(); + CloseableIterator iter = + tEnv().executeSql("select id,upper_name,description,weight,p from sink").collect(); + List list = CollectionUtil.iteratorToList(iter); + iter.close(); + + List results = + list.stream() + .map(row -> Row.project(row, new int[] {0, 1, 2, 3})) + .map(row -> row.getKind().shortString() + "(" + row.toString() + ")") + .collect(Collectors.toList()); + + Assert.assertEquals(EXPECTED, results); + + // check partition value + for (Row row : list) { + Assert.assertEquals(1, row.getField(4)); + } + } + + private static byte[] readBytes(String resource) throws IOException { + final URL url = DebeziumJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllBytes(path); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 48081bafb..92f823013 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -53,177 +53,185 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -/** - * Tests for {@link DebeziumJsonFormatFactory}. - */ +/** Tests for {@link DebeziumJsonFormatFactory}. */ public class DebeziumJsonFormatFactoryTest extends TestLogger { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - private static final TableSchema SCHEMA = TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); - - private static final DataType PHYSICAL_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); - - @Test - public void testSeDeSchema() { - final DebeziumJsonDeserializationSchema expectedDeser = new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - false, - true, - TimestampFormat.ISO_8601); - - final Map options = getAllOptions(); - - final DynamicTableSource actualSource = createTableSource(options); - assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - DeserializationSchema actualDeser = scanSourceMock.valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); - - assertEquals(expectedDeser, actualDeser); - - final DebeziumJsonSerializationSchema expectedSer = new DebeziumJsonSerializationSchema( - (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), - TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, - "null"); - - final DynamicTableSink actualSink = createTableSink(options); - assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = sinkMock.valueFormat - .createRuntimeEncoder( - new SinkRuntimeProviderContext(false), - PHYSICAL_DATA_TYPE); - - assertEquals(expectedSer, actualSer); - } - - @Test - public void testInvalidIgnoreParseError() { - thrown.expect(containsCause(new IllegalArgumentException( - "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); - - final Map options = - getModifiedOptions(opts -> opts.put("debezium-json.ignore-parse-errors", "abc")); - - createTableSource(options); - } - - @Test - public void testSchemaIncludeOption() { - Map options = getAllOptions(); - options.put("debezium-json.schema-include", "true"); - - final DebeziumJsonDeserializationSchema expectedDeser = new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - true, - true, - TimestampFormat.ISO_8601); - final DynamicTableSource actualSource = createTableSource(options); - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - DeserializationSchema actualDeser = scanSourceMock.valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); - assertEquals(expectedDeser, actualDeser); - - try { - final DynamicTableSink actualSink = createTableSink(options); - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - // should fail - sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), - PHYSICAL_DATA_TYPE); - fail(); - } catch (Exception e) { - assertEquals( - e.getCause().getCause().getMessage(), - "Debezium JSON serialization doesn't support " + - "'debezium-json.schema-include' option been set to true."); - } - } - - @Test - public void testInvalidOptionForTimestampFormat() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("debezium-json.timestamp-format.standard", "test")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new ValidationException("Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - createTableSource(tableOptions); - } - - @Test - public void testInvalidOptionForMapNullKeyMode() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("debezium-json.map-null-key.mode", "invalid")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new ValidationException("Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - createTableSink(tableOptions); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifiedOptions(Consumer> optionModifier) { - Map options = getAllOptions(); - optionModifier.accept(options); - return options; - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", "debezium-json"); - options.put("debezium-json.ignore-parse-errors", "true"); - options.put("debezium-json.timestamp-format.standard", "ISO-8601"); - options.put("debezium-json.map-null-key.mode", "LITERAL"); - options.put("debezium-json.map-null-key.literal", "null"); - return options; - } - - private static DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - DebeziumJsonFormatFactoryTest.class.getClassLoader(), - false); - } - - private static DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - DebeziumJsonFormatFactoryTest.class.getClassLoader(), - false); - } + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = + TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + + private static final DataType PHYSICAL_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); + + @Test + public void testSeDeSchema() { + final DebeziumJsonDeserializationSchema expectedDeser = + new DebeziumJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), + false, + true, + TimestampFormat.ISO_8601); + + final Map options = getAllOptions(); + + final DynamicTableSource actualSource = createTableSource(options); + assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = + scanSourceMock.valueFormat.createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); + + assertEquals(expectedDeser, actualDeser); + + final DebeziumJsonSerializationSchema expectedSer = + new DebeziumJsonSerializationSchema( + (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); + + final DynamicTableSink actualSink = createTableSink(options); + assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = + sinkMock.valueFormat.createRuntimeEncoder( + new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); + + assertEquals(expectedSer, actualSer); + } + + @Test + public void testInvalidIgnoreParseError() { + thrown.expect( + containsCause( + new IllegalArgumentException( + "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); + + final Map options = + getModifiedOptions(opts -> opts.put("debezium-json.ignore-parse-errors", "abc")); + + createTableSource(options); + } + + @Test + public void testSchemaIncludeOption() { + Map options = getAllOptions(); + options.put("debezium-json.schema-include", "true"); + + final DebeziumJsonDeserializationSchema expectedDeser = + new DebeziumJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), + true, + true, + TimestampFormat.ISO_8601); + final DynamicTableSource actualSource = createTableSource(options); + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + DeserializationSchema actualDeser = + scanSourceMock.valueFormat.createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); + assertEquals(expectedDeser, actualDeser); + + try { + final DynamicTableSink actualSink = createTableSink(options); + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + // should fail + sinkMock.valueFormat.createRuntimeEncoder( + new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); + fail(); + } catch (Exception e) { + assertEquals( + e.getCause().getCause().getMessage(), + "Debezium JSON serialization doesn't support " + + "'debezium-json.schema-include' option been set to true."); + } + } + + @Test + public void testInvalidOptionForTimestampFormat() { + final Map tableOptions = + getModifiedOptions( + opts -> opts.put("debezium-json.timestamp-format.standard", "test")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + createTableSource(tableOptions); + } + + @Test + public void testInvalidOptionForMapNullKeyMode() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("debezium-json.map-null-key.mode", "invalid")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); + createTableSink(tableOptions); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifiedOptions(Consumer> optionModifier) { + Map options = getAllOptions(); + optionModifier.accept(options); + return options; + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", "debezium-json"); + options.put("debezium-json.ignore-parse-errors", "true"); + options.put("debezium-json.timestamp-format.standard", "ISO-8601"); + options.put("debezium-json.map-null-key.mode", "LITERAL"); + options.put("debezium-json.map-null-key.literal", "null"); + return options; + } + + private static DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + DebeziumJsonFormatFactoryTest.class.getClassLoader(), + false); + } + + private static DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + DebeziumJsonFormatFactoryTest.class.getClassLoader(), + false); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java index c8c1ef2f8..4fe152089 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -63,279 +63,287 @@ */ public class DebeziumJsonSerDeSchemaTest { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - private static final DataType PHYSICAL_DATA_TYPE = ROW( - FIELD("id", INT().notNull()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", FLOAT())); - - @Test - public void testSerializationAndSchemaIncludeDeserialization() throws Exception { - testSerializationDeserialization("debezium-data-schema-include.txt", true); - } - - @Test - public void testSerializationAndSchemaExcludeDeserialization() throws Exception { - testSerializationDeserialization("debezium-data-schema-exclude.txt", false); - } - - @Test - public void testSerializationAndPostgresSchemaIncludeDeserialization() throws Exception { - testSerializationDeserialization("debezium-postgres-data-schema-include.txt", true); - } - - @Test - public void testSerializationAndPostgresSchemaExcludeDeserialization() throws Exception { - testSerializationDeserialization("debezium-postgres-data-schema-exclude.txt", false); - } - - @Test - public void testPostgresDefaultReplicaIdentify() { - try { - testSerializationDeserialization("debezium-postgres-data-replica-identity.txt", false); - } catch (Exception e) { - assertTrue(ExceptionUtils.findThrowableWithMessage(e, - "The \"before\" field of UPDATE message is null, if you are using Debezium Postgres Connector, " + - "please check the Postgres table has been set REPLICA IDENTITY to FULL level.").isPresent()); - } - } - - @Test - public void testTombstoneMessages() throws Exception { - DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - false, - false, - TimestampFormat.ISO_8601); - SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(null, collector); - deserializationSchema.deserialize(new byte[]{}, collector); - assertTrue(collector.list.isEmpty()); - } - - @Test - public void testDeserializationWithMetadata() throws Exception { - testDeserializationWithMetadata( - "debezium-data-schema-include.txt", - true, - row -> { - assertThat(row.getInt(0), equalTo(101)); - assertThat(row.getString(1).toString(), equalTo("scooter")); - assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); - assertThat(row.getFloat(3), equalTo(3.14f)); - assertThat( - row.getString(4).toString(), - startsWith("{\"type\":\"struct\",\"fields\":[{\"type\":\"struct\",\"fields\":[{\"type\":\"int32\",\"optional\":false,\"field\":\"id\"},")); - assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1589355606100L)); - assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(0L)); - assertThat(row.getString(7).toString(), equalTo("inventory")); - assertThat(row.isNullAt(8), equalTo(true)); - assertThat(row.getString(9).toString(), equalTo("products")); - assertThat(row.getMap(10).size(), equalTo(14)); - } - ); - - testDeserializationWithMetadata( - "debezium-data-schema-exclude.txt", - false, - row -> { - assertThat(row.getInt(0), equalTo(101)); - assertThat(row.getString(1).toString(), equalTo("scooter")); - assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); - assertThat(row.getFloat(3), equalTo(3.14f)); - assertThat(row.isNullAt(4), equalTo(true)); - assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1589355606100L)); - assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(0L)); - assertThat(row.getString(7).toString(), equalTo("inventory")); - assertThat(row.isNullAt(8), equalTo(true)); - assertThat(row.getString(9).toString(), equalTo("products")); - assertThat(row.getMap(10).size(), equalTo(14)); - } - ); - - testDeserializationWithMetadata( - "debezium-postgres-data-schema-exclude.txt", - false, - row -> { - assertThat(row.getInt(0), equalTo(101)); - assertThat(row.getString(1).toString(), equalTo("scooter")); - assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); - assertThat(row.getFloat(3), equalTo(3.14f)); - assertThat(row.isNullAt(4), equalTo(true)); - assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1596001099434L)); - assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(1596001099434L)); - assertThat(row.getString(7).toString(), equalTo("postgres")); - assertThat(row.getString(8).toString(), equalTo("inventory")); - assertThat(row.getString(9).toString(), equalTo("products")); - assertThat(row.getMap(10).size(), equalTo(11)); - } - ); - } - - private void testSerializationDeserialization(String resourceFile, boolean schemaInclude) throws Exception { - List lines = readLines(resourceFile); - DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - schemaInclude, - false, - TimestampFormat.ISO_8601); - - SimpleCollector collector = new SimpleCollector(); - for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); - } - - // Debezium captures change data (`debezium-data-schema-include.txt`) on the `product` table: - // - // CREATE TABLE product ( - // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - // name VARCHAR(255), - // description VARCHAR(512), - // weight FLOAT - // ); - // ALTER TABLE product AUTO_INCREMENT = 101; - // - // INSERT INTO product - // VALUES (default,"scooter","Small 2-wheel scooter",3.14), - // (default,"car battery","12V car battery",8.1), - // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), - // (default,"hammer","12oz carpenter's hammer",0.75), - // (default,"hammer","14oz carpenter's hammer",0.875), - // (default,"hammer","16oz carpenter's hammer",1.0), - // (default,"rocks","box of assorted rocks",5.3), - // (default,"jacket","water resistent black wind breaker",0.1), - // (default,"spare tire","24 inch spare tire",22.2); - // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; - // UPDATE product SET weight='5.1' WHERE id=107; - // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); - // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); - // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110; - // UPDATE product SET weight='5.17' WHERE id=111; - // DELETE FROM product WHERE id=111; - List expected = Arrays.asList( - "+I(101,scooter,Small 2-wheel scooter,3.14)", - "+I(102,car battery,12V car battery,8.1)", - "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", - "+I(104,hammer,12oz carpenter's hammer,0.75)", - "+I(105,hammer,14oz carpenter's hammer,0.875)", - "+I(106,hammer,16oz carpenter's hammer,1.0)", - "+I(107,rocks,box of assorted rocks,5.3)", - "+I(108,jacket,water resistent black wind breaker,0.1)", - "+I(109,spare tire,24 inch spare tire,22.2)", - "-U(106,hammer,16oz carpenter's hammer,1.0)", - "+U(106,hammer,18oz carpenter hammer,1.0)", - "-U(107,rocks,box of assorted rocks,5.3)", - "+U(107,rocks,box of assorted rocks,5.1)", - "+I(110,jacket,water resistent white wind breaker,0.2)", - "+I(111,scooter,Big 2-wheel scooter ,5.18)", - "-U(110,jacket,water resistent white wind breaker,0.2)", - "+U(110,jacket,new water resistent white wind breaker,0.5)", - "-U(111,scooter,Big 2-wheel scooter ,5.18)", - "+U(111,scooter,Big 2-wheel scooter ,5.17)", - "-D(111,scooter,Big 2-wheel scooter ,5.17)" - ); - List actual = collector.list.stream() - .map(Object::toString) - .collect(Collectors.toList()); - assertEquals(expected, actual); - - DebeziumJsonSerializationSchema serializationSchema = new DebeziumJsonSerializationSchema( - (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), - TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.LITERAL, - "null"); - - serializationSchema.open(null); - actual = new ArrayList<>(); - for (RowData rowData : collector.list) { - actual.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); - } - - expected = Arrays.asList( - "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op\":\"c\"}", - "{\"before\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"after\":null,\"op\":\"d\"}", - "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op\":\"c\"}", - "{\"before\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"after\":null,\"op\":\"d\"}", - "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op\":\"c\"}", - "{\"before\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"after\":null,\"op\":\"d\"}", - "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op\":\"c\"}", - "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":null,\"op\":\"d\"}", - "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op\":\"c\"}", - "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op\":\"d\"}" - ); - assertEquals(expected, actual); - } - - private void testDeserializationWithMetadata( - String resourceFile, - boolean schemaInclude, - Consumer testConsumer) throws Exception { - // we only read the first line for keeping the test simple - final String firstLine = readLines(resourceFile).get(0); - - final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); - - final DataType producedDataType = DataTypeUtils.appendRowFields( - PHYSICAL_DATA_TYPE, - requestedMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList())); - - final DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - requestedMetadata, - InternalTypeInfo.of(producedDataType.getLogicalType()), - schemaInclude, - false, - TimestampFormat.ISO_8601); - - final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - - assertEquals(1, collector.list.size()); - testConsumer.accept(collector.list.get(0)); - } - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - private static List readLines(String resource) throws IOException { - final URL url = DebeziumJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return Files.readAllLines(path); - } - - private static class SimpleCollector implements Collector { - - private List list = new ArrayList<>(); - - @Override - public void collect(RowData record) { - list.add(record); - } - - @Override - public void close() { - // do nothing - } - } + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static final DataType PHYSICAL_DATA_TYPE = + ROW( + FIELD("id", INT().notNull()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", FLOAT())); + + @Test + public void testSerializationAndSchemaIncludeDeserialization() throws Exception { + testSerializationDeserialization("debezium-data-schema-include.txt", true); + } + + @Test + public void testSerializationAndSchemaExcludeDeserialization() throws Exception { + testSerializationDeserialization("debezium-data-schema-exclude.txt", false); + } + + @Test + public void testSerializationAndPostgresSchemaIncludeDeserialization() throws Exception { + testSerializationDeserialization("debezium-postgres-data-schema-include.txt", true); + } + + @Test + public void testSerializationAndPostgresSchemaExcludeDeserialization() throws Exception { + testSerializationDeserialization("debezium-postgres-data-schema-exclude.txt", false); + } + + @Test + public void testPostgresDefaultReplicaIdentify() { + try { + testSerializationDeserialization("debezium-postgres-data-replica-identity.txt", false); + } catch (Exception e) { + assertTrue( + ExceptionUtils.findThrowableWithMessage( + e, + "The \"before\" field of UPDATE message is null, if you are using Debezium Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level.") + .isPresent()); + } + } + + @Test + public void testTombstoneMessages() throws Exception { + DebeziumJsonDeserializationSchema deserializationSchema = + new DebeziumJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), + false, + false, + TimestampFormat.ISO_8601); + SimpleCollector collector = new SimpleCollector(); + deserializationSchema.deserialize(null, collector); + deserializationSchema.deserialize(new byte[] {}, collector); + assertTrue(collector.list.isEmpty()); + } + + @Test + public void testDeserializationWithMetadata() throws Exception { + testDeserializationWithMetadata( + "debezium-data-schema-include.txt", + true, + row -> { + assertThat(row.getInt(0), equalTo(101)); + assertThat(row.getString(1).toString(), equalTo("scooter")); + assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); + assertThat(row.getFloat(3), equalTo(3.14f)); + assertThat( + row.getString(4).toString(), + startsWith( + "{\"type\":\"struct\",\"fields\":[{\"type\":\"struct\",\"fields\":[{\"type\":\"int32\",\"optional\":false,\"field\":\"id\"},")); + assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1589355606100L)); + assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(0L)); + assertThat(row.getString(7).toString(), equalTo("inventory")); + assertThat(row.isNullAt(8), equalTo(true)); + assertThat(row.getString(9).toString(), equalTo("products")); + assertThat(row.getMap(10).size(), equalTo(14)); + }); + + testDeserializationWithMetadata( + "debezium-data-schema-exclude.txt", + false, + row -> { + assertThat(row.getInt(0), equalTo(101)); + assertThat(row.getString(1).toString(), equalTo("scooter")); + assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); + assertThat(row.getFloat(3), equalTo(3.14f)); + assertThat(row.isNullAt(4), equalTo(true)); + assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1589355606100L)); + assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(0L)); + assertThat(row.getString(7).toString(), equalTo("inventory")); + assertThat(row.isNullAt(8), equalTo(true)); + assertThat(row.getString(9).toString(), equalTo("products")); + assertThat(row.getMap(10).size(), equalTo(14)); + }); + + testDeserializationWithMetadata( + "debezium-postgres-data-schema-exclude.txt", + false, + row -> { + assertThat(row.getInt(0), equalTo(101)); + assertThat(row.getString(1).toString(), equalTo("scooter")); + assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); + assertThat(row.getFloat(3), equalTo(3.14f)); + assertThat(row.isNullAt(4), equalTo(true)); + assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1596001099434L)); + assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(1596001099434L)); + assertThat(row.getString(7).toString(), equalTo("postgres")); + assertThat(row.getString(8).toString(), equalTo("inventory")); + assertThat(row.getString(9).toString(), equalTo("products")); + assertThat(row.getMap(10).size(), equalTo(11)); + }); + } + + private void testSerializationDeserialization(String resourceFile, boolean schemaInclude) + throws Exception { + List lines = readLines(resourceFile); + DebeziumJsonDeserializationSchema deserializationSchema = + new DebeziumJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), + schemaInclude, + false, + TimestampFormat.ISO_8601); + + SimpleCollector collector = new SimpleCollector(); + for (String line : lines) { + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + } + + // Debezium captures change data (`debezium-data-schema-include.txt`) on the `product` + // table: + // + // CREATE TABLE product ( + // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + // name VARCHAR(255), + // description VARCHAR(512), + // weight FLOAT + // ); + // ALTER TABLE product AUTO_INCREMENT = 101; + // + // INSERT INTO product + // VALUES (default,"scooter","Small 2-wheel scooter",3.14), + // (default,"car battery","12V car battery",8.1), + // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 + // to #3",0.8), + // (default,"hammer","12oz carpenter's hammer",0.75), + // (default,"hammer","14oz carpenter's hammer",0.875), + // (default,"hammer","16oz carpenter's hammer",1.0), + // (default,"rocks","box of assorted rocks",5.3), + // (default,"jacket","water resistent black wind breaker",0.1), + // (default,"spare tire","24 inch spare tire",22.2); + // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; + // UPDATE product SET weight='5.1' WHERE id=107; + // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); + // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); + // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' + // WHERE id=110; + // UPDATE product SET weight='5.17' WHERE id=111; + // DELETE FROM product WHERE id=111; + List expected = + Arrays.asList( + "+I(101,scooter,Small 2-wheel scooter,3.14)", + "+I(102,car battery,12V car battery,8.1)", + "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", + "+I(104,hammer,12oz carpenter's hammer,0.75)", + "+I(105,hammer,14oz carpenter's hammer,0.875)", + "+I(106,hammer,16oz carpenter's hammer,1.0)", + "+I(107,rocks,box of assorted rocks,5.3)", + "+I(108,jacket,water resistent black wind breaker,0.1)", + "+I(109,spare tire,24 inch spare tire,22.2)", + "-U(106,hammer,16oz carpenter's hammer,1.0)", + "+U(106,hammer,18oz carpenter hammer,1.0)", + "-U(107,rocks,box of assorted rocks,5.3)", + "+U(107,rocks,box of assorted rocks,5.1)", + "+I(110,jacket,water resistent white wind breaker,0.2)", + "+I(111,scooter,Big 2-wheel scooter ,5.18)", + "-U(110,jacket,water resistent white wind breaker,0.2)", + "+U(110,jacket,new water resistent white wind breaker,0.5)", + "-U(111,scooter,Big 2-wheel scooter ,5.18)", + "+U(111,scooter,Big 2-wheel scooter ,5.17)", + "-D(111,scooter,Big 2-wheel scooter ,5.17)"); + List actual = + collector.list.stream().map(Object::toString).collect(Collectors.toList()); + assertEquals(expected, actual); + + DebeziumJsonSerializationSchema serializationSchema = + new DebeziumJsonSerializationSchema( + (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); + + serializationSchema.open(null); + actual = new ArrayList<>(); + for (RowData rowData : collector.list) { + actual.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + + expected = + Arrays.asList( + "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op\":\"c\"}", + "{\"before\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op\":\"c\"}", + "{\"before\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op\":\"c\"}", + "{\"before\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op\":\"c\"}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op\":\"c\"}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op\":\"d\"}"); + assertEquals(expected, actual); + } + + private void testDeserializationWithMetadata( + String resourceFile, boolean schemaInclude, Consumer testConsumer) + throws Exception { + // we only read the first line for keeping the test simple + final String firstLine = readLines(resourceFile).get(0); + + final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); + + final DataType producedDataType = + DataTypeUtils.appendRowFields( + PHYSICAL_DATA_TYPE, + requestedMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList())); + + final DebeziumJsonDeserializationSchema deserializationSchema = + new DebeziumJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + requestedMetadata, + InternalTypeInfo.of(producedDataType.getLogicalType()), + schemaInclude, + false, + TimestampFormat.ISO_8601); + + final SimpleCollector collector = new SimpleCollector(); + deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); + + assertEquals(1, collector.list.size()); + testConsumer.accept(collector.list.get(0)); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private static List readLines(String resource) throws IOException { + final URL url = DebeziumJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllLines(path); + } + + private static class SimpleCollector implements Collector { + + private List list = new ArrayList<>(); + + @Override + public void collect(RowData record) { + list.add(record); + } + + @Override + public void close() { + // do nothing + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java index 395d5248e..e7d691018 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java @@ -50,138 +50,143 @@ import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.junit.Assert.assertEquals; -/** - * Tests for {@link MaxwellJsonFormatFactory}. - */ +/** Tests for {@link MaxwellJsonFormatFactory}. */ public class MaxwellJsonFormatFactoryTest extends TestLogger { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - private static final TableSchema SCHEMA = TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); - - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); - - @Test - public void testSeDeSchema() { - final MaxwellJsonDeserializationSchema expectedDeser = new MaxwellJsonDeserializationSchema( - ROW_TYPE, - InternalTypeInfo.of(ROW_TYPE), - true, - TimestampFormat.ISO_8601); - - final MaxwellJsonSerializationSchema expectedSer = new MaxwellJsonSerializationSchema( - ROW_TYPE, - TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, - "null"); - - final Map options = getAllOptions(); - - final DynamicTableSource actualSource = createTableSource(options); - assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - DeserializationSchema actualDeser = scanSourceMock.valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, - SCHEMA.toRowDataType()); - - assertEquals(expectedDeser, actualDeser); - - final DynamicTableSink actualSink = createTableSink(options); - assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = sinkMock.valueFormat - .createRuntimeEncoder( - new SinkRuntimeProviderContext(false), - SCHEMA.toRowDataType()); - - assertEquals(expectedSer, actualSer); - } - - @Test - public void testInvalidIgnoreParseError() { - thrown.expect(containsCause(new IllegalArgumentException( - "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); - - final Map options = - getModifiedOptions(opts -> opts.put("maxwell-json.ignore-parse-errors", "abc")); - - createTableSource(options); - } - - @Test - public void testInvalidOptionForTimestampFormat() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("maxwell-json.timestamp-format.standard", "test")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new ValidationException("Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - createTableSource(tableOptions); - } - - @Test - public void testInvalidOptionForMapNullKeyMode() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("maxwell-json.map-null-key.mode", "invalid")); - - thrown.expect(ValidationException.class); - thrown.expect(containsCause(new ValidationException("Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - createTableSink(tableOptions); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifiedOptions(Consumer> optionModifier) { - Map options = getAllOptions(); - optionModifier.accept(options); - return options; - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", "maxwell-json"); - options.put("maxwell-json.ignore-parse-errors", "true"); - options.put("maxwell-json.timestamp-format.standard", "ISO-8601"); - options.put("maxwell-json.map-null-key.mode", "LITERAL"); - options.put("maxwell-json.map-null-key.literal", "null"); - return options; - } - - private static DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - MaxwellJsonFormatFactoryTest.class.getClassLoader(), - false); - } - - private static DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - MaxwellJsonFormatFactoryTest.class.getClassLoader(), - false); - } + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static final TableSchema SCHEMA = + TableSchema.builder() + .field("a", DataTypes.STRING()) + .field("b", DataTypes.INT()) + .field("c", DataTypes.BOOLEAN()) + .build(); + + private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + @Test + public void testSeDeSchema() { + final MaxwellJsonDeserializationSchema expectedDeser = + new MaxwellJsonDeserializationSchema( + ROW_TYPE, InternalTypeInfo.of(ROW_TYPE), true, TimestampFormat.ISO_8601); + + final MaxwellJsonSerializationSchema expectedSer = + new MaxwellJsonSerializationSchema( + ROW_TYPE, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null"); + + final Map options = getAllOptions(); + + final DynamicTableSource actualSource = createTableSource(options); + assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = + scanSourceMock.valueFormat.createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); + + assertEquals(expectedDeser, actualDeser); + + final DynamicTableSink actualSink = createTableSink(options); + assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = + sinkMock.valueFormat.createRuntimeEncoder( + new SinkRuntimeProviderContext(false), SCHEMA.toRowDataType()); + + assertEquals(expectedSer, actualSer); + } + + @Test + public void testInvalidIgnoreParseError() { + thrown.expect( + containsCause( + new IllegalArgumentException( + "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); + + final Map options = + getModifiedOptions(opts -> opts.put("maxwell-json.ignore-parse-errors", "abc")); + + createTableSource(options); + } + + @Test + public void testInvalidOptionForTimestampFormat() { + final Map tableOptions = + getModifiedOptions( + opts -> opts.put("maxwell-json.timestamp-format.standard", "test")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + createTableSource(tableOptions); + } + + @Test + public void testInvalidOptionForMapNullKeyMode() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("maxwell-json.map-null-key.mode", "invalid")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); + createTableSink(tableOptions); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifiedOptions(Consumer> optionModifier) { + Map options = getAllOptions(); + optionModifier.accept(options); + return options; + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", "maxwell-json"); + options.put("maxwell-json.ignore-parse-errors", "true"); + options.put("maxwell-json.timestamp-format.standard", "ISO-8601"); + options.put("maxwell-json.map-null-key.mode", "LITERAL"); + options.put("maxwell-json.map-null-key.literal", "null"); + return options; + } + + private static DynamicTableSource createTableSource(Map options) { + return FactoryUtil.createTableSource( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock source"), + new Configuration(), + MaxwellJsonFormatFactoryTest.class.getClassLoader(), + false); + } + + private static DynamicTableSink createTableSink(Map options) { + return FactoryUtil.createTableSink( + null, + ObjectIdentifier.of("default", "default", "t1"), + new CatalogTableImpl(SCHEMA, options, "mock sink"), + new Configuration(), + MaxwellJsonFormatFactoryTest.class.getClassLoader(), + false); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java index fef5ede23..99f432fb7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java @@ -49,153 +49,152 @@ * Tests for {@link MaxwellJsonSerializationSchema} and {@link MaxwellJsonDeserializationSchema}. */ public class MaxwellJsonSerDerTest { - private static final RowType SCHEMA = (RowType) ROW( - FIELD("id", INT().notNull()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", FLOAT()) - ).getLogicalType(); - - @Test - public void testSerializationDeserialization() throws Exception { - List lines = readLines("maxwell-data.txt"); - MaxwellJsonDeserializationSchema deserializationSchema = new MaxwellJsonDeserializationSchema( - SCHEMA, - InternalTypeInfo.of(SCHEMA), - false, - TimestampFormat.ISO_8601); - - SimpleCollector collector = new SimpleCollector(); - for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); - } - - // Maxwell captures change data (`maxwell-data.txt`) on the `product` table: - // - // CREATE TABLE product ( - // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - // name VARCHAR(255), - // description VARCHAR(512), - // weight FLOAT - // ); - // ALTER TABLE product AUTO_INCREMENT = 101; - // - // INSERT INTO product - // VALUES (default,"scooter","Small 2-wheel scooter",3.14), - // (default,"car battery","12V car battery",8.1), - // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), - // (default,"hammer","12oz carpenter's hammer",0.75), - // (default,"hammer","14oz carpenter's hammer",0.875), - // (default,"hammer","16oz carpenter's hammer",1.0), - // (default,"rocks","box of assorted rocks",5.3), - // (default,"jacket","water resistent black wind breaker",0.1), - // (default,"spare tire","24 inch spare tire",22.2); - // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; - // UPDATE product SET weight='5.1' WHERE id=107; - // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); - // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); - // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110; - // UPDATE product SET weight='5.17' WHERE id=111; - // DELETE FROM product WHERE id=111; - // UPDATE product SET weight='5.17' WHERE id=102 or id = 101; - // DELETE FROM product WHERE id=102 or id = 103; - List expected = Arrays.asList( - "+I(101,scooter,Small 2-wheel scooter,3.14)", - "+I(102,car battery,12V car battery,8.1)", - "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", - "+I(104,hammer,12oz carpenter's hammer,0.75)", - "+I(105,hammer,14oz carpenter's hammer,0.875)", - "+I(106,hammer,16oz carpenter's hammer,1.0)", - "+I(107,rocks,box of assorted rocks,5.3)", - "+I(108,jacket,water resistent black wind breaker,0.1)", - "+I(109,spare tire,24 inch spare tire,22.2)", - "-U(106,hammer,16oz carpenter's hammer,1.0)", - "+U(106,hammer,18oz carpenter hammer,1.0)", - "-U(107,rocks,box of assorted rocks,5.3)", - "+U(107,rocks,box of assorted rocks,5.1)", - "+I(110,jacket,water resistent white wind breaker,0.2)", - "+I(111,scooter,Big 2-wheel scooter ,5.18)", - "-U(110,jacket,water resistent white wind breaker,0.2)", - "+U(110,jacket,new water resistent white wind breaker,0.5)", - "-U(111,scooter,Big 2-wheel scooter ,5.18)", - "+U(111,scooter,Big 2-wheel scooter ,5.17)", - "-D(111,scooter,Big 2-wheel scooter ,5.17)", - "-U(101,scooter,Small 2-wheel scooter,3.14)", - "+U(101,scooter,Small 2-wheel scooter,5.17)", - "-U(102,car battery,12V car battery,8.1)", - "+U(102,car battery,12V car battery,5.17)", - "-D(102,car battery,12V car battery,5.17)", - "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)" - ); - List actual = collector.list.stream() - .map(Object::toString) - .collect(Collectors.toList()); - assertEquals(expected, actual); - - MaxwellJsonSerializationSchema serializationSchema = new MaxwellJsonSerializationSchema( - SCHEMA, - TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.LITERAL, - "null"); - serializationSchema.open(null); - List result = new ArrayList<>(); - for (RowData rowData : collector.list) { - result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); - } - List expectedResult = Arrays.asList( - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"insert\"}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"insert\"}", - "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"insert\"}", - "{\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"type\":\"insert\"}", - "{\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"type\":\"insert\"}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"insert\"}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"insert\"}", - "{\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"type\":\"insert\"}", - "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"type\":\"insert\"}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"delete\"}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"type\":\"insert\"}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"delete\"}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"type\":\"insert\"}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"insert\"}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"insert\"}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"delete\"}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"type\":\"insert\"}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"delete\"}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"insert\"}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"delete\"}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"delete\"}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17},\"type\":\"insert\"}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"delete\"}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"insert\"}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"delete\"}", - "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"delete\"}" - ); - assertEquals(expectedResult, result); - } - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - private static List readLines(String resource) throws IOException { - final URL url = MaxwellJsonSerDerTest.class.getClassLoader().getResource(resource); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return Files.readAllLines(path); - } - - private static class SimpleCollector implements Collector { - - private List list = new ArrayList<>(); - - @Override - public void collect(RowData record) { - list.add(record); - } - - @Override - public void close() { - // do nothing - } - } + private static final RowType SCHEMA = + (RowType) + ROW( + FIELD("id", INT().notNull()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", FLOAT())) + .getLogicalType(); + + @Test + public void testSerializationDeserialization() throws Exception { + List lines = readLines("maxwell-data.txt"); + MaxwellJsonDeserializationSchema deserializationSchema = + new MaxwellJsonDeserializationSchema( + SCHEMA, InternalTypeInfo.of(SCHEMA), false, TimestampFormat.ISO_8601); + + SimpleCollector collector = new SimpleCollector(); + for (String line : lines) { + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + } + + // Maxwell captures change data (`maxwell-data.txt`) on the `product` table: + // + // CREATE TABLE product ( + // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + // name VARCHAR(255), + // description VARCHAR(512), + // weight FLOAT + // ); + // ALTER TABLE product AUTO_INCREMENT = 101; + // + // INSERT INTO product + // VALUES (default,"scooter","Small 2-wheel scooter",3.14), + // (default,"car battery","12V car battery",8.1), + // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 + // to #3",0.8), + // (default,"hammer","12oz carpenter's hammer",0.75), + // (default,"hammer","14oz carpenter's hammer",0.875), + // (default,"hammer","16oz carpenter's hammer",1.0), + // (default,"rocks","box of assorted rocks",5.3), + // (default,"jacket","water resistent black wind breaker",0.1), + // (default,"spare tire","24 inch spare tire",22.2); + // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; + // UPDATE product SET weight='5.1' WHERE id=107; + // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); + // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); + // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' + // WHERE id=110; + // UPDATE product SET weight='5.17' WHERE id=111; + // DELETE FROM product WHERE id=111; + // UPDATE product SET weight='5.17' WHERE id=102 or id = 101; + // DELETE FROM product WHERE id=102 or id = 103; + List expected = + Arrays.asList( + "+I(101,scooter,Small 2-wheel scooter,3.14)", + "+I(102,car battery,12V car battery,8.1)", + "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", + "+I(104,hammer,12oz carpenter's hammer,0.75)", + "+I(105,hammer,14oz carpenter's hammer,0.875)", + "+I(106,hammer,16oz carpenter's hammer,1.0)", + "+I(107,rocks,box of assorted rocks,5.3)", + "+I(108,jacket,water resistent black wind breaker,0.1)", + "+I(109,spare tire,24 inch spare tire,22.2)", + "-U(106,hammer,16oz carpenter's hammer,1.0)", + "+U(106,hammer,18oz carpenter hammer,1.0)", + "-U(107,rocks,box of assorted rocks,5.3)", + "+U(107,rocks,box of assorted rocks,5.1)", + "+I(110,jacket,water resistent white wind breaker,0.2)", + "+I(111,scooter,Big 2-wheel scooter ,5.18)", + "-U(110,jacket,water resistent white wind breaker,0.2)", + "+U(110,jacket,new water resistent white wind breaker,0.5)", + "-U(111,scooter,Big 2-wheel scooter ,5.18)", + "+U(111,scooter,Big 2-wheel scooter ,5.17)", + "-D(111,scooter,Big 2-wheel scooter ,5.17)", + "-U(101,scooter,Small 2-wheel scooter,3.14)", + "+U(101,scooter,Small 2-wheel scooter,5.17)", + "-U(102,car battery,12V car battery,8.1)", + "+U(102,car battery,12V car battery,5.17)", + "-D(102,car battery,12V car battery,5.17)", + "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)"); + List actual = + collector.list.stream().map(Object::toString).collect(Collectors.toList()); + assertEquals(expected, actual); + + MaxwellJsonSerializationSchema serializationSchema = + new MaxwellJsonSerializationSchema( + SCHEMA, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, "null"); + serializationSchema.open(null); + List result = new ArrayList<>(); + for (RowData rowData : collector.list) { + result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + List expectedResult = + Arrays.asList( + "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"insert\"}", + "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"insert\"}", + "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"insert\"}", + "{\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"type\":\"insert\"}", + "{\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"type\":\"insert\"}", + "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"insert\"}", + "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"insert\"}", + "{\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"type\":\"insert\"}", + "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"type\":\"insert\"}", + "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"delete\"}", + "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"type\":\"insert\"}", + "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"delete\"}", + "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"type\":\"insert\"}", + "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"insert\"}", + "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"insert\"}", + "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"delete\"}", + "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"type\":\"insert\"}", + "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"delete\"}", + "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"insert\"}", + "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"delete\"}", + "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"delete\"}", + "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17},\"type\":\"insert\"}", + "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"delete\"}", + "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"insert\"}", + "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"delete\"}", + "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"delete\"}"); + assertEquals(expectedResult, result); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private static List readLines(String resource) throws IOException { + final URL url = MaxwellJsonSerDerTest.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllLines(path); + } + + private static class SimpleCollector implements Collector { + + private List list = new ArrayList<>(); + + @Override + public void collect(RowData record) { + list.add(record); + } + + @Override + public void close() { + // do nothing + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java index c5b19e7f0..59ee17820 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java @@ -32,133 +32,128 @@ import static org.apache.flink.util.InstantiationUtil.serializeObject; /** - * Matcher that provides a common way for asserting results of {@link DeserializationSchema}. It takes into account - * e.g. the fact that serialization schema during runtime might be used after serializing it over a wire. Usage: + * Matcher that provides a common way for asserting results of {@link DeserializationSchema}. It + * takes into account e.g. the fact that serialization schema during runtime might be used after + * serializing it over a wire. Usage: * *

    - *
  • when asserting for result after deserializing a row - *
    {@code
    - *      byte[] jsonBytes = ...
    - *      Row expectedRow = ...
    - *      final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema);
    + *   
  • when asserting for result after deserializing a row + *
    {@code
    + * byte[] jsonBytes = ...
    + * Row expectedRow = ...
    + * final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema);
      *
    - *      assertThat(jsonBytes, whenDeserializedWith(deserializationSchema)
    - *          .matches(expectedRow));
    + * assertThat(jsonBytes, whenDeserializedWith(deserializationSchema)
    + *     .matches(expectedRow));
      * }
    - *
  • - * - *
  • to check if an exception is thrown during serialization: - *
    {@code
    - *      assertThat(serializedJson,
    - *          whenDeserializedWith(deserializationSchema)
    - *              .failsWithException(hasCause(instanceOf(IllegalStateException.class))));
    + *   
  • to check if an exception is thrown during serialization: + *
    {@code
    + * assertThat(serializedJson,
    + *     whenDeserializedWith(deserializationSchema)
    + *         .failsWithException(hasCause(instanceOf(IllegalStateException.class))));
      * }
    - *
  • *
*/ public abstract class DeserializationSchemaMatcher extends TypeSafeMatcher { - final DeserializationSchema deserializationSchema; - - private DeserializationSchemaMatcher(DeserializationSchema deserializationSchema) { - this.deserializationSchema = deserializationSchema; - } - - public static DeserializationSchemaMatcherBuilder whenDeserializedWith(DeserializationSchema deserializationSchema) { - return new DeserializationSchemaMatcherBuilder(deserializationSchema); - } - - private static class DeserializationSchemaResultMatcher extends DeserializationSchemaMatcher { - - private final Row expected; - - private DeserializationSchemaResultMatcher( - DeserializationSchema deserializationSchema, - Row expected) { - super(deserializationSchema); - - this.expected = expected; - } - - @Override - protected boolean matchesSafely(byte[] item) { - try { - return Objects.deepEquals(deserializationSchema.deserialize(item), expected); - } catch (IOException e) { - throw new AssertionError("Could not deserialize", e); - } - } - - @Override - public void describeTo(Description description) { - description.appendValue(expected); - } - } - - private static class DeserializationSchemaExceptionMatcher extends DeserializationSchemaMatcher { - - private final Matcher exceptionMatcher; - private Throwable thrownException = null; - - private DeserializationSchemaExceptionMatcher( - DeserializationSchema deserializationSchema, - Matcher exceptionMatcher) { - super(deserializationSchema); - this.exceptionMatcher = exceptionMatcher; - } - - @Override - protected boolean matchesSafely(byte[] item) { - try { - deserializationSchema.deserialize(item); - } catch (IOException e) { - thrownException = e; - } - return exceptionMatcher.matches(thrownException); - } - - @Override - public void describeTo(Description description) { - exceptionMatcher.describeTo(description); - } - - @Override - protected void describeMismatchSafely(byte[] item, Description mismatchDescription) { - exceptionMatcher.describeMismatch(thrownException, mismatchDescription); - } - } - - /** - * Builder for {@link DeserializationSchemaMatcher}. - */ - public static class DeserializationSchemaMatcherBuilder { - - private DeserializationSchema deserializationSchema; - - private DeserializationSchemaMatcherBuilder(DeserializationSchema deserializationSchema) { - try { - // we serialize and deserialize the schema to test runtime behavior - // when the schema is shipped to the cluster - this.deserializationSchema = deserializeObject( - serializeObject(deserializationSchema), - this.getClass().getClassLoader()); - } catch (IOException | ClassNotFoundException e) { - throw new RuntimeException(e); - } - } - - public DeserializationSchemaMatcher equalsTo(Row row) { - return new DeserializationSchemaResultMatcher( - deserializationSchema, - row - ); - } - - public DeserializationSchemaMatcher failsWithException(Matcher exceptionMatcher) { - return new DeserializationSchemaExceptionMatcher( - deserializationSchema, - exceptionMatcher - ); - } - } + final DeserializationSchema deserializationSchema; + + private DeserializationSchemaMatcher(DeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + } + + public static DeserializationSchemaMatcherBuilder whenDeserializedWith( + DeserializationSchema deserializationSchema) { + return new DeserializationSchemaMatcherBuilder(deserializationSchema); + } + + private static class DeserializationSchemaResultMatcher extends DeserializationSchemaMatcher { + + private final Row expected; + + private DeserializationSchemaResultMatcher( + DeserializationSchema deserializationSchema, Row expected) { + super(deserializationSchema); + + this.expected = expected; + } + + @Override + protected boolean matchesSafely(byte[] item) { + try { + return Objects.deepEquals(deserializationSchema.deserialize(item), expected); + } catch (IOException e) { + throw new AssertionError("Could not deserialize", e); + } + } + + @Override + public void describeTo(Description description) { + description.appendValue(expected); + } + } + + private static class DeserializationSchemaExceptionMatcher + extends DeserializationSchemaMatcher { + + private final Matcher exceptionMatcher; + private Throwable thrownException = null; + + private DeserializationSchemaExceptionMatcher( + DeserializationSchema deserializationSchema, + Matcher exceptionMatcher) { + super(deserializationSchema); + this.exceptionMatcher = exceptionMatcher; + } + + @Override + protected boolean matchesSafely(byte[] item) { + try { + deserializationSchema.deserialize(item); + } catch (IOException e) { + thrownException = e; + } + return exceptionMatcher.matches(thrownException); + } + + @Override + public void describeTo(Description description) { + exceptionMatcher.describeTo(description); + } + + @Override + protected void describeMismatchSafely(byte[] item, Description mismatchDescription) { + exceptionMatcher.describeMismatch(thrownException, mismatchDescription); + } + } + + /** Builder for {@link DeserializationSchemaMatcher}. */ + public static class DeserializationSchemaMatcherBuilder { + + private DeserializationSchema deserializationSchema; + + private DeserializationSchemaMatcherBuilder( + DeserializationSchema deserializationSchema) { + try { + // we serialize and deserialize the schema to test runtime behavior + // when the schema is shipped to the cluster + this.deserializationSchema = + deserializeObject( + serializeObject(deserializationSchema), + this.getClass().getClassLoader()); + } catch (IOException | ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + public DeserializationSchemaMatcher equalsTo(Row row) { + return new DeserializationSchemaResultMatcher(deserializationSchema, row); + } + + public DeserializationSchemaMatcher failsWithException( + Matcher exceptionMatcher) { + return new DeserializationSchemaExceptionMatcher( + deserializationSchema, exceptionMatcher); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java index 4cda5dd37..be0951551 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java @@ -33,160 +33,156 @@ import static org.apache.flink.util.InstantiationUtil.serializeObject; /** - * Matcher that provides a common way for asserting results of {@link SerializationSchema}. It takes into account - * e.g. the fact that serialization schema during runtime might be used after serializing and deserializing it over - * a wire. Usage: + * Matcher that provides a common way for asserting results of {@link SerializationSchema}. It takes + * into account e.g. the fact that serialization schema during runtime might be used after + * serializing and deserializing it over a wire. Usage: * *
    - *
  • when asserting for result after serializing and deserializing a row - *
    {@code
    - *      final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema);
    - *      final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema);
    + *   
  • when asserting for result after serializing and deserializing a row + *
    {@code
    + * final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema);
    + * final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema);
      *
    - *      assertThat(row, whenSerializedWith(serializationSchema)
    - *          .andDeserializedWith(deserializationSchema)
    - *          .matches(row));
    + * assertThat(row, whenSerializedWith(serializationSchema)
    + *     .andDeserializedWith(deserializationSchema)
    + *     .matches(row));
      * }
    - *
  • - * - *
  • to check if an exception is thrown during serialization: - *
    {@code
    - *      assertThat(row, whenSerializedWith(serializationSchema).failsWithException(instanceOf(RuntimeException.class)));
    + *   
  • to check if an exception is thrown during serialization: + *
    {@code
    + * assertThat(row, whenSerializedWith(serializationSchema).failsWithException(instanceOf(RuntimeException.class)));
      * }
    - *
  • *
*/ public abstract class SerializationSchemaMatcher extends TypeSafeMatcher { - final SerializationSchema serializationSchema; - - private SerializationSchemaMatcher(SerializationSchema serializationSchema) { - this.serializationSchema = serializationSchema; - } - - public static SerializationSchemaMatcherBuilder whenSerializedWith(SerializationSchema serializationSchema) { - return new SerializationSchemaMatcherBuilder(serializationSchema); - } - - private static class SerializationSchemaResultMatcher extends SerializationSchemaMatcher { - - private final Row expected; - private final DeserializationSchema deserializationSchema; - - private SerializationSchemaResultMatcher( - SerializationSchema serializationSchema, - DeserializationSchema deserializationSchema, - Row expected) { - super(serializationSchema); - - this.expected = expected; - this.deserializationSchema = deserializationSchema; - } - - @Override - protected boolean matchesSafely(Row item) { - try { - return Objects.deepEquals( - deserializationSchema.deserialize(serializationSchema.serialize(item)), - expected); - } catch (IOException e) { - throw new AssertionError("Could not deserialize", e); - } - } - - @Override - public void describeTo(Description description) { - description.appendValue(expected); - } - } - - private static class SerializationSchemaExceptionMatcher extends SerializationSchemaMatcher { - - private final Matcher exceptionMatcher; - private Throwable thrownException = null; - - private SerializationSchemaExceptionMatcher( - SerializationSchema serializationSchema, - Matcher exceptionMatcher) { - super(serializationSchema); - this.exceptionMatcher = exceptionMatcher; - } - - @Override - protected boolean matchesSafely(Row item) { - try { - serializationSchema.serialize(item); - } catch (Exception e) { - thrownException = e; - } - return exceptionMatcher.matches(thrownException); - } - - @Override - public void describeTo(Description description) { - exceptionMatcher.describeTo(description); - } - - @Override - protected void describeMismatchSafely(Row item, Description mismatchDescription) { - exceptionMatcher.describeMismatch(thrownException, mismatchDescription); - } - } - - /** - * Builder for {@link SerializationSchemaMatcher} that can assert results after serialize and deserialize. - */ - public static class SerializationWithDeserializationSchemaMatcherBuilder { - - private SerializationSchema serializationSchema; - private DeserializationSchema deserializationSchema; - - private SerializationWithDeserializationSchemaMatcherBuilder( - SerializationSchema serializationSchema, - DeserializationSchema deserializationSchema) { - try { - // we serialize and deserialize the schema to test runtime behavior - // when the schema is shipped to the cluster - this.serializationSchema = deserializeObject( - serializeObject(serializationSchema), - this.getClass().getClassLoader()); - this.deserializationSchema = deserializeObject( - serializeObject(deserializationSchema), - this.getClass().getClassLoader()); - } catch (IOException | ClassNotFoundException e) { - throw new RuntimeException(e); - } - } - - public SerializationSchemaMatcher equalsTo(Row expected) { - return new SerializationSchemaResultMatcher( - serializationSchema, - deserializationSchema, - expected - ); - } - } - - /** - * Builder for {@link SerializationSchemaMatcher}. - */ - public static class SerializationSchemaMatcherBuilder { - - private SerializationSchema serializationSchema; - - private SerializationSchemaMatcherBuilder(SerializationSchema serializationSchema) { - this.serializationSchema = serializationSchema; - } - - public SerializationWithDeserializationSchemaMatcherBuilder andDeserializedWith(DeserializationSchema deserializationSchema) { - return new SerializationWithDeserializationSchemaMatcherBuilder(serializationSchema, deserializationSchema); - } - - public SerializationSchemaMatcher failsWithException(Matcher exceptionMatcher) { - return new SerializationSchemaExceptionMatcher( - serializationSchema, - exceptionMatcher - ); - } - } + final SerializationSchema serializationSchema; + + private SerializationSchemaMatcher(SerializationSchema serializationSchema) { + this.serializationSchema = serializationSchema; + } + + public static SerializationSchemaMatcherBuilder whenSerializedWith( + SerializationSchema serializationSchema) { + return new SerializationSchemaMatcherBuilder(serializationSchema); + } + + private static class SerializationSchemaResultMatcher extends SerializationSchemaMatcher { + + private final Row expected; + private final DeserializationSchema deserializationSchema; + + private SerializationSchemaResultMatcher( + SerializationSchema serializationSchema, + DeserializationSchema deserializationSchema, + Row expected) { + super(serializationSchema); + + this.expected = expected; + this.deserializationSchema = deserializationSchema; + } + + @Override + protected boolean matchesSafely(Row item) { + try { + return Objects.deepEquals( + deserializationSchema.deserialize(serializationSchema.serialize(item)), + expected); + } catch (IOException e) { + throw new AssertionError("Could not deserialize", e); + } + } + + @Override + public void describeTo(Description description) { + description.appendValue(expected); + } + } + + private static class SerializationSchemaExceptionMatcher extends SerializationSchemaMatcher { + + private final Matcher exceptionMatcher; + private Throwable thrownException = null; + + private SerializationSchemaExceptionMatcher( + SerializationSchema serializationSchema, + Matcher exceptionMatcher) { + super(serializationSchema); + this.exceptionMatcher = exceptionMatcher; + } + + @Override + protected boolean matchesSafely(Row item) { + try { + serializationSchema.serialize(item); + } catch (Exception e) { + thrownException = e; + } + return exceptionMatcher.matches(thrownException); + } + + @Override + public void describeTo(Description description) { + exceptionMatcher.describeTo(description); + } + + @Override + protected void describeMismatchSafely(Row item, Description mismatchDescription) { + exceptionMatcher.describeMismatch(thrownException, mismatchDescription); + } + } + + /** + * Builder for {@link SerializationSchemaMatcher} that can assert results after serialize and + * deserialize. + */ + public static class SerializationWithDeserializationSchemaMatcherBuilder { + + private SerializationSchema serializationSchema; + private DeserializationSchema deserializationSchema; + + private SerializationWithDeserializationSchemaMatcherBuilder( + SerializationSchema serializationSchema, + DeserializationSchema deserializationSchema) { + try { + // we serialize and deserialize the schema to test runtime behavior + // when the schema is shipped to the cluster + this.serializationSchema = + deserializeObject( + serializeObject(serializationSchema), + this.getClass().getClassLoader()); + this.deserializationSchema = + deserializeObject( + serializeObject(deserializationSchema), + this.getClass().getClassLoader()); + } catch (IOException | ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + public SerializationSchemaMatcher equalsTo(Row expected) { + return new SerializationSchemaResultMatcher( + serializationSchema, deserializationSchema, expected); + } + } + + /** Builder for {@link SerializationSchemaMatcher}. */ + public static class SerializationSchemaMatcherBuilder { + + private SerializationSchema serializationSchema; + + private SerializationSchemaMatcherBuilder(SerializationSchema serializationSchema) { + this.serializationSchema = serializationSchema; + } + + public SerializationWithDeserializationSchemaMatcherBuilder andDeserializedWith( + DeserializationSchema deserializationSchema) { + return new SerializationWithDeserializationSchemaMatcherBuilder( + serializationSchema, deserializationSchema); + } + + public SerializationSchemaMatcher failsWithException( + Matcher exceptionMatcher) { + return new SerializationSchemaExceptionMatcher(serializationSchema, exceptionMatcher); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java index 5436b2d2d..89ed2135e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java @@ -29,118 +29,119 @@ import java.util.List; import java.util.Map; -/** - * Tests for the {@link Json} descriptor. - */ +/** Tests for the {@link Json} descriptor. */ public class JsonTest extends DescriptorTestBase { - private static final String JSON_SCHEMA = - "{" + - " 'title': 'Person'," + - " 'type': 'object'," + - " 'properties': {" + - " 'firstName': {" + - " 'type': 'string'" + - " }," + - " 'lastName': {" + - " 'type': 'string'" + - " }," + - " 'age': {" + - " 'description': 'Age in years'," + - " 'type': 'integer'," + - " 'minimum': 0" + - " }" + - " }," + - " 'required': ['firstName', 'lastName']" + - "}"; - - @Test(expected = ValidationException.class) - public void testInvalidMissingField() { - addPropertyAndVerify(descriptors().get(0), "format.fail-on-missing-field", "DDD"); - } - - @Test(expected = ValidationException.class) - public void testDuplicateSchema() { - // we add an additional non-json schema - addPropertyAndVerify(descriptors().get(0), "format.schema", "DDD"); - } - - @Test(expected = ValidationException.class) - public void testInvalidIgnoreParseErrors() { - addPropertyAndVerify(descriptors().get(0), "format.ignore-parse-errors", "DDD"); - } - - // -------------------------------------------------------------------------------------------- - - @Override - public List descriptors() { - final Descriptor desc1 = new Json().jsonSchema("test"); - - final Descriptor desc2 = new Json().jsonSchema(JSON_SCHEMA).failOnMissingField(true); - - final Descriptor desc3 = new Json() - .schema( - Types.ROW( - new String[]{"test1", "test2"}, - new TypeInformation[]{Types.STRING(), Types.SQL_TIMESTAMP()})) - .failOnMissingField(true); - - final Descriptor desc4 = new Json().deriveSchema(); - - final Descriptor desc5 = new Json().failOnMissingField(false); - - final Descriptor desc6 = new Json().jsonSchema(JSON_SCHEMA).ignoreParseErrors(false); - - final Descriptor desc7 = new Json().ignoreParseErrors(true); - - return Arrays.asList(desc1, desc2, desc3, desc4, desc5, desc6, desc7); - } - - @Override - public List> properties() { - final Map props1 = new HashMap<>(); - props1.put("format.type", "json"); - props1.put("format.property-version", "1"); - props1.put("format.json-schema", "test"); - - final Map props2 = new HashMap<>(); - props2.put("format.type", "json"); - props2.put("format.property-version", "1"); - props2.put("format.json-schema", JSON_SCHEMA); - props2.put("format.fail-on-missing-field", "true"); - - final Map props3 = new HashMap<>(); - props3.put("format.type", "json"); - props3.put("format.property-version", "1"); - props3.put("format.schema", "ROW"); - props3.put("format.fail-on-missing-field", "true"); - - final Map props4 = new HashMap<>(); - props4.put("format.type", "json"); - props4.put("format.property-version", "1"); - props4.put("format.derive-schema", "true"); - - final Map props5 = new HashMap<>(); - props5.put("format.type", "json"); - props5.put("format.property-version", "1"); - props5.put("format.fail-on-missing-field", "false"); - - final Map props6 = new HashMap<>(); - props6.put("format.type", "json"); - props6.put("format.property-version", "1"); - props6.put("format.json-schema", JSON_SCHEMA); - props6.put("format.ignore-parse-errors", "false"); - - final Map props7 = new HashMap<>(); - props7.put("format.type", "json"); - props7.put("format.property-version", "1"); - props7.put("format.ignore-parse-errors", "true"); - - return Arrays.asList(props1, props2, props3, props4, props5, props6, props7); - } - - @Override - public DescriptorValidator validator() { - return new JsonValidator(); - } + private static final String JSON_SCHEMA = + "{" + + " 'title': 'Person'," + + " 'type': 'object'," + + " 'properties': {" + + " 'firstName': {" + + " 'type': 'string'" + + " }," + + " 'lastName': {" + + " 'type': 'string'" + + " }," + + " 'age': {" + + " 'description': 'Age in years'," + + " 'type': 'integer'," + + " 'minimum': 0" + + " }" + + " }," + + " 'required': ['firstName', 'lastName']" + + "}"; + + @Test(expected = ValidationException.class) + public void testInvalidMissingField() { + addPropertyAndVerify(descriptors().get(0), "format.fail-on-missing-field", "DDD"); + } + + @Test(expected = ValidationException.class) + public void testDuplicateSchema() { + // we add an additional non-json schema + addPropertyAndVerify(descriptors().get(0), "format.schema", "DDD"); + } + + @Test(expected = ValidationException.class) + public void testInvalidIgnoreParseErrors() { + addPropertyAndVerify(descriptors().get(0), "format.ignore-parse-errors", "DDD"); + } + + // -------------------------------------------------------------------------------------------- + + @Override + public List descriptors() { + final Descriptor desc1 = new Json().jsonSchema("test"); + + final Descriptor desc2 = new Json().jsonSchema(JSON_SCHEMA).failOnMissingField(true); + + final Descriptor desc3 = + new Json() + .schema( + Types.ROW( + new String[] {"test1", "test2"}, + new TypeInformation[] { + Types.STRING(), Types.SQL_TIMESTAMP() + })) + .failOnMissingField(true); + + final Descriptor desc4 = new Json().deriveSchema(); + + final Descriptor desc5 = new Json().failOnMissingField(false); + + final Descriptor desc6 = new Json().jsonSchema(JSON_SCHEMA).ignoreParseErrors(false); + + final Descriptor desc7 = new Json().ignoreParseErrors(true); + + return Arrays.asList(desc1, desc2, desc3, desc4, desc5, desc6, desc7); + } + + @Override + public List> properties() { + final Map props1 = new HashMap<>(); + props1.put("format.type", "json"); + props1.put("format.property-version", "1"); + props1.put("format.json-schema", "test"); + + final Map props2 = new HashMap<>(); + props2.put("format.type", "json"); + props2.put("format.property-version", "1"); + props2.put("format.json-schema", JSON_SCHEMA); + props2.put("format.fail-on-missing-field", "true"); + + final Map props3 = new HashMap<>(); + props3.put("format.type", "json"); + props3.put("format.property-version", "1"); + props3.put("format.schema", "ROW"); + props3.put("format.fail-on-missing-field", "true"); + + final Map props4 = new HashMap<>(); + props4.put("format.type", "json"); + props4.put("format.property-version", "1"); + props4.put("format.derive-schema", "true"); + + final Map props5 = new HashMap<>(); + props5.put("format.type", "json"); + props5.put("format.property-version", "1"); + props5.put("format.fail-on-missing-field", "false"); + + final Map props6 = new HashMap<>(); + props6.put("format.type", "json"); + props6.put("format.property-version", "1"); + props6.put("format.json-schema", JSON_SCHEMA); + props6.put("format.ignore-parse-errors", "false"); + + final Map props7 = new HashMap<>(); + props7.put("format.type", "json"); + props7.put("format.property-version", "1"); + props7.put("format.ignore-parse-errors", "true"); + + return Arrays.asList(props1, props2, props3, props4, props5, props6, props7); + } + + @Override + public DescriptorValidator validator() { + return new JsonValidator(); + } } From e88df0331a1a2bdd2c0a1839f07c67570dd1338a Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 29 Dec 2020 11:35:15 +0100 Subject: [PATCH 120/322] [FLINK-20790][build][avro] Move generated files to target/generated[-test]-sources --- flink-confluent-schema-registry-e2e-tests/pom.xml | 4 ++-- .../src/main/{ => resources}/avro/user.avsc | 0 2 files changed, 2 insertions(+), 2 deletions(-) rename flink-confluent-schema-registry-e2e-tests/src/main/{ => resources}/avro/user.avsc (100%) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 9b706c61d..14b5ff40f 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -99,8 +99,8 @@ under the License. schema - ${project.basedir}/src/main/avro/ - ${project.basedir}/src/main/java/ + ${project.basedir}/src/main/resources/avro/ + ${project.basedir}/target/generated-sources/ PRIVATE **/*.avsc diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/avro/user.avsc b/flink-confluent-schema-registry-e2e-tests/src/main/resources/avro/user.avsc similarity index 100% rename from flink-confluent-schema-registry-e2e-tests/src/main/avro/user.avsc rename to flink-confluent-schema-registry-e2e-tests/src/main/resources/avro/user.avsc From 6a81b614e57e45b7d5f7857ccae7d7b85958708d Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Thu, 31 Dec 2020 10:34:00 +0800 Subject: [PATCH 121/322] [FLINK-20385][canal-json] Allow to read metadata for canal-json format This closes #14464 --- .../json/canal/CanalJsonDecodingFormat.java | 225 ++++++++++++++++++ .../canal/CanalJsonDeserializationSchema.java | 200 ++++++++++++---- .../json/canal/CanalJsonFormatFactory.java | 34 +-- .../canal/CanalJsonFormatFactoryTest.java | 16 +- .../json/canal/CanalJsonSerDeSchemaTest.java | 100 +++++++- 5 files changed, 479 insertions(+), 96 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java new file mode 100644 index 000000000..81e5287ac --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.canal; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.formats.json.canal.CanalJsonDeserializationSchema.MetadataConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.types.RowKind; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** {@link DecodingFormat} for Canal using JSON encoding. */ +public class CanalJsonDecodingFormat implements DecodingFormat> { + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + private List metadataKeys; + + // -------------------------------------------------------------------------------------------- + // Canal-specific attributes + // -------------------------------------------------------------------------------------------- + + private final @Nullable String database; + + private final @Nullable String table; + + private final boolean ignoreParseErrors; + + private final TimestampFormat timestampFormat; + + public CanalJsonDecodingFormat( + String database, + String table, + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { + this.database = database; + this.table = table; + this.ignoreParseErrors = ignoreParseErrors; + this.timestampFormat = timestampFormat; + this.metadataKeys = Collections.emptyList(); + } + + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType physicalDataType) { + final List readableMetadata = + metadataKeys.stream() + .map( + k -> + Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .collect(Collectors.toList()); + final List metadataFields = + readableMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList()); + final DataType producedDataType = + DataTypeUtils.appendRowFields(physicalDataType, metadataFields); + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + return CanalJsonDeserializationSchema.builder( + physicalDataType, readableMetadata, producedTypeInfo) + .setDatabase(database) + .setTable(table) + .setIgnoreParseErrors(ignoreParseErrors) + .setTimestampFormat(timestampFormat) + .build(); + } + + @Override + public Map listReadableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + Stream.of(ReadableMetadata.values()) + .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyReadableMetadata(List metadataKeys) { + this.metadataKeys = metadataKeys; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + + /** List of metadata that can be read with this format. */ + enum ReadableMetadata { + DATABASE( + "database", + DataTypes.STRING().nullable(), + DataTypes.FIELD("database", DataTypes.STRING()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getString(pos); + } + }), + + TABLE( + "table", + DataTypes.STRING().nullable(), + DataTypes.FIELD("table", DataTypes.STRING()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getString(pos); + } + }), + + SQL_TYPE( + "sql-type", + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.INT().nullable()).nullable(), + DataTypes.FIELD( + "sqlType", + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.INT().nullable())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getMap(pos); + } + }), + + PK_NAMES( + "pk-names", + DataTypes.ARRAY(DataTypes.STRING()).nullable(), + DataTypes.FIELD("pkNames", DataTypes.ARRAY(DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getArray(pos); + } + }), + + INGESTION_TIMESTAMP( + "ingestion-timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + DataTypes.FIELD("ts", DataTypes.BIGINT()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + return TimestampData.fromEpochMillis(row.getLong(pos)); + } + }); + + final String key; + + final DataType dataType; + + final DataTypes.Field requiredJsonField; + + final MetadataConverter converter; + + ReadableMetadata( + String key, + DataType dataType, + DataTypes.Field requiredJsonField, + MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.requiredJsonField = requiredJsonField; + this.converter = converter; + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index 6a81f315a..00bb7483b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -23,27 +23,31 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.formats.json.canal.CanalJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; import org.apache.flink.types.RowKind; import org.apache.flink.util.Collector; import javax.annotation.Nullable; import java.io.IOException; +import java.io.Serializable; +import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; import static java.lang.String.format; -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** * Deserialization schema from Canal JSON to Flink Table/SQL internal data structure {@link - * RowData}. The deserialization schema knows Debezium's schema definition and can extract the - * database data and convert into {@link RowData} with {@link RowKind}. + * RowData}. The deserialization schema knows Canal's schema definition and can extract the database + * data and convert into {@link RowData} with {@link RowKind}. * *

Deserializes a byte[] message as a JSON object and reads the specified fields. * @@ -59,11 +63,17 @@ public final class CanalJsonDeserializationSchema implements DeserializationSche private static final String OP_DELETE = "DELETE"; private static final String OP_CREATE = "CREATE"; - /** The deserializer to deserialize Debezium JSON data. */ + /** The deserializer to deserialize Canal JSON data. */ private final JsonRowDataDeserializationSchema jsonDeserializer; - /** TypeInformation of the produced {@link RowData}. * */ - private final TypeInformation resultTypeInfo; + /** Flag that indicates that an additional projection is required for metadata. */ + private final boolean hasMetadata; + + /** Metadata to be extracted for every record. */ + private final MetadataConverter[] metadataConverters; + + /** {@link TypeInformation} of the produced {@link RowData} (physical + meta data). */ + private final TypeInformation producedTypeInfo; /** Only read changelogs from the specific database. */ private final @Nullable String database; @@ -78,27 +88,31 @@ public final class CanalJsonDeserializationSchema implements DeserializationSche private final int fieldCount; private CanalJsonDeserializationSchema( - RowType rowType, - TypeInformation resultTypeInfo, + DataType physicalDataType, + List requestedMetadata, + TypeInformation producedTypeInfo, @Nullable String database, @Nullable String table, boolean ignoreParseErrors, - TimestampFormat timestampFormatOption) { - this.resultTypeInfo = resultTypeInfo; - this.database = database; - this.table = table; - this.ignoreParseErrors = ignoreParseErrors; - this.fieldCount = rowType.getFieldCount(); + TimestampFormat timestampFormat) { + final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata); this.jsonDeserializer = new JsonRowDataDeserializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - // the result type is never used, so it's fine to pass in Canal's result - // type - resultTypeInfo, + jsonRowType, + // the result type is never used, so it's fine to pass in the produced type + // info + producedTypeInfo, false, // ignoreParseErrors already contains the functionality of // failOnMissingField ignoreParseErrors, - timestampFormatOption); + timestampFormat); + this.hasMetadata = requestedMetadata.size() > 0; + this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata); + this.producedTypeInfo = producedTypeInfo; + this.database = database; + this.table = table; + this.ignoreParseErrors = ignoreParseErrors; + this.fieldCount = ((RowType) physicalDataType.getLogicalType()).getFieldCount(); } // ------------------------------------------------------------------------------------------ @@ -106,23 +120,31 @@ private CanalJsonDeserializationSchema( // ------------------------------------------------------------------------------------------ /** Creates A builder for building a {@link CanalJsonDeserializationSchema}. */ - public static Builder builder(RowType rowType, TypeInformation resultTypeInfo) { - return new Builder(rowType, resultTypeInfo); + public static Builder builder( + DataType physicalDataType, + List requestedMetadata, + TypeInformation producedTypeInfo) { + return new Builder(physicalDataType, requestedMetadata, producedTypeInfo); } /** A builder for creating a {@link CanalJsonDeserializationSchema}. */ @Internal public static final class Builder { - private final RowType rowType; - private final TypeInformation resultTypeInfo; + private final DataType physicalDataType; + private final List requestedMetadata; + private final TypeInformation producedTypeInfo; private String database = null; private String table = null; private boolean ignoreParseErrors = false; private TimestampFormat timestampFormat = TimestampFormat.SQL; - private Builder(RowType rowType, TypeInformation resultTypeInfo) { - this.rowType = rowType; - this.resultTypeInfo = resultTypeInfo; + private Builder( + DataType physicalDataType, + List requestedMetadata, + TypeInformation producedTypeInfo) { + this.physicalDataType = physicalDataType; + this.requestedMetadata = requestedMetadata; + this.producedTypeInfo = producedTypeInfo; } public Builder setDatabase(String database) { @@ -147,7 +169,13 @@ public Builder setTimestampFormat(TimestampFormat timestampFormat) { public CanalJsonDeserializationSchema build() { return new CanalJsonDeserializationSchema( - rowType, resultTypeInfo, database, table, ignoreParseErrors, timestampFormat); + physicalDataType, + requestedMetadata, + producedTypeInfo, + database, + table, + ignoreParseErrors, + timestampFormat); } } @@ -162,7 +190,7 @@ public RowData deserialize(byte[] message) throws IOException { @Override public void deserialize(byte[] message, Collector out) throws IOException { try { - RowData row = jsonDeserializer.deserialize(message); + GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); if (database != null) { String currentDatabase = row.getString(3).toString(); if (!database.equals(currentDatabase)) { @@ -180,9 +208,9 @@ public void deserialize(byte[] message, Collector out) throws IOExcepti // "data" field is an array of row, contains inserted rows ArrayData data = row.getArray(0); for (int i = 0; i < data.size(); i++) { - RowData insert = data.getRow(i, fieldCount); + GenericRowData insert = (GenericRowData) data.getRow(i, fieldCount); insert.setRowKind(RowKind.INSERT); - out.collect(insert); + emitRow(row, insert, out); } } else if (OP_UPDATE.equals(type)) { // "data" field is an array of row, contains new rows @@ -203,16 +231,16 @@ public void deserialize(byte[] message, Collector out) throws IOExcepti } before.setRowKind(RowKind.UPDATE_BEFORE); after.setRowKind(RowKind.UPDATE_AFTER); - out.collect(before); - out.collect(after); + emitRow(row, before, out); + emitRow(row, after, out); } } else if (OP_DELETE.equals(type)) { // "data" field is an array of row, contains deleted rows ArrayData data = row.getArray(0); for (int i = 0; i < data.size(); i++) { - RowData insert = data.getRow(i, fieldCount); + GenericRowData insert = (GenericRowData) data.getRow(i, fieldCount); insert.setRowKind(RowKind.DELETE); - out.collect(insert); + emitRow(row, insert, out); } } else if (OP_CREATE.equals(type)) { // "data" field is null and "type" is "CREATE" which means @@ -235,6 +263,27 @@ public void deserialize(byte[] message, Collector out) throws IOExcepti } } + private void emitRow( + GenericRowData rootRow, GenericRowData physicalRow, Collector out) { + // shortcut in case no output projection is required + if (!hasMetadata) { + out.collect(physicalRow); + return; + } + final int physicalArity = physicalRow.getArity(); + final int metadataArity = metadataConverters.length; + final GenericRowData producedRow = + new GenericRowData(physicalRow.getRowKind(), physicalArity + metadataArity); + for (int physicalPos = 0; physicalPos < physicalArity; physicalPos++) { + producedRow.setField(physicalPos, physicalRow.getField(physicalPos)); + } + for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { + producedRow.setField( + physicalArity + metadataPos, metadataConverters[metadataPos].convert(rootRow)); + } + out.collect(producedRow); + } + @Override public boolean isEndOfStream(RowData nextElement) { return false; @@ -242,7 +291,7 @@ public boolean isEndOfStream(RowData nextElement) { @Override public TypeInformation getProducedType() { - return resultTypeInfo; + return producedTypeInfo; } @Override @@ -254,26 +303,81 @@ public boolean equals(Object o) { return false; } CanalJsonDeserializationSchema that = (CanalJsonDeserializationSchema) o; - return ignoreParseErrors == that.ignoreParseErrors - && fieldCount == that.fieldCount - && Objects.equals(jsonDeserializer, that.jsonDeserializer) - && Objects.equals(resultTypeInfo, that.resultTypeInfo); + return Objects.equals(jsonDeserializer, that.jsonDeserializer) + && hasMetadata == that.hasMetadata + && Objects.equals(producedTypeInfo, that.producedTypeInfo) + && Objects.equals(database, that.database) + && Objects.equals(table, that.table) + && ignoreParseErrors == that.ignoreParseErrors + && fieldCount == that.fieldCount; } @Override public int hashCode() { - return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors, fieldCount); + return Objects.hash( + jsonDeserializer, + hasMetadata, + producedTypeInfo, + database, + table, + ignoreParseErrors, + fieldCount); } - private static RowType createJsonRowType(DataType databaseSchema) { + // -------------------------------------------------------------------------------------------- + + private static RowType createJsonRowType( + DataType physicalDataType, List readableMetadata) { // Canal JSON contains other information, e.g. "ts", "sql", but we don't need them - return (RowType) + DataType root = DataTypes.ROW( - DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)), - DataTypes.FIELD("old", DataTypes.ARRAY(databaseSchema)), - DataTypes.FIELD("type", DataTypes.STRING()), - DataTypes.FIELD("database", DataTypes.STRING()), - DataTypes.FIELD("table", DataTypes.STRING())) - .getLogicalType(); + DataTypes.FIELD("data", DataTypes.ARRAY(physicalDataType)), + DataTypes.FIELD("old", DataTypes.ARRAY(physicalDataType)), + DataTypes.FIELD("type", DataTypes.STRING()), + ReadableMetadata.DATABASE.requiredJsonField, + ReadableMetadata.TABLE.requiredJsonField); + // append fields that are required for reading metadata in the root + final List rootMetadataFields = + readableMetadata.stream() + .filter(m -> m != ReadableMetadata.DATABASE && m != ReadableMetadata.TABLE) + .map(m -> m.requiredJsonField) + .distinct() + .collect(Collectors.toList()); + return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType(); + } + + private static MetadataConverter[] createMetadataConverters( + RowType jsonRowType, List requestedMetadata) { + return requestedMetadata.stream() + .map(m -> convert(jsonRowType, m)) + .toArray(MetadataConverter[]::new); + } + + private static MetadataConverter convert(RowType jsonRowType, ReadableMetadata metadata) { + final int pos = jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); + return new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData root, int unused) { + return metadata.converter.convert(root, pos); + } + }; + } + + // -------------------------------------------------------------------------------------------- + + /** + * Converter that extracts a metadata field from the row that comes out of the JSON schema and + * converts it to the desired data type. + */ + interface MetadataConverter extends Serializable { + + // Method for top-level access. + default Object convert(GenericRowData row) { + return convert(row, -1); + } + + Object convert(GenericRowData row, int pos); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java index b199c6f2b..35e5919bf 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.formats.json.JsonOptions; @@ -29,7 +28,6 @@ import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DeserializationFormatFactory; import org.apache.flink.table.factories.DynamicTableFactory; @@ -67,36 +65,12 @@ public DecodingFormat> createDecodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); validateDecodingFormatOptions(formatOptions); + final String database = formatOptions.getOptional(DATABASE_INCLUDE).orElse(null); + final String table = formatOptions.getOptional(TABLE_INCLUDE).orElse(null); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); - String database = formatOptions.getOptional(DATABASE_INCLUDE).orElse(null); - String table = formatOptions.getOptional(TABLE_INCLUDE).orElse(null); + final TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - return new DecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType producedDataType) { - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation rowDataTypeInfo = - context.createTypeInformation(producedDataType); - return CanalJsonDeserializationSchema.builder(rowType, rowDataTypeInfo) - .setIgnoreParseErrors(ignoreParseErrors) - .setTimestampFormat(timestampFormatOption) - .setDatabase(database) - .setTable(table) - .build(); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - }; + return new CanalJsonDecodingFormat(database, table, ignoreParseErrors, timestampFormat); } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 633fa1a18..3a140a9a9 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -36,6 +36,7 @@ import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; @@ -43,6 +44,7 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; @@ -61,7 +63,9 @@ public class CanalJsonFormatFactoryTest extends TestLogger { .field("c", DataTypes.BOOLEAN()) .build(); - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + private static final DataType PHYSICAL_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); + + private static final RowType ROW_TYPE = (RowType) PHYSICAL_DATA_TYPE.getLogicalType(); private static final InternalTypeInfo ROW_TYPE_INFO = InternalTypeInfo.of(ROW_TYPE); @@ -71,7 +75,8 @@ public void testDefaultOptions() { // test Deser CanalJsonDeserializationSchema expectedDeser = - CanalJsonDeserializationSchema.builder(ROW_TYPE, ROW_TYPE_INFO) + CanalJsonDeserializationSchema.builder( + PHYSICAL_DATA_TYPE, Collections.emptyList(), ROW_TYPE_INFO) .setIgnoreParseErrors(false) .setTimestampFormat(TimestampFormat.SQL) .build(); @@ -98,7 +103,8 @@ public void testUserDefinedOptions() { // test Deser CanalJsonDeserializationSchema expectedDeser = - CanalJsonDeserializationSchema.builder(ROW_TYPE, ROW_TYPE_INFO) + CanalJsonDeserializationSchema.builder( + PHYSICAL_DATA_TYPE, Collections.emptyList(), ROW_TYPE_INFO) .setIgnoreParseErrors(true) .setTimestampFormat(TimestampFormat.ISO_8601) .setDatabase("mydb") @@ -198,7 +204,7 @@ private static DeserializationSchema createDeserializationSchema( (TestDynamicTableFactory.DynamicTableSourceMock) source; return scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); + ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); } private static SerializationSchema createSerializationSchema( @@ -217,6 +223,6 @@ private static SerializationSchema createSerializationSchema( (TestDynamicTableFactory.DynamicTableSinkMock) sink; return sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), SCHEMA.toRowDataType()); + new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index e390aad61..fbfd1544f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -20,9 +20,13 @@ import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.formats.json.canal.CanalJsonDecodingFormat.ReadableMetadata; +import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; import org.apache.flink.util.Collector; import org.junit.Rule; @@ -37,7 +41,9 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.function.Consumer; import java.util.stream.Collectors; import static org.apache.flink.table.api.DataTypes.FIELD; @@ -45,38 +51,78 @@ import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; +import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; /** Tests for {@link CanalJsonSerializationSchema} and {@link CanalJsonDeserializationSchema}. */ public class CanalJsonSerDeSchemaTest { @Rule public ExpectedException thrown = ExpectedException.none(); - private static final RowType SCHEMA = - (RowType) - ROW( - FIELD("id", INT().notNull()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", FLOAT())) - .getLogicalType(); + private static final DataType PHYSICAL_DATA_TYPE = + ROW( + FIELD("id", INT().notNull()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", FLOAT())); @Test public void testFilteringTables() throws Exception { List lines = readLines("canal-data-filter-table.txt"); CanalJsonDeserializationSchema deserializationSchema = - CanalJsonDeserializationSchema.builder(SCHEMA, InternalTypeInfo.of(SCHEMA)) + CanalJsonDeserializationSchema.builder( + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType())) .setDatabase("mydb") .setTable("product") .build(); runTest(lines, deserializationSchema); } + @Test + public void testDeserializationWithMetadata() throws Exception { + testDeserializationWithMetadata( + "canal-data.txt", + null, + null, + row -> { + assertThat(row.getInt(0), equalTo(101)); + assertThat(row.getString(1).toString(), equalTo("scooter")); + assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); + assertThat(row.getFloat(3), equalTo(3.14f)); + assertThat(row.getString(4).toString(), equalTo("inventory")); + assertThat(row.getString(5).toString(), equalTo("products2")); + assertThat(row.getMap(6).size(), equalTo(4)); + assertThat(row.getArray(7).getString(0).toString(), equalTo("id")); + assertThat(row.getTimestamp(8, 3).getMillisecond(), equalTo(1589373515477L)); + }); + testDeserializationWithMetadata( + "canal-data-filter-table.txt", + "mydb", + "product", + row -> { + assertThat(row.getInt(0), equalTo(101)); + assertThat(row.getString(1).toString(), equalTo("scooter")); + assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); + assertThat(row.getFloat(3), equalTo(3.14f)); + assertThat(row.getString(4).toString(), equalTo("mydb")); + assertThat(row.getString(5).toString(), equalTo("product")); + assertThat(row.getMap(6).size(), equalTo(4)); + assertThat(row.getArray(7).getString(0).toString(), equalTo("id")); + assertThat(row.getTimestamp(8, 3).getMillisecond(), equalTo(1598944146308L)); + }); + } + @Test public void testSerializationDeserialization() throws Exception { List lines = readLines("canal-data.txt"); CanalJsonDeserializationSchema deserializationSchema = - CanalJsonDeserializationSchema.builder(SCHEMA, InternalTypeInfo.of(SCHEMA)) + CanalJsonDeserializationSchema.builder( + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType())) .setIgnoreParseErrors(false) .setTimestampFormat(TimestampFormat.ISO_8601) .build(); @@ -119,8 +165,8 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali // WHERE id=110; // UPDATE product SET weight='5.17' WHERE id=111; // DELETE FROM product WHERE id=111; - // UPDATE products SET weight='5.17' WHERE id=102 or id = 101; - // DELETE FROM products WHERE id=102 or id = 103; + // UPDATE product SET weight='5.17' WHERE id=102 or id = 101; + // DELETE FROM product WHERE id=102 or id = 103; List expected = Arrays.asList( "+I(101,scooter,Small 2-wheel scooter,3.14)", @@ -156,7 +202,7 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali // test Serialization CanalJsonSerializationSchema serializationSchema = new CanalJsonSerializationSchema( - SCHEMA, + (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, "null"); @@ -199,6 +245,34 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali assertEquals(expectedResult, result); } + private void testDeserializationWithMetadata( + String resourceFile, String database, String table, Consumer testConsumer) + throws Exception { + // we only read the first line for keeping the test simple + final String firstLine = readLines(resourceFile).get(0); + final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); + final DataType producedDataType = + DataTypeUtils.appendRowFields( + PHYSICAL_DATA_TYPE, + requestedMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList())); + final CanalJsonDeserializationSchema deserializationSchema = + CanalJsonDeserializationSchema.builder( + PHYSICAL_DATA_TYPE, + requestedMetadata, + InternalTypeInfo.of(producedDataType.getLogicalType())) + .setDatabase(database) + .setTable(table) + .setIgnoreParseErrors(false) + .setTimestampFormat(TimestampFormat.ISO_8601) + .build(); + final SimpleCollector collector = new SimpleCollector(); + deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); + assertEquals(9, collector.list.size()); + testConsumer.accept(collector.list.get(0)); + } + // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- From a3a93685920af3fbd3a4d5fed01f28f5fa1381d9 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 4 Jan 2021 14:06:36 +0100 Subject: [PATCH 122/322] [FLINK-20841][git] Remove .gitignore entries for generated files --- flink-confluent-schema-registry-e2e-tests/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 flink-confluent-schema-registry-e2e-tests/.gitignore diff --git a/flink-confluent-schema-registry-e2e-tests/.gitignore b/flink-confluent-schema-registry-e2e-tests/.gitignore deleted file mode 100644 index b83a75282..000000000 --- a/flink-confluent-schema-registry-e2e-tests/.gitignore +++ /dev/null @@ -1 +0,0 @@ -src/main/java/example/avro From 12991123c0eb4855dbfa1ac1058182e4ff201b9c Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 18 Dec 2020 11:32:55 +0100 Subject: [PATCH 123/322] [FLINK-18090] Update tests for new Row.toString All tests in modules apart from the Blink planner/runtime module have been updated. Otherwise we use a JUnit rule to make the migration of the remaining tests incremental. This closes #14568. --- .../json/JsonBatchFileSystemITCase.java | 4 ++ .../DebeziumJsonFileSystemITCase.java | 44 +++++++++---------- 2 files changed, 26 insertions(+), 22 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java index c677a2f9d..6cb7e080b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java @@ -20,10 +20,12 @@ import org.apache.flink.table.api.TableResult; import org.apache.flink.table.planner.runtime.batch.sql.BatchFileSystemITCaseBase; +import org.apache.flink.table.utils.LegacyRowResource; import org.apache.flink.types.Row; import org.apache.flink.util.FileUtils; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; import java.io.File; @@ -37,6 +39,8 @@ /** ITCase to test json format for {@link JsonFormatFactory}. */ public class JsonBatchFileSystemITCase extends BatchFileSystemITCaseBase { + @Rule public final LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE; + @Override public String[] formatProperties() { List ret = new ArrayList<>(); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java index 529fb71ad..7ab0e90af 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java @@ -42,26 +42,26 @@ public class DebeziumJsonFileSystemITCase extends StreamingTestBase { private static final List EXPECTED = Arrays.asList( - "+I(101,SCOOTER,Small 2-wheel scooter,3.14)", - "+I(102,CAR BATTERY,12V car battery,8.1)", - "+I(103,12-PACK DRILL BITS,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", - "+I(104,HAMMER,12oz carpenter's hammer,0.75)", - "+I(105,HAMMER,14oz carpenter's hammer,0.875)", - "+I(106,HAMMER,16oz carpenter's hammer,1.0)", - "+I(107,ROCKS,box of assorted rocks,5.3)", - "+I(108,JACKET,water resistent black wind breaker,0.1)", - "+I(109,SPARE TIRE,24 inch spare tire,22.2)", - "-D(106,HAMMER,16oz carpenter's hammer,1.0)", // -U - "+I(106,HAMMER,18oz carpenter hammer,1.0)", // +U - "-D(107,ROCKS,box of assorted rocks,5.3)", // -U - "+I(107,ROCKS,box of assorted rocks,5.1)", // +U - "+I(110,JACKET,water resistent white wind breaker,0.2)", - "+I(111,SCOOTER,Big 2-wheel scooter ,5.18)", - "-D(110,JACKET,water resistent white wind breaker,0.2)", // -U - "+I(110,JACKET,new water resistent white wind breaker,0.5)", // +U - "-D(111,SCOOTER,Big 2-wheel scooter ,5.18)", // -U - "+I(111,SCOOTER,Big 2-wheel scooter ,5.17)", // +U - "-D(111,SCOOTER,Big 2-wheel scooter ,5.17)"); + "+I[101, SCOOTER, Small 2-wheel scooter, 3.14]", + "+I[102, CAR BATTERY, 12V car battery, 8.1]", + "+I[103, 12-PACK DRILL BITS, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+I[104, HAMMER, 12oz carpenter's hammer, 0.75]", + "+I[105, HAMMER, 14oz carpenter's hammer, 0.875]", + "+I[106, HAMMER, 16oz carpenter's hammer, 1.0]", + "+I[107, ROCKS, box of assorted rocks, 5.3]", + "+I[108, JACKET, water resistent black wind breaker, 0.1]", + "+I[109, SPARE TIRE, 24 inch spare tire, 22.2]", + "-D[106, HAMMER, 16oz carpenter's hammer, 1.0]", // -U + "+I[106, HAMMER, 18oz carpenter hammer, 1.0]", // +U + "-D[107, ROCKS, box of assorted rocks, 5.3]", // -U + "+I[107, ROCKS, box of assorted rocks, 5.1]", // +U + "+I[110, JACKET, water resistent white wind breaker, 0.2]", + "+I[111, SCOOTER, Big 2-wheel scooter , 5.18]", + "-D[110, JACKET, water resistent white wind breaker, 0.2]", // -U + "+I[110, JACKET, new water resistent white wind breaker, 0.5]", // +U + "-D[111, SCOOTER, Big 2-wheel scooter , 5.18]", // -U + "+I[111, SCOOTER, Big 2-wheel scooter , 5.17]", // +U + "-D[111, SCOOTER, Big 2-wheel scooter , 5.17]"); private File source; private File sink; @@ -113,7 +113,7 @@ public void testNonPartition() throws Exception { List results = CollectionUtil.iteratorToList(iter).stream() - .map(row -> row.getKind().shortString() + "(" + row.toString() + ")") + .map(Row::toString) .collect(Collectors.toList()); iter.close(); @@ -137,7 +137,7 @@ public void testPartition() throws Exception { List results = list.stream() .map(row -> Row.project(row, new int[] {0, 1, 2, 3})) - .map(row -> row.getKind().shortString() + "(" + row.toString() + ")") + .map(Row::toString) .collect(Collectors.toList()); Assert.assertEquals(EXPECTED, results); From b53a936d86cf04a4052e292e6f981e5f150138c4 Mon Sep 17 00:00:00 2001 From: V1ncentzzZ <45353183+V1ncentzzZ@users.noreply.github.com> Date: Fri, 8 Jan 2021 12:13:18 +0800 Subject: [PATCH 124/322] [FLINK-20773][json] Support to parse unescaped control chars in string node This closes #14508 --- .../flink/formats/json/JsonRowDataDeserializationSchema.java | 2 ++ .../flink/formats/json/JsonRowDataSerDeSchemaTest.java | 5 ++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index a0c417563..ebdad24dd 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -26,6 +26,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.json.JsonReadFeature; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -90,6 +91,7 @@ public JsonRowDataDeserializationSchema( if (hasDecimalType) { objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); } + objectMapper.configure(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(), true); } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index ae3c2c2d4..633c9b081 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -700,7 +700,10 @@ private void testParseErrors(TestSpec spec) throws Exception { "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12+0800\"}'."), TestSpec.json("{\"id\":1,\"factor\":799.929496989092949698}") .rowType(ROW(FIELD("id", INT()), FIELD("factor", DECIMAL(38, 18)))) - .expect(Row.of(1, new BigDecimal("799.929496989092949698")))); + .expect(Row.of(1, new BigDecimal("799.929496989092949698"))), + TestSpec.json("{\"id\":\"\tstring field\"}") // test to parse control chars + .rowType(ROW(FIELD("id", STRING()))) + .expect(Row.of("\tstring field"))); private static Map createHashMap( String k1, Integer v1, String k2, Integer v2) { From 0b7fce062c7059baf2060700f73c766bd5cf9341 Mon Sep 17 00:00:00 2001 From: Xue Wang Date: Fri, 8 Jan 2021 22:31:32 +0800 Subject: [PATCH 125/322] [FLINK-20321][formats] Fix NPE when using Avro/Json/Csv formats to deserialize null input (#14539) This closes #14539 Co-authored-by: Alex Wang --- .../RegistryAvroRowDataSeDeSchemaTest.java | 3 ++ .../JsonRowDataDeserializationSchema.java | 7 ++- .../canal/CanalJsonDeserializationSchema.java | 5 +- .../json/JsonRowDataSerDeSchemaTest.java | 13 ++++++ .../json/canal/CanalJsonSerDeSchemaTest.java | 46 +++++++++++++------ 5 files changed, 58 insertions(+), 16 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java index 092bc1d21..0061e8b63 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java @@ -54,6 +54,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; /** @@ -149,6 +150,8 @@ private void testRowDataWriteReadWithSchema(Schema schema) throws Exception { serializer.open(null); deserializer.open(null); + assertNull(deserializer.deserialize(null)); + RowData oriData = address2RowData(address); byte[] serialized = serializer.serialize(oriData); RowData rowData = deserializer.deserialize(serialized); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index ebdad24dd..585150148 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -31,6 +31,8 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.Objects; @@ -95,7 +97,10 @@ public JsonRowDataDeserializationSchema( } @Override - public RowData deserialize(byte[] message) throws IOException { + public RowData deserialize(@Nullable byte[] message) throws IOException { + if (message == null) { + return null; + } try { final JsonNode root = objectMapper.readTree(message); return (RowData) runtimeConverter.convert(root); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index 00bb7483b..d823fd939 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -188,7 +188,10 @@ public RowData deserialize(byte[] message) throws IOException { } @Override - public void deserialize(byte[] message, Collector out) throws IOException { + public void deserialize(@Nullable byte[] message, Collector out) throws IOException { + if (message == null || message.length == 0) { + return; + } try { GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); if (database != null) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 633c9b081..28947eda0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -72,6 +72,7 @@ import static org.apache.flink.table.api.DataTypes.TINYINT; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; /** @@ -377,6 +378,18 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { } } + @Test + public void testDeserializationNullRow() throws Exception { + DataType dataType = ROW(FIELD("name", STRING())); + RowType schema = (RowType) dataType.getLogicalType(); + + JsonRowDataDeserializationSchema deserializationSchema = + new JsonRowDataDeserializationSchema( + schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); + + assertNull(deserializationSchema.deserialize(null)); + } + @Test public void testDeserializationMissingNode() throws Exception { DataType dataType = ROW(FIELD("name", STRING())); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index fbfd1544f..1b5b90bf0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -81,6 +81,18 @@ public void testFilteringTables() throws Exception { runTest(lines, deserializationSchema); } + @Test + public void testDeserializeNullRow() throws Exception { + final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); + final CanalJsonDeserializationSchema deserializationSchema = + createCanalJsonDeserializationSchema(null, null, requestedMetadata); + final SimpleCollector collector = new SimpleCollector(); + + deserializationSchema.deserialize(null, collector); + deserializationSchema.deserialize(new byte[0], collector); + assertEquals(0, collector.list.size()); + } + @Test public void testDeserializationWithMetadata() throws Exception { testDeserializationWithMetadata( @@ -251,26 +263,32 @@ private void testDeserializationWithMetadata( // we only read the first line for keeping the test simple final String firstLine = readLines(resourceFile).get(0); final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); + final CanalJsonDeserializationSchema deserializationSchema = + createCanalJsonDeserializationSchema(database, table, requestedMetadata); + final SimpleCollector collector = new SimpleCollector(); + + deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); + assertEquals(9, collector.list.size()); + testConsumer.accept(collector.list.get(0)); + } + + private CanalJsonDeserializationSchema createCanalJsonDeserializationSchema( + String database, String table, List requestedMetadata) { final DataType producedDataType = DataTypeUtils.appendRowFields( PHYSICAL_DATA_TYPE, requestedMetadata.stream() .map(m -> DataTypes.FIELD(m.key, m.dataType)) .collect(Collectors.toList())); - final CanalJsonDeserializationSchema deserializationSchema = - CanalJsonDeserializationSchema.builder( - PHYSICAL_DATA_TYPE, - requestedMetadata, - InternalTypeInfo.of(producedDataType.getLogicalType())) - .setDatabase(database) - .setTable(table) - .setIgnoreParseErrors(false) - .setTimestampFormat(TimestampFormat.ISO_8601) - .build(); - final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - assertEquals(9, collector.list.size()); - testConsumer.accept(collector.list.get(0)); + return CanalJsonDeserializationSchema.builder( + PHYSICAL_DATA_TYPE, + requestedMetadata, + InternalTypeInfo.of(producedDataType.getLogicalType())) + .setDatabase(database) + .setTable(table) + .setIgnoreParseErrors(false) + .setTimestampFormat(TimestampFormat.ISO_8601) + .build(); } // -------------------------------------------------------------------------------------------- From 42cd7331bff546ee0e2118608e96c2ee0ba2d694 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Sat, 9 Jan 2021 20:24:25 +0800 Subject: [PATCH 126/322] [FLINK-20906][legal] Update copyright year to 2021 for NOTICE files. This closes #14598 --- .../src/main/resources/META-INF/NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 0d9003214..15ccc70d7 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-avro-confluent-registry -Copyright 2014-2020 The Apache Software Foundation +Copyright 2014-2021 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From 6be35ca835f0638172237f766814e38798269684 Mon Sep 17 00:00:00 2001 From: LM Kang Date: Wed, 13 Jan 2021 20:15:49 +0800 Subject: [PATCH 127/322] [FLINK-20861][json] Introduce an option for serializing DECIMALs in JSON as plain number instead of scientific notation This closes #14604 --- .../flink/formats/json/JsonFormatFactory.java | 11 ++- .../flink/formats/json/JsonOptions.java | 7 ++ .../json/JsonRowDataSerializationSchema.java | 23 +++++- .../json/canal/CanalJsonFormatFactory.java | 11 ++- .../canal/CanalJsonSerializationSchema.java | 6 +- .../debezium/DebeziumJsonFormatFactory.java | 11 ++- .../DebeziumJsonSerializationSchema.java | 6 +- .../maxwell/MaxwellJsonFormatFactory.java | 11 ++- .../MaxwellJsonSerializationSchema.java | 6 +- .../formats/json/JsonFormatFactoryTest.java | 4 +- .../json/JsonRowDataSerDeSchemaTest.java | 75 +++++++++++++++++-- .../canal/CanalJsonFormatFactoryTest.java | 10 ++- .../json/canal/CanalJsonSerDeSchemaTest.java | 3 +- .../DebeziumJsonFormatFactoryTest.java | 4 +- .../debezium/DebeziumJsonSerDeSchemaTest.java | 3 +- .../maxwell/MaxwellJsonFormatFactoryTest.java | 4 +- .../json/maxwell/MaxwellJsonSerDerTest.java | 6 +- 17 files changed, 172 insertions(+), 29 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index 386bdd8db..51338b273 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -40,6 +40,7 @@ import java.util.HashSet; import java.util.Set; +import static org.apache.flink.formats.json.JsonOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; import static org.apache.flink.formats.json.JsonOptions.FAIL_ON_MISSING_FIELD; import static org.apache.flink.formats.json.JsonOptions.IGNORE_PARSE_ERRORS; import static org.apache.flink.formats.json.JsonOptions.MAP_NULL_KEY_LITERAL; @@ -98,13 +99,20 @@ public EncodingFormat> createEncodingFormat( JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); String mapNullKeyLiteral = formatOptions.get(MAP_NULL_KEY_LITERAL); + final boolean encodeDecimalAsPlainNumber = + formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); + return new EncodingFormat>() { @Override public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); return new JsonRowDataSerializationSchema( - rowType, timestampOption, mapNullKeyMode, mapNullKeyLiteral); + rowType, + timestampOption, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); } @Override @@ -132,6 +140,7 @@ public Set> optionalOptions() { options.add(TIMESTAMP_FORMAT); options.add(MAP_NULL_KEY_MODE); options.add(MAP_NULL_KEY_LITERAL); + options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); return options; } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java index ac21981cf..3ffa18bbf 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java @@ -73,6 +73,13 @@ public class JsonOptions { + " Option ISO-8601 will parse input timestamp in \"yyyy-MM-ddTHH:mm:ss.s{precision}\" format and output timestamp in the same format." + " Option SQL will parse input timestamp in \"yyyy-MM-dd HH:mm:ss.s{precision}\" format and output timestamp in the same format."); + public static final ConfigOption ENCODE_DECIMAL_AS_PLAIN_NUMBER = + ConfigOptions.key("encode.decimal-as-plain-number") + .booleanType() + .defaultValue(false) + .withDescription( + "Optional flag to specify whether to encode all decimals as plain numbers instead of possible scientific notations, false by default."); + // -------------------------------------------------------------------------------------------- // Option enumerations // -------------------------------------------------------------------------------------------- diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index a7d3d674c..b2afe6200 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -23,6 +23,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; @@ -59,21 +60,29 @@ public class JsonRowDataSerializationSchema implements SerializationSchema> createEncodingFormat( JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); + final boolean encodeDecimalAsPlainNumber = + formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); + return new EncodingFormat>() { @Override public ChangelogMode getChangelogMode() { @@ -100,7 +104,11 @@ public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); return new CanalJsonSerializationSchema( - rowType, timestampFormat, mapNullKeyMode, mapNullKeyLiteral); + rowType, + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); } }; } @@ -124,6 +132,7 @@ public Set> optionalOptions() { options.add(TABLE_INCLUDE); options.add(JSON_MAP_NULL_KEY_MODE); options.add(JSON_MAP_NULL_KEY_LITERAL); + options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); return options; } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java index 9b5a2c03e..e80de2bff 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java @@ -58,13 +58,15 @@ public CanalJsonSerializationSchema( RowType rowType, TimestampFormat timestampFormat, JsonOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral) { + String mapNullKeyLiteral, + boolean encodeDecimalAsPlainNumber) { jsonSerializer = new JsonRowDataSerializationSchema( createJsonRowType(fromLogicalToDataType(rowType)), timestampFormat, mapNullKeyMode, - mapNullKeyLiteral); + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java index 79934a3b1..bed3dd645 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -41,6 +41,7 @@ import java.util.HashSet; import java.util.Set; +import static org.apache.flink.formats.json.JsonOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; import static org.apache.flink.formats.json.debezium.DebeziumJsonOptions.IGNORE_PARSE_ERRORS; import static org.apache.flink.formats.json.debezium.DebeziumJsonOptions.JSON_MAP_NULL_KEY_LITERAL; import static org.apache.flink.formats.json.debezium.DebeziumJsonOptions.JSON_MAP_NULL_KEY_MODE; @@ -85,6 +86,9 @@ public EncodingFormat> createEncodingFormat( JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); + final boolean encodeDecimalAsPlainNumber = + formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); + return new EncodingFormat>() { @Override @@ -102,7 +106,11 @@ public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); return new DebeziumJsonSerializationSchema( - rowType, timestampFormat, mapNullKeyMode, mapNullKeyLiteral); + rowType, + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); } }; } @@ -125,6 +133,7 @@ public Set> optionalOptions() { options.add(TIMESTAMP_FORMAT); options.add(JSON_MAP_NULL_KEY_MODE); options.add(JSON_MAP_NULL_KEY_LITERAL); + options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); return options; } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java index 4f71a09b5..9bde53db5 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java @@ -55,13 +55,15 @@ public DebeziumJsonSerializationSchema( RowType rowType, TimestampFormat timestampFormat, JsonOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral) { + String mapNullKeyLiteral, + boolean encodeDecimalAsPlainNumber) { jsonSerializer = new JsonRowDataSerializationSchema( createJsonRowType(fromLogicalToDataType(rowType)), timestampFormat, mapNullKeyMode, - mapNullKeyLiteral); + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java index 0a57cbe43..c8d47f0a6 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java @@ -43,6 +43,7 @@ import java.util.HashSet; import java.util.Set; +import static org.apache.flink.formats.json.JsonOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; import static org.apache.flink.formats.json.maxwell.MaxwellJsonOptions.IGNORE_PARSE_ERRORS; import static org.apache.flink.formats.json.maxwell.MaxwellJsonOptions.JSON_MAP_NULL_KEY_LITERAL; import static org.apache.flink.formats.json.maxwell.MaxwellJsonOptions.JSON_MAP_NULL_KEY_MODE; @@ -101,6 +102,9 @@ public EncodingFormat> createEncodingFormat( JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); + final boolean encodeDecimalAsPlainNumber = + formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); + return new EncodingFormat>() { @Override @@ -118,7 +122,11 @@ public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); return new MaxwellJsonSerializationSchema( - rowType, timestampFormat, mapNullKeyMode, mapNullKeyLiteral); + rowType, + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); } }; } @@ -140,6 +148,7 @@ public Set> optionalOptions() { options.add(TIMESTAMP_FORMAT); options.add(JSON_MAP_NULL_KEY_MODE); options.add(JSON_MAP_NULL_KEY_LITERAL); + options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); return options; } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java index 36c52ae6f..66e534b8c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java @@ -55,13 +55,15 @@ public MaxwellJsonSerializationSchema( RowType rowType, TimestampFormat timestampFormat, JsonOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral) { + String mapNullKeyLiteral, + boolean encodeDecimalAsPlainNumber) { this.jsonSerializer = new JsonRowDataSerializationSchema( createJsonRowType(fromLogicalToDataType(rowType)), timestampFormat, mapNullKeyMode, - mapNullKeyLiteral); + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); this.timestampFormat = timestampFormat; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index ba6e23958..ffc64e45d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -174,7 +174,8 @@ private void testSchemaSerializationSchema(Map options) { ROW_TYPE, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, - "null"); + "null", + true); final DynamicTableSink actualSink = createTableSink(options); assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; @@ -211,6 +212,7 @@ private Map getAllOptions() { options.put("json.timestamp-format.standard", "ISO-8601"); options.put("json.map-null-key.mode", "LITERAL"); options.put("json.map-null-key.literal", "null"); + options.put("json.encode.decimal-as-plain-number", "true"); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 28947eda0..7a565124f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -195,7 +195,8 @@ public void testSerDe() throws Exception { schema, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, - "null"); + "null", + true); byte[] actualBytes = serializationSchema.serialize(rowData); assertEquals(new String(serializedJson), new String(actualBytes)); @@ -284,7 +285,8 @@ public void testSerDeMultiRows() throws Exception { rowType, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, - "null"); + "null", + true); ObjectMapper objectMapper = new ObjectMapper(); @@ -368,7 +370,8 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { rowType, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, - "null"); + "null", + true); for (int i = 0; i < jsons.length; i++) { String json = jsons[i]; @@ -484,7 +487,11 @@ public void testSerDeSQLTimestampFormat() throws Exception { rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema( - rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, "null"); + rowType, + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.LITERAL, + "null", + true); ObjectMapper objectMapper = new ObjectMapper(); @@ -523,7 +530,11 @@ public void testSerializationMapNullKey() throws Exception { JsonRowDataSerializationSchema serializationSchema1 = new JsonRowDataSerializationSchema( - rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.FAIL, "null"); + rowType, + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.FAIL, + "null", + true); // expect message for serializationSchema1 String errorMessage1 = "JSON format doesn't support to serialize map data with null keys." @@ -531,7 +542,11 @@ public void testSerializationMapNullKey() throws Exception { JsonRowDataSerializationSchema serializationSchema2 = new JsonRowDataSerializationSchema( - rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.DROP, "null"); + rowType, + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.DROP, + "null", + true); // expect result for serializationSchema2 String expectResult2 = "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1}}}"; @@ -540,7 +555,8 @@ public void testSerializationMapNullKey() throws Exception { rowType, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, - "nullKey"); + "nullKey", + true); // expect result for serializationSchema3 String expectResult3 = "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1,\"nullKey\":2},\"nullKey\":{\"no-null key\":1,\"nullKey\":2}}}"; @@ -562,6 +578,51 @@ public void testSerializationMapNullKey() throws Exception { assertEquals(expectResult3, new String(actual3)); } + @Test + public void testSerializationDecimalEncode() throws Exception { + RowType schema = + (RowType) + ROW( + FIELD("decimal1", DECIMAL(9, 6)), + FIELD("decimal2", DECIMAL(20, 0)), + FIELD("decimal3", DECIMAL(11, 9))) + .getLogicalType(); + + TypeInformation resultTypeInfo = InternalTypeInfo.of(schema); + + JsonRowDataDeserializationSchema deserializer = + new JsonRowDataDeserializationSchema( + schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); + + JsonRowDataSerializationSchema plainDecimalSerializer = + new JsonRowDataSerializationSchema( + schema, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null", + true); + JsonRowDataSerializationSchema scientificDecimalSerializer = + new JsonRowDataSerializationSchema( + schema, + TimestampFormat.ISO_8601, + JsonOptions.MapNullKeyMode.LITERAL, + "null", + false); + + String plainDecimalJson = + "{\"decimal1\":123.456789,\"decimal2\":454621864049246170,\"decimal3\":0.000000027}"; + RowData rowData = deserializer.deserialize(plainDecimalJson.getBytes()); + + String plainDecimalResult = new String(plainDecimalSerializer.serialize(rowData)); + assertEquals(plainDecimalJson, plainDecimalResult); + + String scientificDecimalJson = + "{\"decimal1\":123.456789,\"decimal2\":4.5462186404924617E+17,\"decimal3\":2.7E-8}"; + + String scientificDecimalResult = new String(scientificDecimalSerializer.serialize(rowData)); + assertEquals(scientificDecimalJson, scientificDecimalResult); + } + @Test public void testJsonParse() throws Exception { for (TestSpec spec : testData) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 3a140a9a9..10adde092 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -86,7 +86,11 @@ public void testDefaultOptions() { // test Ser CanalJsonSerializationSchema expectedSer = new CanalJsonSerializationSchema( - ROW_TYPE, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.FAIL, "null"); + ROW_TYPE, + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.FAIL, + "null", + false); SerializationSchema actualSer = createSerializationSchema(options); assertEquals(expectedSer, actualSer); } @@ -100,6 +104,7 @@ public void testUserDefinedOptions() { options.put("canal-json.table.include", "mytable"); options.put("canal-json.map-null-key.mode", "LITERAL"); options.put("canal-json.map-null-key.literal", "nullKey"); + options.put("canal-json.encode.decimal-as-plain-number", "true"); // test Deser CanalJsonDeserializationSchema expectedDeser = @@ -119,7 +124,8 @@ public void testUserDefinedOptions() { ROW_TYPE, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, - "nullKey"); + "nullKey", + true); SerializationSchema actualSer = createSerializationSchema(options); assertEquals(expectedSer, actualSer); } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index 1b5b90bf0..51e82ece5 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -217,7 +217,8 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, - "null"); + "null", + true); serializationSchema.open(null); List result = new ArrayList<>(); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 92f823013..cda0188f3 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -95,7 +95,8 @@ public void testSeDeSchema() { (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, - "null"); + "null", + true); final DynamicTableSink actualSink = createTableSink(options); assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; @@ -212,6 +213,7 @@ private Map getAllOptions() { options.put("debezium-json.timestamp-format.standard", "ISO-8601"); options.put("debezium-json.map-null-key.mode", "LITERAL"); options.put("debezium-json.map-null-key.literal", "null"); + options.put("debezium-json.encode.decimal-as-plain-number", "true"); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java index 4fe152089..db98d89d8 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -257,7 +257,8 @@ private void testSerializationDeserialization(String resourceFile, boolean schem (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, - "null"); + "null", + true); serializationSchema.open(null); actual = new ArrayList<>(); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java index e7d691018..d7c12acd3 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java @@ -74,7 +74,8 @@ public void testSeDeSchema() { ROW_TYPE, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, - "null"); + "null", + true); final Map options = getAllOptions(); @@ -167,6 +168,7 @@ private Map getAllOptions() { options.put("maxwell-json.timestamp-format.standard", "ISO-8601"); options.put("maxwell-json.map-null-key.mode", "LITERAL"); options.put("maxwell-json.map-null-key.literal", "null"); + options.put("maxwell-json.encode.decimal-as-plain-number", "true"); return options; } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java index 99f432fb7..8ae814312 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java @@ -135,7 +135,11 @@ public void testSerializationDeserialization() throws Exception { MaxwellJsonSerializationSchema serializationSchema = new MaxwellJsonSerializationSchema( - SCHEMA, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, "null"); + SCHEMA, + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.LITERAL, + "null", + true); serializationSchema.open(null); List result = new ArrayList<>(); for (RowData rowData : collector.list) { From 842359a65096b975746ca697bb4766251d2509b9 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Thu, 14 Jan 2021 22:05:07 +0800 Subject: [PATCH 128/322] [FLINK-20885][canal-json] Fix deserialization exception when using 'canal-json.table.include' to filter binlogs of multiple tables This closes #14631 --- .../json/JsonRowDataDeserializationSchema.java | 11 +++++++++-- .../json/canal/CanalJsonDeserializationSchema.java | 11 ++++++----- .../formats/json/JsonRowDataSerDeSchemaTest.java | 9 ++------- .../src/test/resources/canal-data-filter-table.txt | 4 +++- 4 files changed, 20 insertions(+), 15 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index 585150148..c54cefdfa 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -102,8 +102,7 @@ public RowData deserialize(@Nullable byte[] message) throws IOException { return null; } try { - final JsonNode root = objectMapper.readTree(message); - return (RowData) runtimeConverter.convert(root); + return convertToRowData(deserializeToJsonNode(message)); } catch (Throwable t) { if (ignoreParseErrors) { return null; @@ -113,6 +112,14 @@ public RowData deserialize(@Nullable byte[] message) throws IOException { } } + public JsonNode deserializeToJsonNode(byte[] message) throws IOException { + return objectMapper.readTree(message); + } + + public RowData convertToRowData(JsonNode message) { + return (RowData) runtimeConverter.convert(message); + } + @Override public boolean isEndOfStream(RowData nextElement) { return false; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index d823fd939..667741b14 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -34,6 +34,8 @@ import org.apache.flink.types.RowKind; import org.apache.flink.util.Collector; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; + import javax.annotation.Nullable; import java.io.IOException; @@ -193,19 +195,18 @@ public void deserialize(@Nullable byte[] message, Collector out) throws return; } try { - GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); + final JsonNode root = jsonDeserializer.deserializeToJsonNode(message); if (database != null) { - String currentDatabase = row.getString(3).toString(); - if (!database.equals(currentDatabase)) { + if (!database.equals(root.get(ReadableMetadata.DATABASE.key).asText())) { return; } } if (table != null) { - String currentTable = row.getString(4).toString(); - if (!table.equals(currentTable)) { + if (!table.equals(root.get(ReadableMetadata.TABLE.key).asText())) { return; } } + final GenericRowData row = (GenericRowData) jsonDeserializer.convertToRowData(root); String type = row.getString(2).toString(); // "type" field if (OP_INSERT.equals(type)) { // "data" field is an array of row, contains inserted rows diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 7a565124f..84a3c707f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -432,13 +432,8 @@ public void testDeserializationMissingField() throws Exception { // fail on missing field deserializationSchema = - deserializationSchema = - new JsonRowDataDeserializationSchema( - schema, - InternalTypeInfo.of(schema), - true, - false, - TimestampFormat.ISO_8601); + new JsonRowDataDeserializationSchema( + schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); String errorMessage = "Failed to deserialize JSON '{\"id\":123123123}'."; try { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt index 7db1b224d..3d45031c9 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt @@ -11,4 +11,6 @@ {"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"mydb","es":1598944337000,"id":9,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944337341,"type":"DELETE"} {"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"5.17"},{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"}],"database":"mydb","es":1598944337000,"id":10,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"weight":"3.14"},{"weight":"8.1"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944337663,"type":"UPDATE"} {"data":[{"order_number":"10002","order_date":"2016-01-17","purchaser":"1002","quantity":"2","product_id":"105"}],"database":"mydb","es":1598944374000,"id":11,"isDdl":false,"mysqlType":{"order_number":"INTEGER","order_date":"DATE","purchaser":"INTEGER","quantity":"INTEGER","product_id":"INTEGER"},"old":null,"pkNames":["order_number"],"sql":"","sqlType":{"order_number":4,"order_date":91,"purchaser":4,"quantity":4,"product_id":4},"table":"orders","ts":1598944374999,"type":"DELETE"} -{"data":[{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"}],"database":"mydb","es":1598944418000,"id":12,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944418418,"type":"DELETE"} \ No newline at end of file +{"data":[{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"}],"database":"mydb","es":1598944418000,"id":12,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944418418,"type":"DELETE"} +{"data":null,"database":"mydb","es":1598944271000,"id":13,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE TABLE project (\n id VARCHAR(255) NOT NULL,\n name VARCHAR(255) NOT NULL,\n description VARCHAR(255) NOT NULL,\n weight FLOAT NOT NULL\n)","sqlType":null,"table":"projects","ts":1598944271192,"type":"CREATE"} +{"data":[{"id":"A101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"A102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"A103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"A104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"A105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"A106","name":"hammer","description":"16oz carpenter's hammer","weight":"1.0"},{"id":"A107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"A108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"A109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"mydb","es":1598944132000,"id":14,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"project","ts":1598944146308,"type":"INSERT"} \ No newline at end of file From f4edddf8b1a0c1ba45ed92264dd2c18e6f5c5bc6 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Fri, 15 Jan 2021 22:38:34 +0800 Subject: [PATCH 129/322] [FLINK-20953][canal-json] Support regular expression to filter databases and tables for canal-json This closes #14652 --- .../canal/CanalJsonDeserializationSchema.java | 17 +++++++++++++++-- .../formats/json/canal/CanalJsonOptions.java | 6 ++++-- .../json/canal/CanalJsonSerDeSchemaTest.java | 4 ++-- 3 files changed, 21 insertions(+), 6 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index 667741b14..8cbf2794e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -42,6 +42,7 @@ import java.io.Serializable; import java.util.List; import java.util.Objects; +import java.util.regex.Pattern; import java.util.stream.Collectors; import static java.lang.String.format; @@ -89,6 +90,12 @@ public final class CanalJsonDeserializationSchema implements DeserializationSche /** Number of fields. */ private final int fieldCount; + /** Pattern of the specific database. */ + private final Pattern databasePattern; + + /** Pattern of the specific table. */ + private final Pattern tablePattern; + private CanalJsonDeserializationSchema( DataType physicalDataType, List requestedMetadata, @@ -115,6 +122,8 @@ private CanalJsonDeserializationSchema( this.table = table; this.ignoreParseErrors = ignoreParseErrors; this.fieldCount = ((RowType) physicalDataType.getLogicalType()).getFieldCount(); + this.databasePattern = database == null ? null : Pattern.compile(database); + this.tablePattern = table == null ? null : Pattern.compile(table); } // ------------------------------------------------------------------------------------------ @@ -197,12 +206,16 @@ public void deserialize(@Nullable byte[] message, Collector out) throws try { final JsonNode root = jsonDeserializer.deserializeToJsonNode(message); if (database != null) { - if (!database.equals(root.get(ReadableMetadata.DATABASE.key).asText())) { + if (!databasePattern + .matcher(root.get(ReadableMetadata.DATABASE.key).asText()) + .matches()) { return; } } if (table != null) { - if (!table.equals(root.get(ReadableMetadata.TABLE.key).asText())) { + if (!tablePattern + .matcher(root.get(ReadableMetadata.TABLE.key).asText()) + .matches()) { return; } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java index f39c0d0c0..24ad21fb2 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java @@ -40,14 +40,16 @@ public class CanalJsonOptions { .stringType() .noDefaultValue() .withDescription( - "Only read changelog rows which match the specific database (by comparing the \"database\" meta field in the record)."); + "An optional regular expression to only read the specific databases changelog rows by regular matching the \"database\" meta field in the Canal record." + + "The pattern string is compatible with Java's Pattern."); public static final ConfigOption TABLE_INCLUDE = ConfigOptions.key("table.include") .stringType() .noDefaultValue() .withDescription( - "Only read changelog rows which match the specific table (by comparing the \"table\" meta field in the record)."); + "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the Canal record." + + "The pattern string is compatible with Java's Pattern."); // -------------------------------------------------------------------------------------------- // Validation diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index 51e82ece5..59a5071da 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -75,8 +75,8 @@ public void testFilteringTables() throws Exception { PHYSICAL_DATA_TYPE, Collections.emptyList(), InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType())) - .setDatabase("mydb") - .setTable("product") + .setDatabase("^my.*") + .setTable("^prod.*") .build(); runTest(lines, deserializationSchema); } From f717f2739d9efcb59f19795cc391e204b04f7ebf Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Fri, 22 Jan 2021 13:53:20 +0800 Subject: [PATCH 130/322] [FLINK-19446][canal-json] Fix canal-json format parse UPDATE record with null value will get wrong result This closes #14693 --- .../canal/CanalJsonDeserializationSchema.java | 15 +++++++++++---- .../json/canal/CanalJsonSerDeSchemaTest.java | 8 ++++---- .../test/resources/canal-data-filter-table.txt | 4 ++-- .../src/test/resources/canal-data.txt | 4 ++-- 4 files changed, 19 insertions(+), 12 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index 8cbf2794e..acdaecf58 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -61,6 +61,7 @@ public final class CanalJsonDeserializationSchema implements DeserializationSchema { private static final long serialVersionUID = 1L; + private static final String FIELD_OLD = "old"; private static final String OP_INSERT = "INSERT"; private static final String OP_UPDATE = "UPDATE"; private static final String OP_DELETE = "DELETE"; @@ -87,6 +88,9 @@ public final class CanalJsonDeserializationSchema implements DeserializationSche /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ private final boolean ignoreParseErrors; + /** Names of fields. */ + private final List fieldNames; + /** Number of fields. */ private final int fieldCount; @@ -121,7 +125,9 @@ private CanalJsonDeserializationSchema( this.database = database; this.table = table; this.ignoreParseErrors = ignoreParseErrors; - this.fieldCount = ((RowType) physicalDataType.getLogicalType()).getFieldCount(); + final RowType physicalRowType = ((RowType) physicalDataType.getLogicalType()); + this.fieldNames = physicalRowType.getFieldNames(); + this.fieldCount = physicalRowType.getFieldCount(); this.databasePattern = database == null ? null : Pattern.compile(database); this.tablePattern = table == null ? null : Pattern.compile(table); } @@ -238,10 +244,11 @@ public void deserialize(@Nullable byte[] message, Collector out) throws // the underlying JSON deserialization schema always produce GenericRowData. GenericRowData after = (GenericRowData) data.getRow(i, fieldCount); GenericRowData before = (GenericRowData) old.getRow(i, fieldCount); + final JsonNode oldField = root.get(FIELD_OLD); for (int f = 0; f < fieldCount; f++) { - if (before.isNullAt(f)) { - // not null fields in "old" (before) means the fields are changed - // null/empty fields in "old" (before) means the fields are not changed + if (before.isNullAt(f) && oldField.findValue(fieldNames.get(f)) == null) { + // fields in "old" (before) means the fields are changed + // fields not in "old" (before) means the fields are not changed // so we just copy the not changed fields into before before.setField(f, after.getField(f)); } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index 59a5071da..37cf8b7ee 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -186,11 +186,11 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", "+I(104,hammer,12oz carpenter's hammer,0.75)", "+I(105,hammer,14oz carpenter's hammer,0.875)", - "+I(106,hammer,16oz carpenter's hammer,1.0)", + "+I(106,hammer,null,1.0)", "+I(107,rocks,box of assorted rocks,5.3)", "+I(108,jacket,water resistent black wind breaker,0.1)", "+I(109,spare tire,24 inch spare tire,22.2)", - "-U(106,hammer,16oz carpenter's hammer,1.0)", + "-U(106,hammer,null,1.0)", "+U(106,hammer,18oz carpenter hammer,1.0)", "-U(107,rocks,box of assorted rocks,5.3)", "+U(107,rocks,box of assorted rocks,5.1)", @@ -233,11 +233,11 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"INSERT\"}", "{\"data\":[{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75}],\"type\":\"INSERT\"}", "{\"data\":[{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0}],\"type\":\"INSERT\"}", + "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"INSERT\"}", "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"INSERT\"}", "{\"data\":[{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1}],\"type\":\"INSERT\"}", "{\"data\":[{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0}],\"type\":\"DELETE\"}", + "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"DELETE\"}", "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0}],\"type\":\"INSERT\"}", "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"DELETE\"}", "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1}],\"type\":\"INSERT\"}", diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt index 3d45031c9..68661d762 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt @@ -1,5 +1,5 @@ -{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"106","name":"hammer","description":"16oz carpenter's hammer","weight":"1.0"},{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"mydb","es":1598944132000,"id":1,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944146308,"type":"INSERT"} -{"data":[{"id":"106","name":"hammer","description":"18oz carpenter hammer","weight":"1.0"}],"database":"mydb","es":1598944202000,"id":2,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"description":"16oz carpenter's hammer"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944202218,"type":"UPDATE"} +{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"106","name":"hammer","description":null,"weight":"1.0"},{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"mydb","es":1598944132000,"id":1,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944146308,"type":"INSERT"} +{"data":[{"id":"106","name":"hammer","description":"18oz carpenter hammer","weight":"1.0"}],"database":"mydb","es":1598944202000,"id":2,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"description":null}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944202218,"type":"UPDATE"} {"data":null,"database":"mydb","es":1598944271000,"id":3,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE TABLE orders (\n order_number INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,\n order_date DATE NOT NULL,\n purchaser INTEGER NOT NULL,\n quantity INTEGER NOT NULL,\n product_id INTEGER NOT NULL\n) AUTO_INCREMENT = 10001","sqlType":null,"table":"orders","ts":1598944271192,"type":"CREATE"} {"data":[{"order_number":"10001","order_date":"2016-01-16","purchaser":"1001","quantity":"1","product_id":"102"},{"order_number":"10002","order_date":"2016-01-17","purchaser":"1002","quantity":"2","product_id":"105"},{"order_number":"10003","order_date":"2016-02-19","purchaser":"1002","quantity":"2","product_id":"106"},{"order_number":"10004","order_date":"2016-02-21","purchaser":"1003","quantity":"1","product_id":"107"}],"database":"mydb","es":1598944275000,"id":4,"isDdl":false,"mysqlType":{"order_number":"INTEGER","order_date":"DATE","purchaser":"INTEGER","quantity":"INTEGER","product_id":"INTEGER"},"old":null,"pkNames":["order_number"],"sql":"","sqlType":{"order_number":4,"order_date":91,"purchaser":4,"quantity":4,"product_id":4},"table":"orders","ts":1598944275018,"type":"INSERT"} {"data":[{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.1"}],"database":"mydb","es":1598944279000,"id":5,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944279665,"type":"UPDATE"} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt index 530468aa4..a83b7da4f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt @@ -1,5 +1,5 @@ -{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"106","name":"hammer","description":"16oz carpenter's hammer","weight":"1.0"},{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"inventory","es":1589373515000,"id":3,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373515477,"type":"INSERT"} -{"data":[{"id":"106","name":"hammer","description":"18oz carpenter hammer","weight":"1.0"}],"database":"inventory","es":1589373546000,"id":4,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"description":"16oz carpenter's hammer"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373546301,"type":"UPDATE"} +{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"106","name":"hammer","description":null,"weight":"1.0"},{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"inventory","es":1589373515000,"id":3,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373515477,"type":"INSERT"} +{"data":[{"id":"106","name":"hammer","description":"18oz carpenter hammer","weight":"1.0"}],"database":"inventory","es":1589373546000,"id":4,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"description":null}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373546301,"type":"UPDATE"} {"data":[{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.1"}],"database":"inventory","es":1589373549000,"id":5,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373549489,"type":"UPDATE"} {"data":[{"id":"110","name":"jacket","description":"water resistent white wind breaker","weight":"0.2"}],"database":"inventory","es":1589373552000,"id":6,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373552882,"type":"INSERT"} {"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.18"}],"database":"inventory","es":1589373555000,"id":7,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373555457,"type":"INSERT"} From 17c5aaa54c5b696d37934e61bde63c424341dcbb Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Thu, 21 Jan 2021 19:50:46 +0800 Subject: [PATCH 131/322] [FLINK-21020][build] Bump Jackson to 2.12.1 --- .../src/main/resources/META-INF/NOTICE | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 15ccc70d7..92f02871e 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -7,9 +7,9 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - org.apache.avro:avro:1.10.0 -- com.fasterxml.jackson.core:jackson-core:2.10.1 -- com.fasterxml.jackson.core:jackson-databind:2.10.1 -- com.fasterxml.jackson.core:jackson-annotations:2.10.1 +- com.fasterxml.jackson.core:jackson-core:2.12.1 +- com.fasterxml.jackson.core:jackson-databind:2.12.1 +- com.fasterxml.jackson.core:jackson-annotations:2.12.1 - org.apache.commons:commons-compress:1.20 - io.confluent:kafka-schema-registry-client:5.5.2 - org.apache.kafka:kafka-clients:5.5.2-ccs From 9cd50f6602c433c5ef668a95426daca390fed298 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Tue, 2 Feb 2021 11:51:18 +0800 Subject: [PATCH 132/322] [FLINK-21172][canal][json] Support 'event-timestampcanal-json format include es field (#14792) --- .../json/canal/CanalJsonDecodingFormat.java | 16 ++++++++++++++++ .../json/canal/CanalJsonSerDeSchemaTest.java | 2 ++ 2 files changed, 18 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java index 81e5287ac..791dd59d9 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java @@ -194,6 +194,22 @@ public Object convert(GenericRowData row, int pos) { new MetadataConverter() { private static final long serialVersionUID = 1L; + @Override + public Object convert(GenericRowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + return TimestampData.fromEpochMillis(row.getLong(pos)); + } + }), + + EVENT_TIMESTAMP( + "event-timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + DataTypes.FIELD("es", DataTypes.BIGINT()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + @Override public Object convert(GenericRowData row, int pos) { if (row.isNullAt(pos)) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index 37cf8b7ee..3a193841a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -109,6 +109,7 @@ public void testDeserializationWithMetadata() throws Exception { assertThat(row.getMap(6).size(), equalTo(4)); assertThat(row.getArray(7).getString(0).toString(), equalTo("id")); assertThat(row.getTimestamp(8, 3).getMillisecond(), equalTo(1589373515477L)); + assertThat(row.getTimestamp(9, 3).getMillisecond(), equalTo(1589373515000L)); }); testDeserializationWithMetadata( "canal-data-filter-table.txt", @@ -124,6 +125,7 @@ public void testDeserializationWithMetadata() throws Exception { assertThat(row.getMap(6).size(), equalTo(4)); assertThat(row.getArray(7).getString(0).toString(), equalTo("id")); assertThat(row.getTimestamp(8, 3).getMillisecond(), equalTo(1598944146308L)); + assertThat(row.getTimestamp(9, 3).getMillisecond(), equalTo(1598944132000L)); }); } From 9f8a7fd64f736335332a4ad11ab1e20605304dce Mon Sep 17 00:00:00 2001 From: Seth Wiesman Date: Sun, 7 Feb 2021 12:48:20 -0600 Subject: [PATCH 133/322] [FLINK-21193][docs] Migrate Flink docs from Jekyll to Hugo --- .../table/formats/avro-confluent.md | 223 ++++++++++ .../docs/connectors/table/formats/debezium.md | 385 ++++++++++++++++++ .../table/formats/avro-confluent.md | 215 ++++++++++ .../docs/connectors/table/formats/debezium.md | 383 +++++++++++++++++ 4 files changed, 1206 insertions(+) create mode 100644 docs/content.zh/docs/connectors/table/formats/avro-confluent.md create mode 100644 docs/content.zh/docs/connectors/table/formats/debezium.md create mode 100644 docs/content/docs/connectors/table/formats/avro-confluent.md create mode 100644 docs/content/docs/connectors/table/formats/debezium.md diff --git a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md new file mode 100644 index 000000000..996269946 --- /dev/null +++ b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md @@ -0,0 +1,223 @@ +--- +title: Confluent Avro +weight: 4 +type: docs +aliases: + - /zh/dev/table/connectors/formats/avro-confluent.html +--- + + +# Confluent Avro Format + +Format: Serialization Schema +Format: Deserialization Schema + + + +Avro Schema Registry (``avro-confluent``) 格式能让你读取被 ``io.confluent.kafka.serializers.KafkaAvroSerializer``序列化的记录,以及可以写入成能被 ``io.confluent.kafka.serializers.KafkaAvroDeserializer``反序列化的记录。 + +当以这种格式读取(反序列化)记录时,将根据记录中编码的 schema 版本 id 从配置的 Confluent Schema Registry 中获取 Avro writer schema ,而从 table schema 中推断出 reader schema。 + +当以这种格式写入(序列化)记录时,Avro schema 是从 table schema 中推断出来的,并会用来检索要与数据一起编码的 schema id。我们会在配置的 Confluent Schema Registry 中配置的 [subject](https://docs.confluent.io/current/schema-registry/index.html#schemas-subjects-and-topics) 下,检索 schema id。subject 通过 `avro-confluent.schema-registry.subject` 参数来制定。 + +The Avro Schema Registry format can only be used in conjunction with the [Apache Kafka SQL connector]({{< ref "docs/connectors/table/kafka" >}}) or the [Upsert Kafka SQL Connector]({{< ref "docs/connectors/table/upsert-kafka" >}}). + +依赖 +------------ + +{{< sql_download_table "avro-confluent" >}} + +如何创建使用 Avro-Confluent 格式的表 +---------------- + +以下是一个使用 Kafka 连接器和 Confluent Avro 格式创建表的示例。 + +{{< tabs "3df131fd-0e20-4635-a8f9-3574a764db7a" >}} +{{< tab "SQL" >}} + +Example of a table using raw UTF-8 string as Kafka key and Avro records registered in the Schema Registry as Kafka values: + +```sql +CREATE TABLE user_created ( + + -- one column mapped to the Kafka raw UTF-8 key + the_kafka_key STRING, + + -- a few columns mapped to the Avro fields of the Kafka value + id STRING, + name STRING, + email STRING + +) WITH ( + + 'connector' = 'kafka', + 'topic' = 'user_events_example1', + 'properties.bootstrap.servers' = 'localhost:9092', + + -- UTF-8 string as Kafka keys, using the 'the_kafka_key' table column + 'key.format' = 'raw', + 'key.fields' = 'the_kafka_key', + + 'value.format' = 'avro-confluent', + 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.fields-include' = 'EXCEPT_KEY' +) +``` + +We can write data into the kafka table as follows: + +```sql +INSERT INTO user_created +SELECT + -- replicating the user id into a column mapped to the kafka key + id as the_kafka_key, + + -- all values + id, name, email +FROM some_table +``` + +--- + +Example of a table with both the Kafka key and value registered as Avro records in the Schema Registry: + +```sql +CREATE TABLE user_created ( + + -- one column mapped to the 'id' Avro field of the Kafka key + kafka_key_id STRING, + + -- a few columns mapped to the Avro fields of the Kafka value + id STRING, + name STRING, + email STRING + +) WITH ( + + 'connector' = 'kafka', + 'topic' = 'user_events_example2', + 'properties.bootstrap.servers' = 'localhost:9092', + + -- Watch out: schema evolution in the context of a Kafka key is almost never backward nor + -- forward compatible due to hash partitioning. + 'key.format' = 'avro-confluent', + 'key.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'key.fields' = 'kafka_key_id', + + -- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id' + -- => adding a prefix to the table column associated to the Kafka key field avoids clashes + 'key.fields-prefix' = 'kafka_key_', + + 'value.format' = 'avro-confluent', + 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.fields-include' = 'EXCEPT_KEY', + + -- subjects have a default value since Flink 1.13, though can be overriden: + 'key.avro-confluent.schema-registry.subject' = 'user_events_example2-key2', + 'value.avro-confluent.schema-registry.subject' = 'user_events_example2-value2' +) +``` + +--- +Example of a table using the upsert connector with the Kafka value registered as an Avro record in the Schema Registry: + +```sql +CREATE TABLE user_created ( + + -- one column mapped to the Kafka raw UTF-8 key + kafka_key_id STRING, + + -- a few columns mapped to the Avro fields of the Kafka value + id STRING, + name STRING, + email STRING, + + -- upsert-kafka connector requires a primary key to define the upsert behavior + PRIMARY KEY (kafka_key_id) NOT ENFORCED + +) WITH ( + + 'connector' = 'upsert-kafka', + 'topic' = 'user_events_example3', + 'properties.bootstrap.servers' = 'localhost:9092', + + -- UTF-8 string as Kafka keys + -- We don't specify 'key.fields' in this case since it's dictated by the primary key of the table + 'key.format' = 'raw', + + -- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id' + -- => adding a prefix to the table column associated to the kafka key field to avoid clashes + 'key.fields-prefix' = 'kafka_key_', + + 'value.format' = 'avro-confluent', + 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.fields-include' = 'EXCEPT_KEY' +) +``` +{{< /tab >}} +{{< /tabs >}} + +Format 参数 +---------------- + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
参数是否必选默认值类型描述
format
必选(none)String指定要使用的格式,这里应该是 'avro-confluent'
avro-confluent.schema-registry.url
必选(none)String用于获取/注册 schemas 的 Confluent Schema Registry 的URL。
avro-confluent.schema-registry.subject
可选(none)StringConfluent Schema Registry 主题,用于在序列化期间注册此格式使用的 schema。默认 kafka 和 upsert-kafka 连接器会使用 "<topic_name>-value" 或者 "<topic_name>-key" 作为 subject 名字。但是对于其他连接器(如 filesystem)则在当做 sink 使用时需要显式指定 subject 名字。
+ +数据类型映射 +---------------- + +目前 Apache Flink 都是从 table schema 去推断反序列化期间的 Avro reader schema 和序列化期间的 Avro writer schema。显式地定义 Avro schema 暂不支持。 +[Apache Avro Format]({{< ref "docs/connectors/table/formats/avro" >}}#data-type-mapping)中描述了 Flink 数据类型和 Avro 类型的对应关系。 + +除了此处列出的类型之外,Flink 还支持读取/写入可为空(nullable)的类型。 Flink 将可为空的类型映射到 Avro `union(something, null)`, 其中 `something` 是从 Flink 类型转换的 Avro 类型。 + +您可以参考 [Avro Specification](https://avro.apache.org/docs/current/spec.html) 以获取有关 Avro 类型的更多信息。 diff --git a/docs/content.zh/docs/connectors/table/formats/debezium.md b/docs/content.zh/docs/connectors/table/formats/debezium.md new file mode 100644 index 000000000..e860d3356 --- /dev/null +++ b/docs/content.zh/docs/connectors/table/formats/debezium.md @@ -0,0 +1,385 @@ +--- +title: Debezium +weight: 5 +type: docs +aliases: + - /zh/dev/table/connectors/formats/debezium.html +--- + + +# Debezium Format + +{{< label "Changelog-Data-Capture Format" >}} +{{< label "Format: Serialization Schema" >}} +{{< label "Format: Deserialization Schema" >}} + +[Debezium](https://debezium.io/) 是一个 CDC(Changelog Data Capture,变更数据捕获)的工具,可以把来自 MySQL、PostgreSQL、Oracle、Microsoft SQL Server 和许多其他数据库的更改实时流式传输到 Kafka 中。 Debezium 为变更日志提供了统一的格式结构,并支持使用 JSON 和 Apache Avro 序列化消息。 + +Flink 支持将 Debezium JSON 和 Avro 消息解析为 INSERT / UPDATE / DELETE 消息到 Flink SQL 系统中。在很多情况下,利用这个特性非常的有用,例如 + - 将增量数据从数据库同步到其他系统 + - 日志审计 + - 数据库的实时物化视图 + - 关联维度数据库的变更历史,等等。 + +Flink 还支持将 Flink SQL 中的 INSERT / UPDATE / DELETE 消息编码为 Debezium 格式的 JSON 或 Avro 消息,输出到 Kafka 等存储中。 +但需要注意的是,目前 Flink 还不支持将 UPDATE_BEFORE 和 UPDATE_AFTER 合并为一条 UPDATE 消息。因此,Flink 将 UPDATE_BEFORE 和 UPDATE_AFTER 分别编码为 DELETE 和 INSERT 类型的 Debezium 消息。 + +依赖 +------------ + +#### Debezium Avro + +{{< sql_download_table "debezium-avro-confluent" >}} + +#### Debezium Json + +{{< sql_download_table "debezium-json" >}} + +*注意: 请参考 [Debezium 文档](https://debezium.io/documentation/reference/1.3/index.html),了解如何设置 Debezium Kafka Connect 用来将变更日志同步到 Kafka 主题。* + + +如何使用 Debezium Format +---------------- + + +Debezium 为变更日志提供了统一的格式,这是一个 JSON 格式的从 MySQL product 表捕获的更新操作的简单示例: + +```json +{ + "before": { + "id": 111, + "name": "scooter", + "description": "Big 2-wheel scooter", + "weight": 5.18 + }, + "after": { + "id": 111, + "name": "scooter", + "description": "Big 2-wheel scooter", + "weight": 5.15 + }, + "source": {...}, + "op": "u", + "ts_ms": 1589362330904, + "transaction": null +} +``` + +*注意: 请参考 [Debezium 文档](https://debezium.io/documentation/reference/1.3/connectors/mysql.html#mysql-connector-events_debezium),了解每个字段的含义。* + +MySQL 产品表有4列(`id`、`name`、`description`、`weight`)。上面的 JSON 消息是 `products` 表上的一条更新事件,其中 `id = 111` 的行的 `weight` 值从 `5.18` 更改为 `5.15`。假设此消息已同步到 Kafka 主题 `products_binlog`,则可以使用以下 DDL 来使用此主题并解析更改事件。 + +{{< tabs "0b6703c1-021e-4506-a579-b72b8408c0cf" >}} +{{< tab "SQL" >}} +```sql +CREATE TABLE topic_products ( + -- schema 与 MySQL 的 products 表完全相同 + id BIGINT, + name STRING, + description STRING, + weight DECIMAL(10, 2) +) WITH ( + 'connector' = 'kafka', + 'topic' = 'products_binlog', + 'properties.bootstrap.servers' = 'localhost:9092', + 'properties.group.id' = 'testGroup', + -- 使用 'debezium-json' format 来解析 Debezium 的 JSON 消息 + -- 如果 Debezium 用 Avro 编码消息,请使用 'debezium-avro-confluent' + 'format' = 'debezium-json' -- 如果 Debezium 用 Avro 编码消息,请使用 'debezium-avro-confluent' +) +``` +{{< /tab >}} +{{< /tabs >}} + +在某些情况下,用户在设置 Debezium Kafka Connect 时,可能会开启 Kafka 的配置 `'value.converter.schemas.enable'`,用来在消息体中包含 schema 信息。然后,Debezium JSON 消息可能如下所示: + +```json +{ + "schema": {...}, + "payload": { + "before": { + "id": 111, + "name": "scooter", + "description": "Big 2-wheel scooter", + "weight": 5.18 + }, + "after": { + "id": 111, + "name": "scooter", + "description": "Big 2-wheel scooter", + "weight": 5.15 + }, + "source": {...}, + "op": "u", + "ts_ms": 1589362330904, + "transaction": null + } +} +``` + +为了解析这一类信息,你需要在上述 DDL WITH 子句中添加选项 `'debezium-json.schema-include' = 'true'`(默认为 false)。通常情况下,建议不要包含 schema 的描述,因为这样会使消息变得非常冗长,并降低解析性能。 + +在将主题注册为 Flink 表之后,可以将 Debezium 消息用作变更日志源。 + +{{< tabs "6a84a0e8-2e56-49db-9089-e836290f8239" >}} +{{< tab "SQL" >}} +```sql +-- MySQL "products" 的实时物化视图 +-- 计算相同产品的最新平均重量 +SELECT name, AVG(weight) FROM topic_products GROUP BY name; + +-- 将 MySQL "products" 表的所有数据和增量更改同步到 +-- Elasticsearch "products" 索引,供将来查找 +INSERT INTO elasticsearch_products +SELECT * FROM topic_products; +``` +{{< /tab >}} +{{< /tabs >}} + +Available Metadata +------------------ + +The following format metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition. + +Attention Format metadata fields are only available if the +corresponding connector forwards format metadata. Currently, only the Kafka connector is able to expose +metadata fields for its value format. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyData TypeDescription
schemaSTRING NULLJSON string describing the schema of the payload. Null if the schema is not included in + the Debezium record.
ingestion-timestampTIMESTAMP(3) WITH LOCAL TIME ZONE NULLThe timestamp at which the connector processed the event. Corresponds to the ts_ms + field in the Debezium record.
source.timestampTIMESTAMP(3) WITH LOCAL TIME ZONE NULLThe timestamp at which the source system created the event. Corresponds to the source.ts_ms + field in the Debezium record.
source.databaseSTRING NULLThe originating database. Corresponds to the source.db field in the + Debezium record if available.
source.schemaSTRING NULLThe originating database schema. Corresponds to the source.schema field in the + Debezium record if available.
source.tableSTRING NULLThe originating database table. Corresponds to the source.table or source.collection + field in the Debezium record if available.
source.propertiesMAP<STRING, STRING> NULLMap of various source properties. Corresponds to the source field in the Debezium record.
+ +The following example shows how to access Debezium metadata fields in Kafka: + +```sql +CREATE TABLE KafkaTable ( + origin_ts TIMESTAMP(3) METADATA FROM 'value.ingestion-timestamp' VIRTUAL, + event_time TIMESTAMP(3) METADATA FROM 'value.source.timestamp' VIRTUAL, + origin_database STRING METADATA FROM 'value.source.database' VIRTUAL, + origin_schema STRING METADATA FROM 'value.source.schema' VIRTUAL, + origin_table STRING METADATA FROM 'value.source.table' VIRTUAL, + origin_properties MAP METADATA FROM 'value.source.properties' VIRTUAL, + user_id BIGINT, + item_id BIGINT, + behavior STRING +) WITH ( + 'connector' = 'kafka', + 'topic' = 'user_behavior', + 'properties.bootstrap.servers' = 'localhost:9092', + 'properties.group.id' = 'testGroup', + 'scan.startup.mode' = 'earliest-offset', + 'value.format' = 'debezium-json' +); +``` + +Format 参数 +---------------- + +Flink 提供了 `debezium-avro-confluent` 和 `debezium-json` 两种 format 来解析 Debezium 生成的 JSON 格式和 Avro 格式的消息。 +请使用 `debezium-avro-confluent` 来解析 Debezium 的 Avro 消息,使用 `debezium-json` 来解析 Debezium 的 JSON 消息。 + +#### Debezium Avro + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
参数是否必选默认值类型描述
format
必选(none)String指定要使用的格式,此处应为 'debezium-avro-confluent'
debezium-avro-confluent.schema-registry.url
必选(none)String用于获取/注册 schemas 的 Confluent Schema Registry 的 URL。
debezium-avro-confluent.schema-registry.subject
可选(none)StringConfluent Schema Registry主题,用于在序列化期间注册此格式使用的 schema。默认 kafka 连接器会使用 "<topic_name>-value" 作为默认的 subject 名字,但是对于其他连接器(如 filesystem)则在当做 sink 使用时需要显式指定 subject 名字。
+ +#### Debezium Json + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
参数是否必选默认值类型描述
format
必选(none)String指定要使用的格式,此处应为 'debezium-json'
debezium-json.schema-include
可选falseBoolean设置 Debezium Kafka Connect 时,用户可以启用 Kafka 配置 'value.converter.schemas.enable' 以在消息中包含 schema。此选项表明 Debezium JSON 消息是否包含 schema。
debezium-json.ignore-parse-errors
可选falseBoolean当解析异常时,是跳过当前字段或行,还是抛出错误失败(默认为 false,即抛出错误失败)。如果忽略字段的解析异常,则会将该字段值设置为null
debezium-json.timestamp-format.standard
可选'SQL'String声明输入和输出的时间戳格式。当前支持的格式为'SQL' 以及 'ISO-8601': +
    +
  • 可选参数 'SQL' 将会以 "yyyy-MM-dd HH:mm:ss.s{precision}" 的格式解析时间戳, 例如 '2020-12-30 12:13:14.123',且会以相同的格式输出。
  • +
  • 可选参数 'ISO-8601' 将会以 "yyyy-MM-ddTHH:mm:ss.s{precision}" 的格式解析输入时间戳, 例如 '2020-12-30T12:13:14.123' ,且会以相同的格式输出。
  • +
+
debezium-json.map-null-key.mode
选填'FAIL'String指定处理 Map 中 key 值为空的方法. 当前支持的值有 'FAIL', 'DROP''LITERAL': +
    +
  • Option 'FAIL' 将抛出异常,如果遇到 Map 中 key 值为空的数据。
  • +
  • Option 'DROP' 将丢弃 Map 中 key 值为空的数据项。
  • +
  • Option 'LITERAL' 将使用字符串常量来替换 Map 中的空 key 值。字符串常量的值由 'debezium-json.map-null-key.literal' 定义。
  • +
+
debezium-json.map-null-key.literal
选填'null'String'debezium-json.map-null-key.mode' 是 LITERAL 的时候,指定字符串常量替换 Map 中的空 key 值。
debezium-json.encode.decimal-as-plain-number
选填falseBoolean将所有 DECIMAL 类型的数据保持原状,不使用科学计数法表示。例:0.000000027 默认会表示为 2.7E-8。当此选项设为 true 时,则会表示为 0.000000027
+ + +注意事项 +---------------- + +### 重复的变更事件 + +在正常的操作环境下,Debezium 应用能以 **exactly-once** 的语义投递每条变更事件。在这种情况下,Flink 消费 Debezium 产生的变更事件能够工作得很好。 +然而,当有故障发生时,Debezium 应用只能保证 **at-least-once** 的投递语义。可以查看 [Debezium 官方文档](https://debezium.io/documentation/faq/#what_happens_when_an_application_stops_or_crashes) 了解更多关于 Debezium 的消息投递语义。 +这也意味着,在非正常情况下,Debezium 可能会投递重复的变更事件到 Kafka 中,当 Flink 从 Kafka 中消费的时候就会得到重复的事件。 +这可能会导致 Flink query 的运行得到错误的结果或者非预期的异常。因此,建议在这种情况下,将作业参数 [`table.exec.source.cdc-events-duplicate`]({{< ref "docs/dev/table/config" >}}#table-exec-source-cdc-events-duplicate) 设置成 `true`,并在该 source 上定义 PRIMARY KEY。 +框架会生成一个额外的有状态算子,使用该 primary key 来对变更事件去重并生成一个规范化的 changelog 流。 + +### 消费 Debezium Postgres Connector 产生的数据 + +如果你正在使用 [Debezium PostgreSQL Connector](https://debezium.io/documentation/reference/1.2/connectors/postgresql.html) 捕获变更到 Kafka,请确保被监控表的 [REPLICA IDENTITY](https://www.postgresql.org/docs/current/sql-altertable.html#SQL-CREATETABLE-REPLICA-IDENTITY) 已经被配置成 `FULL` 了,默认值是 `DEFAULT`。 +否则,Flink SQL 将无法正确解析 Debezium 数据。 + +当配置为 `FULL` 时,更新和删除事件将完整包含所有列的之前的值。当为其他配置时,更新和删除事件的 "before" 字段将只包含 primary key 字段的值,或者为 null(没有 primary key)。 +你可以通过运行 `ALTER TABLE REPLICA IDENTITY FULL` 来更改 `REPLICA IDENTITY` 的配置。 +请阅读 [Debezium 关于 PostgreSQL REPLICA IDENTITY 的文档](https://debezium.io/documentation/reference/1.2/connectors/postgresql.html#postgresql-replica-identity) 了解更多。 + +数据类型映射 +---------------- + +目前,Debezium Format 使用 JSON Format 进行序列化和反序列化。有关数据类型映射的更多详细信息,请参考 [JSON Format 文档]({{< ref "docs/connectors/table/formats/json" >}}#data-type-mapping) 和 [Confluent Avro Format 文档]({{< ref "docs/connectors/table/formats/avro-confluent" >}}#data-type-mapping)。 + diff --git a/docs/content/docs/connectors/table/formats/avro-confluent.md b/docs/content/docs/connectors/table/formats/avro-confluent.md new file mode 100644 index 000000000..dc36a8750 --- /dev/null +++ b/docs/content/docs/connectors/table/formats/avro-confluent.md @@ -0,0 +1,215 @@ +--- +title: Confluent Avro +weight: 4 +type: docs +aliases: + - /dev/table/connectors/formats/avro-confluent.html +--- + + +# Confluent Avro Format + +{{< label "Format: Serialization Schema" >}} +{{< label "Format: Deserialization Schema" >}} + +The Avro Schema Registry (``avro-confluent``) format allows you to read records that were serialized by the ``io.confluent.kafka.serializers.KafkaAvroSerializer`` and to write records that can in turn be read by the ``io.confluent.kafka.serializers.KafkaAvroDeserializer``. + +When reading (deserializing) a record with this format the Avro writer schema is fetched from the configured Confluent Schema Registry based on the schema version id encoded in the record while the reader schema is inferred from table schema. + +When writing (serializing) a record with this format the Avro schema is inferred from the table schema and used to retrieve a schema id to be encoded with the data. The lookup is performed with in the configured Confluent Schema Registry under the [subject](https://docs.confluent.io/current/schema-registry/index.html#schemas-subjects-and-topics) given in `avro-confluent.schema-registry.subject`. + +The Avro Schema Registry format can only be used in conjunction with the [Apache Kafka SQL connector]({{< ref "docs/connectors/table/kafka" >}}) or the [Upsert Kafka SQL Connector]({{< ref "docs/connectors/table/upsert-kafka" >}}). + +Dependencies +------------ + +{{< sql_download_table "avro-confluent" >}} + +How to create tables with Avro-Confluent format +-------------- + +Example of a table using raw UTF-8 string as Kafka key and Avro records registered in the Schema Registry as Kafka values: + +```sql +CREATE TABLE user_created ( + + -- one column mapped to the Kafka raw UTF-8 key + the_kafka_key STRING, + + -- a few columns mapped to the Avro fields of the Kafka value + id STRING, + name STRING, + email STRING + +) WITH ( + + 'connector' = 'kafka', + 'topic' = 'user_events_example1', + 'properties.bootstrap.servers' = 'localhost:9092', + + -- UTF-8 string as Kafka keys, using the 'the_kafka_key' table column + 'key.format' = 'raw', + 'key.fields' = 'the_kafka_key', + + 'value.format' = 'avro-confluent', + 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.fields-include' = 'EXCEPT_KEY' +) +``` + +We can write data into the kafka table as follows: + +```sql +INSERT INTO user_created +SELECT + -- replicating the user id into a column mapped to the kafka key + id as the_kafka_key, + + -- all values + id, name, email +FROM some_table +``` + +--- + +Example of a table with both the Kafka key and value registered as Avro records in the Schema Registry: + +```sql +CREATE TABLE user_created ( + + -- one column mapped to the 'id' Avro field of the Kafka key + kafka_key_id STRING, + + -- a few columns mapped to the Avro fields of the Kafka value + id STRING, + name STRING, + email STRING + +) WITH ( + + 'connector' = 'kafka', + 'topic' = 'user_events_example2', + 'properties.bootstrap.servers' = 'localhost:9092', + + -- Watch out: schema evolution in the context of a Kafka key is almost never backward nor + -- forward compatible due to hash partitioning. + 'key.format' = 'avro-confluent', + 'key.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'key.fields' = 'kafka_key_id', + + -- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id' + -- => adding a prefix to the table column associated to the Kafka key field avoids clashes + 'key.fields-prefix' = 'kafka_key_', + + 'value.format' = 'avro-confluent', + 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.fields-include' = 'EXCEPT_KEY', + + -- subjects have a default value since Flink 1.13, though can be overriden: + 'key.avro-confluent.schema-registry.subject' = 'user_events_example2-key2', + 'value.avro-confluent.schema-registry.subject' = 'user_events_example2-value2' +) +``` + +--- +Example of a table using the upsert connector with the Kafka value registered as an Avro record in the Schema Registry: + +```sql +CREATE TABLE user_created ( + + -- one column mapped to the Kafka raw UTF-8 key + kafka_key_id STRING, + + -- a few columns mapped to the Avro fields of the Kafka value + id STRING, + name STRING, + email STRING, + + -- upsert-kafka connector requires a primary key to define the upsert behavior + PRIMARY KEY (kafka_key_id) NOT ENFORCED + +) WITH ( + + 'connector' = 'upsert-kafka', + 'topic' = 'user_events_example3', + 'properties.bootstrap.servers' = 'localhost:9092', + + -- UTF-8 string as Kafka keys + -- We don't specify 'key.fields' in this case since it's dictated by the primary key of the table + 'key.format' = 'raw', + + -- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id' + -- => adding a prefix to the table column associated to the kafka key field to avoid clashes + 'key.fields-prefix' = 'kafka_key_', + + 'value.format' = 'avro-confluent', + 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.fields-include' = 'EXCEPT_KEY' +) +``` + + +Format Options +---------------- + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequiredDefaultTypeDescription
format
required(none)StringSpecify what format to use, here should be 'avro-confluent'.
avro-confluent.schema-registry.url
required(none)StringThe URL of the Confluent Schema Registry to fetch/register schemas.
avro-confluent.schema-registry.subject
optional(none)StringThe Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, kafka and upsert-kafka connectors use "<topic_name>-value" or "<topic_name>-key" as the default subject name if avro-confluent is used as the value or key format. But for other connectors (e.g. filesystem), the subject option is required when used as sink.
+ +Data Type Mapping +---------------- + +Currently, Apache Flink always uses the table schema to derive the Avro reader schema during deserialization and Avro writer schema during serialization. Explicitly defining an Avro schema is not supported yet. +See the [Apache Avro Format]({{< ref "docs/connectors/table/formats/avro" >}}#data-type-mapping) for the mapping between Avro and Flink DataTypes. + +In addition to the types listed there, Flink supports reading/writing nullable types. Flink maps nullable types to Avro `union(something, null)`, where `something` is the Avro type converted from Flink type. + +You can refer to [Avro Specification](https://avro.apache.org/docs/current/spec.html) for more information about Avro types. diff --git a/docs/content/docs/connectors/table/formats/debezium.md b/docs/content/docs/connectors/table/formats/debezium.md new file mode 100644 index 000000000..eee4534e9 --- /dev/null +++ b/docs/content/docs/connectors/table/formats/debezium.md @@ -0,0 +1,383 @@ +--- +title: Debezium +weight: 5 +type: docs +aliases: + - /dev/table/connectors/formats/debezium.html +--- + + +# Debezium Format + +{{< label "Changelog-Data-Capture Format" >}} +{{< label "Format: Serialization Schema" >}} +{{< label "Format: Deserialization Schema" >}} + +[Debezium](https://debezium.io/) is a CDC (Changelog Data Capture) tool that can stream changes in real-time from MySQL, PostgreSQL, Oracle, Microsoft SQL Server and many other databases into Kafka. Debezium provides a unified format schema for changelog and supports to serialize messages using JSON and [Apache Avro](https://avro.apache.org/). + +Flink supports to interpret Debezium JSON and Avro messages as INSERT/UPDATE/DELETE messages into Flink SQL system. This is useful in many cases to leverage this feature, such as + - synchronizing incremental data from databases to other systems + - auditing logs + - real-time materialized views on databases + - temporal join changing history of a database table and so on. + +Flink also supports to encode the INSERT/UPDATE/DELETE messages in Flink SQL as Debezium JSON or Avro messages, and emit to external systems like Kafka. +However, currently Flink can't combine UPDATE_BEFORE and UPDATE_AFTER into a single UPDATE message. Therefore, Flink encodes UPDATE_BEFORE and UDPATE_AFTER as DELETE and INSERT Debezium messages. + +Dependencies +------------ + +#### Debezium Avro + +{{< sql_download_table "debezium-avro-confluent" >}} + +#### Debezium Json + +{{< sql_download_table "debezium-json" >}} + + +*Note: please refer to [Debezium documentation](https://debezium.io/documentation/reference/1.3/index.html) about how to setup a Debezium Kafka Connect to synchronize changelog to Kafka topics.* + + +How to use Debezium format +---------------- + +Debezium provides a unified format for changelog, here is a simple example for an update operation captured from a MySQL `products` table in JSON format: + +```json +{ + "before": { + "id": 111, + "name": "scooter", + "description": "Big 2-wheel scooter", + "weight": 5.18 + }, + "after": { + "id": 111, + "name": "scooter", + "description": "Big 2-wheel scooter", + "weight": 5.15 + }, + "source": {...}, + "op": "u", + "ts_ms": 1589362330904, + "transaction": null +} +``` + +*Note: please refer to [Debezium documentation](https://debezium.io/documentation/reference/1.3/connectors/mysql.html#mysql-connector-events_debezium) about the meaning of each fields.* + +The MySQL `products` table has 4 columns (`id`, `name`, `description` and `weight`). The above JSON message is an update change event on the `products` table where the `weight` value of the row with `id = 111` is changed from `5.18` to `5.15`. +Assuming this messages is synchronized to Kafka topic `products_binlog`, then we can use the following DDL to consume this topic and interpret the change events. + +```sql +CREATE TABLE topic_products ( + -- schema is totally the same to the MySQL "products" table + id BIGINT, + name STRING, + description STRING, + weight DECIMAL(10, 2) +) WITH ( + 'connector' = 'kafka', + 'topic' = 'products_binlog', + 'properties.bootstrap.servers' = 'localhost:9092', + 'properties.group.id' = 'testGroup', + -- using 'debezium-json' as the format to interpret Debezium JSON messages + -- please use 'debezium-avro-confluent' if Debezium encodes messages in Avro format + 'format' = 'debezium-json' +) +``` + +In some cases, users may setup the Debezium Kafka Connect with the Kafka configuration `'value.converter.schemas.enable'` enabled to include schema in the message. Then the Debezium JSON message may look like this: + +```json +{ + "schema": {...}, + "payload": { + "before": { + "id": 111, + "name": "scooter", + "description": "Big 2-wheel scooter", + "weight": 5.18 + }, + "after": { + "id": 111, + "name": "scooter", + "description": "Big 2-wheel scooter", + "weight": 5.15 + }, + "source": {...}, + "op": "u", + "ts_ms": 1589362330904, + "transaction": null + } +} +``` + +In order to interpret such messages, you need to add the option `'debezium-json.schema-include' = 'true'` into above DDL WITH clause (`false` by default). Usually, this is not recommended to include schema because this makes the messages very verbose and reduces parsing performance. + +After registering the topic as a Flink table, then you can consume the Debezium messages as a changelog source. + +```sql +-- a real-time materialized view on the MySQL "products" +-- which calculate the latest average of weight for the same products +SELECT name, AVG(weight) FROM topic_products GROUP BY name; + +-- synchronize all the data and incremental changes of MySQL "products" table to +-- Elasticsearch "products" index for future searching +INSERT INTO elasticsearch_products +SELECT * FROM topic_products; +``` + +Available Metadata +------------------ + +The following format metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition. + +Attention Format metadata fields are only available if the +corresponding connector forwards format metadata. Currently, only the Kafka connector is able to expose +metadata fields for its value format. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyData TypeDescription
schemaSTRING NULLJSON string describing the schema of the payload. Null if the schema is not included in + the Debezium record.
ingestion-timestampTIMESTAMP(3) WITH LOCAL TIME ZONE NULLThe timestamp at which the connector processed the event. Corresponds to the ts_ms + field in the Debezium record.
source.timestampTIMESTAMP(3) WITH LOCAL TIME ZONE NULLThe timestamp at which the source system created the event. Corresponds to the source.ts_ms + field in the Debezium record.
source.databaseSTRING NULLThe originating database. Corresponds to the source.db field in the + Debezium record if available.
source.schemaSTRING NULLThe originating database schema. Corresponds to the source.schema field in the + Debezium record if available.
source.tableSTRING NULLThe originating database table. Corresponds to the source.table or source.collection + field in the Debezium record if available.
source.propertiesMAP<STRING, STRING> NULLMap of various source properties. Corresponds to the source field in the Debezium record.
+ +The following example shows how to access Debezium metadata fields in Kafka: + +```sql +CREATE TABLE KafkaTable ( + origin_ts TIMESTAMP(3) METADATA FROM 'value.ingestion-timestamp' VIRTUAL, + event_time TIMESTAMP(3) METADATA FROM 'value.source.timestamp' VIRTUAL, + origin_database STRING METADATA FROM 'value.source.database' VIRTUAL, + origin_schema STRING METADATA FROM 'value.source.schema' VIRTUAL, + origin_table STRING METADATA FROM 'value.source.table' VIRTUAL, + origin_properties MAP METADATA FROM 'value.source.properties' VIRTUAL, + user_id BIGINT, + item_id BIGINT, + behavior STRING +) WITH ( + 'connector' = 'kafka', + 'topic' = 'user_behavior', + 'properties.bootstrap.servers' = 'localhost:9092', + 'properties.group.id' = 'testGroup', + 'scan.startup.mode' = 'earliest-offset', + 'value.format' = 'debezium-json' +); +``` + +Format Options +---------------- + +Flink provides `debezium-avro-confluent` and `debezium-json` formats to interpret Avro or Json messages produced by Debezium. +Use format `debezium-avro-confluent` to interpret Debezium Avro messages and format `debezium-json` to interpret Debezium Json messages. + +{{< tabs "a8edce02-58d5-4e0b-bc4b-75d05a98a0f9" >}} +{{< tab "Debezium Avro" >}} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequiredDefaultTypeDescription
format
required(none)StringSpecify what format to use, here should be 'debezium-avro-confluent'.
debezium-avro-confluent.schema-registry.url
required(none)StringThe URL of the Confluent Schema Registry to fetch/register schemas.
debezium-avro-confluent.schema-registry.subject
optional(none)StringThe Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, kafka connector use "<topic_name>-value" as the default subject name when debezium-avro-confluent is used as the value format. But for other connectors (e.g. filesystem), the subject option is required when used as sink.
+ +{{< /tab >}} +{{< tab "Debezium Json" >}} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequiredDefaultTypeDescription
format
required(none)StringSpecify what format to use, here should be 'debezium-json'.
debezium-json.schema-include
optionalfalseBooleanWhen setting up a Debezium Kafka Connect, users may enable a Kafka configuration 'value.converter.schemas.enable' to include schema in the message. + This option indicates whether the Debezium JSON message includes the schema or not.
debezium-json.ignore-parse-errors
optionalfalseBooleanSkip fields and rows with parse errors instead of failing. + Fields are set to null in case of errors.
debezium-json.timestamp-format.standard
optional'SQL'StringSpecify the input and output timestamp format. Currently supported values are 'SQL' and 'ISO-8601': +
    +
  • Option 'SQL' will parse input timestamp in "yyyy-MM-dd HH:mm:ss.s{precision}" format, e.g '2020-12-30 12:13:14.123' and output timestamp in the same format.
  • +
  • Option 'ISO-8601'will parse input timestamp in "yyyy-MM-ddTHH:mm:ss.s{precision}" format, e.g '2020-12-30T12:13:14.123' and output timestamp in the same format.
  • +
+
debezium-json.map-null-key.mode
optional'FAIL'StringSpecify the handling mode when serializing null keys for map data. Currently supported values are 'FAIL', 'DROP' and 'LITERAL': +
    +
  • Option 'FAIL' will throw exception when encountering map with null key.
  • +
  • Option 'DROP' will drop null key entries for map data.
  • +
  • Option 'LITERAL' will replace null key with string literal. The string literal is defined by debezium-json.map-null-key.literal option.
  • +
+
debezium-json.map-null-key.literal
optional'null'StringSpecify string literal to replace null key when 'debezium-json.map-null-key.mode' is LITERAL.
debezium-json.encode.decimal-as-plain-number
optionalfalseBooleanEncode all decimals as plain numbers instead of possible scientific notations. By default, decimals may be written using scientific notation. For example, 0.000000027 is encoded as 2.7E-8 by default, and will be written as 0.000000027 if set this option to true.
+ + + +Caveats +---------------- + +### Duplicate change events + +Under normal operating scenarios, the Debezium application delivers every change event **exactly-once**. Flink works pretty well when consuming Debezium produced events in this situation. +However, Debezium application works in **at-least-once** delivery if any failover happens. See more details about delivery guarantee from [Debezium documentation](https://debezium.io/documentation/faq/#what_happens_when_an_application_stops_or_crashes). +That means, in the abnormal situations, Debezium may deliver duplicate change events to Kafka and Flink will get the duplicate events. +This may cause Flink query to get wrong results or unexpected exceptions. Thus, it is recommended to set job configuration [`table.exec.source.cdc-events-duplicate`]({{< ref "docs/dev/table/config" >}}#table-exec-source-cdc-events-duplicate) to `true` and define PRIMARY KEY on the source in this situation. +Framework will generate an additional stateful operator, and use the primary key to deduplicate the change events and produce a normalized changelog stream. + +### Consuming data produced by Debezium Postgres Connector + +If you are using [Debezium Connector for PostgreSQL](https://debezium.io/documentation/reference/1.2/connectors/postgresql.html) to capture the changes to Kafka, please make sure the [REPLICA IDENTITY](https://www.postgresql.org/docs/current/sql-altertable.html#SQL-CREATETABLE-REPLICA-IDENTITY) configuration of the monitored PostgreSQL table has been set to `FULL` which is by default `DEFAULT`. +Otherwise, Flink SQL currently will fail to interpret the Debezium data. + +In `FULL` strategy, the UPDATE and DELETE events will contain the previous values of all the table’s columns. In other strategies, the "before" field of UPDATE and DELETE events will only contain primary key columns or null if no primary key. +You can change the `REPLICA IDENTITY` by running `ALTER TABLE REPLICA IDENTITY FULL`. +See more details in [Debezium Documentation for PostgreSQL REPLICA IDENTITY](https://debezium.io/documentation/reference/1.2/connectors/postgresql.html#postgresql-replica-identity). + +Data Type Mapping +---------------- + +Currently, the Debezium format uses JSON and Avro format for serialization and deserialization. Please refer to [JSON Format documentation]({{< ref "docs/connectors/table/formats/json" >}}#data-type-mapping) and [Confluent Avro Format documentation]({< ref "docs/connectors/table/formats/avro-confluent" >}}#data-type-mapping) for more details about the data type mapping. + From 942784c28e0a45a08d6444545dec1b773bff4537 Mon Sep 17 00:00:00 2001 From: yangsanity <471419897@qq.com> Date: Mon, 22 Feb 2021 16:43:18 +0800 Subject: [PATCH 134/322] [FLINK-21176][docs-zh] Translate the updates of 'avro-confluent.md' to Chinese This closes #14896 --- .../table/formats/avro-confluent.md | 51 +++++++++---------- .../table/formats/avro-confluent.md | 4 +- 2 files changed, 27 insertions(+), 28 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md index 996269946..845ca1f90 100644 --- a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md @@ -31,13 +31,13 @@ under the License. -Avro Schema Registry (``avro-confluent``) 格式能让你读取被 ``io.confluent.kafka.serializers.KafkaAvroSerializer``序列化的记录,以及可以写入成能被 ``io.confluent.kafka.serializers.KafkaAvroDeserializer``反序列化的记录。 +Avro Schema Registry (``avro-confluent``) 格式能让你读取被 ``io.confluent.kafka.serializers.KafkaAvroSerializer`` 序列化的记录,以及可以写入成能被 ``io.confluent.kafka.serializers.KafkaAvroDeserializer`` 反序列化的记录。 当以这种格式读取(反序列化)记录时,将根据记录中编码的 schema 版本 id 从配置的 Confluent Schema Registry 中获取 Avro writer schema ,而从 table schema 中推断出 reader schema。 当以这种格式写入(序列化)记录时,Avro schema 是从 table schema 中推断出来的,并会用来检索要与数据一起编码的 schema id。我们会在配置的 Confluent Schema Registry 中配置的 [subject](https://docs.confluent.io/current/schema-registry/index.html#schemas-subjects-and-topics) 下,检索 schema id。subject 通过 `avro-confluent.schema-registry.subject` 参数来制定。 -The Avro Schema Registry format can only be used in conjunction with the [Apache Kafka SQL connector]({{< ref "docs/connectors/table/kafka" >}}) or the [Upsert Kafka SQL Connector]({{< ref "docs/connectors/table/upsert-kafka" >}}). +Avro Schema Registry 格式只能与 [Apache Kafka SQL 连接器]({{< ref "docs/connectors/table/kafka" >}})或 [Upsert Kafka SQL 连接器]({{< ref "docs/connectors/table/upsert-kafka" >}})一起使用。 依赖 ------------ @@ -52,17 +52,17 @@ The Avro Schema Registry format can only be used in conjunction with the [Apache {{< tabs "3df131fd-0e20-4635-a8f9-3574a764db7a" >}} {{< tab "SQL" >}} -Example of a table using raw UTF-8 string as Kafka key and Avro records registered in the Schema Registry as Kafka values: +使用原始的 UTF-8 字符串作为 Kafka 的 key,Schema Registry 中注册的 Avro 记录作为 Kafka 的 values 的表的示例: ```sql CREATE TABLE user_created ( - -- one column mapped to the Kafka raw UTF-8 key + -- 该列映射到 Kafka 原始的 UTF-8 key the_kafka_key STRING, - -- a few columns mapped to the Avro fields of the Kafka value + -- 映射到 Kafka value 中的 Avro 字段的一些列 id STRING, - name STRING, + name STRING, email STRING ) WITH ( @@ -71,7 +71,7 @@ CREATE TABLE user_created ( 'topic' = 'user_events_example1', 'properties.bootstrap.servers' = 'localhost:9092', - -- UTF-8 string as Kafka keys, using the 'the_kafka_key' table column + -- UTF-8 字符串作为 Kafka 的 keys,使用表中的 'the_kafka_key' 列 'key.format' = 'raw', 'key.fields' = 'the_kafka_key', @@ -81,30 +81,30 @@ CREATE TABLE user_created ( ) ``` -We can write data into the kafka table as follows: +我们可以像下面这样将数据写入到 kafka 表中: ```sql INSERT INTO user_created SELECT - -- replicating the user id into a column mapped to the kafka key + -- 将 user id 复制至映射到 kafka key 的列中 id as the_kafka_key, - -- all values + -- 所有的 values id, name, email FROM some_table ``` --- -Example of a table with both the Kafka key and value registered as Avro records in the Schema Registry: +Kafka 的 key 和 value 在 Schema Registry 中都注册为 Avro 记录的表的示例: ```sql CREATE TABLE user_created ( - -- one column mapped to the 'id' Avro field of the Kafka key + -- 该列映射到 Kafka key 中的 Avro 字段 'id' kafka_key_id STRING, - -- a few columns mapped to the Avro fields of the Kafka value + -- 映射到 Kafka value 中的 Avro 字段的一些列 id STRING, name STRING, email STRING @@ -115,41 +115,40 @@ CREATE TABLE user_created ( 'topic' = 'user_events_example2', 'properties.bootstrap.servers' = 'localhost:9092', - -- Watch out: schema evolution in the context of a Kafka key is almost never backward nor - -- forward compatible due to hash partitioning. + -- 注意:由于哈希分区,在 Kafka key 的上下文中,schema 升级几乎从不向后也不向前兼容。 'key.format' = 'avro-confluent', 'key.avro-confluent.schema-registry.url' = 'http://localhost:8082', 'key.fields' = 'kafka_key_id', - -- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id' - -- => adding a prefix to the table column associated to the Kafka key field avoids clashes + -- 在本例中,我们希望 Kafka 的 key 和 value 的 Avro 类型都包含 'id' 字段 + -- => 给表中与 Kafka key 字段关联的列添加一个前缀来避免冲突 'key.fields-prefix' = 'kafka_key_', 'value.format' = 'avro-confluent', 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', 'value.fields-include' = 'EXCEPT_KEY', - -- subjects have a default value since Flink 1.13, though can be overriden: + -- 自 Flink 1.13 起,subjects 具有一个默认值, 但是可以被覆盖: 'key.avro-confluent.schema-registry.subject' = 'user_events_example2-key2', 'value.avro-confluent.schema-registry.subject' = 'user_events_example2-value2' ) ``` --- -Example of a table using the upsert connector with the Kafka value registered as an Avro record in the Schema Registry: +使用 upsert-kafka 连接器,Kafka 的 value 在 Schema Registry 中注册为 Avro 记录的表的示例: ```sql CREATE TABLE user_created ( - -- one column mapped to the Kafka raw UTF-8 key + -- 该列映射到 Kafka 原始的 UTF-8 key kafka_key_id STRING, - -- a few columns mapped to the Avro fields of the Kafka value + -- 映射到 Kafka value 中的 Avro 字段的一些列 id STRING, name STRING, email STRING, - -- upsert-kafka connector requires a primary key to define the upsert behavior + -- upsert-kafka 连接器需要一个主键来定义 upsert 行为 PRIMARY KEY (kafka_key_id) NOT ENFORCED ) WITH ( @@ -158,12 +157,12 @@ CREATE TABLE user_created ( 'topic' = 'user_events_example3', 'properties.bootstrap.servers' = 'localhost:9092', - -- UTF-8 string as Kafka keys - -- We don't specify 'key.fields' in this case since it's dictated by the primary key of the table + -- UTF-8 字符串作为 Kafka 的 keys + -- 在本例中我们不指定 'key.fields',因为它由表的主键决定 'key.format' = 'raw', - -- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id' - -- => adding a prefix to the table column associated to the kafka key field to avoid clashes + -- 在本例中,我们希望 Kafka 的 key 和 value 的 Avro 类型都包含 'id' 字段 + -- => 给表中与 Kafka key 字段关联的列添加一个前缀来避免冲突 'key.fields-prefix' = 'kafka_key_', 'value.format' = 'avro-confluent', diff --git a/docs/content/docs/connectors/table/formats/avro-confluent.md b/docs/content/docs/connectors/table/formats/avro-confluent.md index dc36a8750..e784ce74c 100644 --- a/docs/content/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content/docs/connectors/table/formats/avro-confluent.md @@ -129,7 +129,7 @@ CREATE TABLE user_created ( ``` --- -Example of a table using the upsert connector with the Kafka value registered as an Avro record in the Schema Registry: +Example of a table using the upsert-kafka connector with the Kafka value registered as an Avro record in the Schema Registry: ```sql CREATE TABLE user_created ( @@ -156,7 +156,7 @@ CREATE TABLE user_created ( 'key.format' = 'raw', -- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id' - -- => adding a prefix to the table column associated to the kafka key field to avoid clashes + -- => adding a prefix to the table column associated to the kafka key field avoids clashes 'key.fields-prefix' = 'kafka_key_', 'value.format' = 'avro-confluent', From ed7b269317bbbc9b1d70ce6f2b0970165c06fff9 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Tue, 9 Mar 2021 17:48:28 +0800 Subject: [PATCH 135/322] [FLINK-21620][table] Support abbreviation TIMESTAMP_LTZ for TIMESTAMP WITH LOCAL TIME ZONE This closes #15124. --- docs/content.zh/docs/connectors/table/formats/debezium.md | 4 ++-- docs/content/docs/connectors/table/formats/debezium.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/formats/debezium.md b/docs/content.zh/docs/connectors/table/formats/debezium.md index e860d3356..d0283934b 100644 --- a/docs/content.zh/docs/connectors/table/formats/debezium.md +++ b/docs/content.zh/docs/connectors/table/formats/debezium.md @@ -179,13 +179,13 @@ metadata fields for its value format. ingestion-timestamp - TIMESTAMP(3) WITH LOCAL TIME ZONE NULL + TIMESTAMP_LTZ(3) NULL The timestamp at which the connector processed the event. Corresponds to the ts_ms field in the Debezium record. source.timestamp - TIMESTAMP(3) WITH LOCAL TIME ZONE NULL + TIMESTAMP_LTZ(3) NULL The timestamp at which the source system created the event. Corresponds to the source.ts_ms field in the Debezium record. diff --git a/docs/content/docs/connectors/table/formats/debezium.md b/docs/content/docs/connectors/table/formats/debezium.md index eee4534e9..c64debd32 100644 --- a/docs/content/docs/connectors/table/formats/debezium.md +++ b/docs/content/docs/connectors/table/formats/debezium.md @@ -172,13 +172,13 @@ metadata fields for its value format. ingestion-timestamp - TIMESTAMP(3) WITH LOCAL TIME ZONE NULL + TIMESTAMP_LTZ(3) NULL The timestamp at which the connector processed the event. Corresponds to the ts_ms field in the Debezium record. source.timestamp - TIMESTAMP(3) WITH LOCAL TIME ZONE NULL + TIMESTAMP_LTZ(3) NULL The timestamp at which the source system created the event. Corresponds to the source.ts_ms field in the Debezium record. From 58183352f8356d8aaa85d4a0332d139ed4bf608e Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 18 Mar 2021 12:13:17 +0100 Subject: [PATCH 136/322] [FLINK-21913][table][connectors] Update DynamicTableFactory.Context to use ResolvedCatalogTable This closes #15316. --- .../RegistryAvroFormatFactoryTest.java | 56 +++++++----------- .../DebeziumAvroFormatFactoryTest.java | 51 +++++------------ .../formats/json/JsonFormatFactoryTest.java | 54 ++++-------------- .../canal/CanalJsonFormatFactoryTest.java | 49 ++++------------ .../DebeziumJsonFormatFactoryTest.java | 57 +++++-------------- .../maxwell/MaxwellJsonFormatFactoryTest.java | 57 ++++++------------- 6 files changed, 88 insertions(+), 236 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java index b1aab7903..3171c82c6 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java @@ -20,21 +20,18 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; import org.apache.flink.formats.avro.AvroToRowDataConverters; import org.apache.flink.formats.avro.RowDataToAvroConverters; import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; @@ -49,19 +46,24 @@ import java.util.function.Consumer; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.hamcrest.CoreMatchers.instanceOf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; /** Tests for the {@link RegistryAvroFormatFactory}. */ public class RegistryAvroFormatFactoryTest { - private static final TableSchema SCHEMA = - TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + + private static final ResolvedSchema SCHEMA = + ResolvedSchema.of( + Column.physical("a", DataTypes.STRING()), + Column.physical("b", DataTypes.INT()), + Column.physical("c", DataTypes.BOOLEAN())); + + private static final RowType ROW_TYPE = + (RowType) SCHEMA.toPhysicalRowDataType().getLogicalType(); + private static final String SUBJECT = "test-subject"; private static final String REGISTRY_URL = "http://localhost:8081"; @@ -76,14 +78,14 @@ public void testDeserializationSchema() { AvroToRowDataConverters.createRowConverter(ROW_TYPE), InternalTypeInfo.of(ROW_TYPE)); - final DynamicTableSource actualSource = createTableSource(getDefaultOptions()); + final DynamicTableSource actualSource = createTableSource(SCHEMA, getDefaultOptions()); assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; DeserializationSchema actualDeser = scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); + ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); assertEquals(expectedDeser, actualDeser); } @@ -99,13 +101,13 @@ public void testSerializationSchema() { REGISTRY_URL), RowDataToAvroConverters.createConverter(ROW_TYPE)); - final DynamicTableSink actualSink = createTableSink(getDefaultOptions()); + final DynamicTableSink actualSink = createTableSink(SCHEMA, getDefaultOptions()); assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; SerializationSchema actualSer = - sinkMock.valueFormat.createRuntimeEncoder(null, SCHEMA.toRowDataType()); + sinkMock.valueFormat.createRuntimeEncoder(null, SCHEMA.toPhysicalRowDataType()); assertEquals(expectedSer, actualSer); } @@ -122,7 +124,7 @@ public void testMissingSubjectForSink() { final Map options = getModifiedOptions(opts -> opts.remove("avro-confluent.schema-registry.subject")); - createTableSink(options); + createTableSink(SCHEMA, options); } // ------------------------------------------------------------------------ @@ -151,24 +153,4 @@ private Map getDefaultOptions() { options.put("avro-confluent.schema-registry.url", REGISTRY_URL); return options; } - - private DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - RegistryAvroFormatFactoryTest.class.getClassLoader(), - false); - } - - private DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - RegistryAvroFormatFactoryTest.class.getClassLoader(), - false); - } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java index 3e442a4fa..132c925fb 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java @@ -20,15 +20,12 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; @@ -45,6 +42,8 @@ import java.util.Map; import static junit.framework.TestCase.assertEquals; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.hamcrest.CoreMatchers.instanceOf; import static org.junit.Assert.assertThat; @@ -52,14 +51,14 @@ public class DebeziumAvroFormatFactoryTest extends TestLogger { @Rule public ExpectedException thrown = ExpectedException.none(); - private static final TableSchema SCHEMA = - TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); + private static final ResolvedSchema SCHEMA = + ResolvedSchema.of( + Column.physical("a", DataTypes.STRING()), + Column.physical("b", DataTypes.INT()), + Column.physical("c", DataTypes.BOOLEAN())); - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + private static final RowType ROW_TYPE = + (RowType) SCHEMA.toPhysicalRowDataType().getLogicalType(); private static final String SUBJECT = "test-debezium-avro"; private static final String REGISTRY_URL = "http://localhost:8081"; @@ -94,43 +93,23 @@ private Map getAllOptions() { private static DeserializationSchema createDeserializationSchema( Map options) { - final DynamicTableSource actualSource = createTableSource(options); + final DynamicTableSource actualSource = createTableSource(SCHEMA, options); assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; return scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); + ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); } private static SerializationSchema createSerializationSchema( Map options) { - final DynamicTableSink actualSink = createTableSink(options); + final DynamicTableSink actualSink = createTableSink(SCHEMA, options); assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; return sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), SCHEMA.toRowDataType()); - } - - private static DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - DebeziumAvroFormatFactoryTest.class.getClassLoader(), - false); - } - - private static DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - DebeziumAvroFormatFactoryTest.class.getClassLoader(), - false); + new SinkRuntimeProviderContext(false), SCHEMA.toPhysicalRowDataType()); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index ffc64e45d..036fdbebe 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -20,21 +20,14 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; import org.junit.Rule; @@ -46,20 +39,17 @@ import java.util.function.Consumer; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; +import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; +import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.junit.Assert.assertEquals; /** Tests for the {@link JsonFormatFactory}. */ public class JsonFormatFactoryTest extends TestLogger { @Rule public ExpectedException thrown = ExpectedException.none(); - private static final TableSchema SCHEMA = - TableSchema.builder() - .field("field1", DataTypes.BOOLEAN()) - .field("field2", DataTypes.INT()) - .build(); - - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); - @Test public void testSeDeSchema() { final Map tableOptions = getAllOptions(); @@ -150,20 +140,20 @@ public void testLowerCaseOptionForMapNullKeyMode() { private void testSchemaDeserializationSchema(Map options) { final JsonRowDataDeserializationSchema expectedDeser = new JsonRowDataDeserializationSchema( - ROW_TYPE, - InternalTypeInfo.of(ROW_TYPE), + PHYSICAL_TYPE, + InternalTypeInfo.of(PHYSICAL_TYPE), false, true, TimestampFormat.ISO_8601); - final DynamicTableSource actualSource = createTableSource(options); + final DynamicTableSource actualSource = createTableSource(SCHEMA, options); assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; DeserializationSchema actualDeser = scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); + ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); assertEquals(expectedDeser, actualDeser); } @@ -171,20 +161,20 @@ private void testSchemaDeserializationSchema(Map options) { private void testSchemaSerializationSchema(Map options) { final JsonRowDataSerializationSchema expectedSer = new JsonRowDataSerializationSchema( - ROW_TYPE, + PHYSICAL_TYPE, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, "null", true); - final DynamicTableSink actualSink = createTableSink(options); + final DynamicTableSink actualSink = createTableSink(SCHEMA, options); assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; SerializationSchema actualSer = sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), SCHEMA.toRowDataType()); + new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); assertEquals(expectedSer, actualSer); } @@ -215,24 +205,4 @@ private Map getAllOptions() { options.put("json.encode.decimal-as-plain-number", "true"); return options; } - - private static DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "Mock scan table"), - new Configuration(), - JsonFormatFactoryTest.class.getClassLoader(), - false); - } - - private static DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "Mock sink table"), - new Configuration(), - JsonFormatFactoryTest.class.getClassLoader(), - false); - } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 10adde092..8b477367b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -20,24 +20,16 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; import org.junit.Rule; @@ -50,24 +42,19 @@ import java.util.function.Consumer; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; +import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; +import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.junit.Assert.assertEquals; /** Tests for {@link CanalJsonFormatFactory}. */ public class CanalJsonFormatFactoryTest extends TestLogger { @Rule public ExpectedException thrown = ExpectedException.none(); - private static final TableSchema SCHEMA = - TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); - - private static final DataType PHYSICAL_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); - - private static final RowType ROW_TYPE = (RowType) PHYSICAL_DATA_TYPE.getLogicalType(); - - private static final InternalTypeInfo ROW_TYPE_INFO = InternalTypeInfo.of(ROW_TYPE); + private static final InternalTypeInfo ROW_TYPE_INFO = + InternalTypeInfo.of(PHYSICAL_TYPE); @Test public void testDefaultOptions() { @@ -86,7 +73,7 @@ public void testDefaultOptions() { // test Ser CanalJsonSerializationSchema expectedSer = new CanalJsonSerializationSchema( - ROW_TYPE, + PHYSICAL_TYPE, TimestampFormat.SQL, JsonOptions.MapNullKeyMode.FAIL, "null", @@ -121,7 +108,7 @@ public void testUserDefinedOptions() { // test Ser CanalJsonSerializationSchema expectedSer = new CanalJsonSerializationSchema( - ROW_TYPE, + PHYSICAL_TYPE, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, "nullKey", @@ -196,14 +183,7 @@ private Map getAllOptions() { private static DeserializationSchema createDeserializationSchema( Map options) { - DynamicTableSource source = - FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - CanalJsonFormatFactoryTest.class.getClassLoader(), - false); + DynamicTableSource source = createTableSource(SCHEMA, options); assert source instanceof TestDynamicTableFactory.DynamicTableSourceMock; TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = @@ -215,14 +195,7 @@ private static DeserializationSchema createDeserializationSchema( private static SerializationSchema createSerializationSchema( Map options) { - DynamicTableSink sink = - FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - CanalJsonFormatFactoryTest.class.getClassLoader(), - false); + DynamicTableSink sink = createTableSink(SCHEMA, options); assert sink instanceof TestDynamicTableFactory.DynamicTableSinkMock; TestDynamicTableFactory.DynamicTableSinkMock sinkMock = diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index cda0188f3..bdd014a87 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -20,23 +20,16 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; @@ -50,6 +43,11 @@ import java.util.function.Consumer; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; +import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; +import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -57,29 +55,20 @@ public class DebeziumJsonFormatFactoryTest extends TestLogger { @Rule public ExpectedException thrown = ExpectedException.none(); - private static final TableSchema SCHEMA = - TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); - - private static final DataType PHYSICAL_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); - @Test public void testSeDeSchema() { final DebeziumJsonDeserializationSchema expectedDeser = new DebeziumJsonDeserializationSchema( PHYSICAL_DATA_TYPE, Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), + InternalTypeInfo.of(PHYSICAL_TYPE), false, true, TimestampFormat.ISO_8601); final Map options = getAllOptions(); - final DynamicTableSource actualSource = createTableSource(options); + final DynamicTableSource actualSource = createTableSource(SCHEMA, options); assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; @@ -98,7 +87,7 @@ public void testSeDeSchema() { "null", true); - final DynamicTableSink actualSink = createTableSink(options); + final DynamicTableSink actualSink = createTableSink(SCHEMA, options); assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; @@ -120,7 +109,7 @@ public void testInvalidIgnoreParseError() { final Map options = getModifiedOptions(opts -> opts.put("debezium-json.ignore-parse-errors", "abc")); - createTableSource(options); + createTableSource(SCHEMA, options); } @Test @@ -136,7 +125,7 @@ public void testSchemaIncludeOption() { true, true, TimestampFormat.ISO_8601); - final DynamicTableSource actualSource = createTableSource(options); + final DynamicTableSource actualSource = createTableSource(SCHEMA, options); TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; DeserializationSchema actualDeser = @@ -145,7 +134,7 @@ public void testSchemaIncludeOption() { assertEquals(expectedDeser, actualDeser); try { - final DynamicTableSink actualSink = createTableSink(options); + final DynamicTableSink actualSink = createTableSink(SCHEMA, options); TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; // should fail @@ -171,7 +160,7 @@ public void testInvalidOptionForTimestampFormat() { containsCause( new ValidationException( "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - createTableSource(tableOptions); + createTableSource(SCHEMA, tableOptions); } @Test @@ -184,7 +173,7 @@ public void testInvalidOptionForMapNullKeyMode() { containsCause( new ValidationException( "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - createTableSink(tableOptions); + createTableSink(SCHEMA, tableOptions); } // ------------------------------------------------------------------------ @@ -216,24 +205,4 @@ private Map getAllOptions() { options.put("debezium-json.encode.decimal-as-plain-number", "true"); return options; } - - private static DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - DebeziumJsonFormatFactoryTest.class.getClassLoader(), - false); - } - - private static DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - DebeziumJsonFormatFactoryTest.class.getClassLoader(), - false); - } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java index d7c12acd3..f2fc7d65f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java @@ -20,18 +20,15 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; @@ -48,20 +45,22 @@ import java.util.function.Consumer; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.junit.Assert.assertEquals; /** Tests for {@link MaxwellJsonFormatFactory}. */ public class MaxwellJsonFormatFactoryTest extends TestLogger { @Rule public ExpectedException thrown = ExpectedException.none(); - private static final TableSchema SCHEMA = - TableSchema.builder() - .field("a", DataTypes.STRING()) - .field("b", DataTypes.INT()) - .field("c", DataTypes.BOOLEAN()) - .build(); + private static final ResolvedSchema SCHEMA = + ResolvedSchema.of( + Column.physical("a", DataTypes.STRING()), + Column.physical("b", DataTypes.INT()), + Column.physical("c", DataTypes.BOOLEAN())); - private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType(); + private static final RowType ROW_TYPE = + (RowType) SCHEMA.toPhysicalRowDataType().getLogicalType(); @Test public void testSeDeSchema() { @@ -79,25 +78,25 @@ public void testSeDeSchema() { final Map options = getAllOptions(); - final DynamicTableSource actualSource = createTableSource(options); + final DynamicTableSource actualSource = createTableSource(SCHEMA, options); assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; DeserializationSchema actualDeser = scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, SCHEMA.toRowDataType()); + ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); assertEquals(expectedDeser, actualDeser); - final DynamicTableSink actualSink = createTableSink(options); + final DynamicTableSink actualSink = createTableSink(SCHEMA, options); assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; SerializationSchema actualSer = sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), SCHEMA.toRowDataType()); + new SinkRuntimeProviderContext(false), SCHEMA.toPhysicalRowDataType()); assertEquals(expectedSer, actualSer); } @@ -112,7 +111,7 @@ public void testInvalidIgnoreParseError() { final Map options = getModifiedOptions(opts -> opts.put("maxwell-json.ignore-parse-errors", "abc")); - createTableSource(options); + createTableSource(SCHEMA, options); } @Test @@ -126,7 +125,7 @@ public void testInvalidOptionForTimestampFormat() { containsCause( new ValidationException( "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - createTableSource(tableOptions); + createTableSource(SCHEMA, tableOptions); } @Test @@ -139,7 +138,7 @@ public void testInvalidOptionForMapNullKeyMode() { containsCause( new ValidationException( "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - createTableSink(tableOptions); + createTableSink(SCHEMA, tableOptions); } // ------------------------------------------------------------------------ @@ -171,24 +170,4 @@ private Map getAllOptions() { options.put("maxwell-json.encode.decimal-as-plain-number", "true"); return options; } - - private static DynamicTableSource createTableSource(Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock source"), - new Configuration(), - MaxwellJsonFormatFactoryTest.class.getClassLoader(), - false); - } - - private static DynamicTableSink createTableSink(Map options) { - return FactoryUtil.createTableSink( - null, - ObjectIdentifier.of("default", "default", "t1"), - new CatalogTableImpl(SCHEMA, options, "mock sink"), - new Configuration(), - MaxwellJsonFormatFactoryTest.class.getClassLoader(), - false); - } } From 9fef485193d30c0f34c25b7790d7cb7bca8099d2 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Wed, 24 Mar 2021 17:10:23 +0800 Subject: [PATCH 137/322] [FLINK-21947][csv] Support TIMESTAMP_LTZ type in CSV format This closes #15356 --- .../flink-json-debezium/pom.xml | 6 ++ .../flink/formats/json/JsonFormatFactory.java | 1 + .../flink/formats/json/JsonOptions.java | 1 + .../JsonRowDataDeserializationSchema.java | 1 + .../json/JsonRowDataSerializationSchema.java | 1 + .../json/JsonRowDeserializationSchema.java | 4 +- .../json/JsonRowSerializationSchema.java | 4 +- .../formats/json/JsonToRowDataConverters.java | 11 +-- .../formats/json/RowDataToJsonConverters.java | 11 +-- .../flink/formats/json/TimeFormats.java | 87 ------------------- .../flink/formats/json/TimestampFormat.java | 39 --------- .../json/canal/CanalJsonDecodingFormat.java | 2 +- .../canal/CanalJsonDeserializationSchema.java | 2 +- .../json/canal/CanalJsonFormatFactory.java | 2 +- .../canal/CanalJsonSerializationSchema.java | 2 +- .../debezium/DebeziumJsonDecodingFormat.java | 2 +- .../DebeziumJsonDeserializationSchema.java | 2 +- .../debezium/DebeziumJsonFormatFactory.java | 2 +- .../DebeziumJsonSerializationSchema.java | 2 +- .../MaxwellJsonDeserializationSchema.java | 2 +- .../maxwell/MaxwellJsonFormatFactory.java | 2 +- .../MaxwellJsonSerializationSchema.java | 2 +- .../formats/json/JsonFormatFactoryTest.java | 1 + .../json/JsonRowDataSerDeSchemaTest.java | 1 + .../canal/CanalJsonFormatFactoryTest.java | 2 +- .../json/canal/CanalJsonSerDeSchemaTest.java | 2 +- .../DebeziumJsonFormatFactoryTest.java | 2 +- .../debezium/DebeziumJsonSerDeSchemaTest.java | 2 +- .../maxwell/MaxwellJsonFormatFactoryTest.java | 2 +- .../json/maxwell/MaxwellJsonSerDerTest.java | 2 +- 30 files changed, 45 insertions(+), 157 deletions(-) delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index f7c653f14..a5217b742 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.flink + flink-format-common + ${project.version} + + diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index 51338b273..8d69ec254 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java index 3ffa18bbf..ae6243072 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.ValidationException; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index c54cefdfa..83d1b2dc3 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.RowType; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index b2afe6200..1b5ee58be 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index f6aa03bd1..6cfd84c1a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -64,8 +64,8 @@ import static java.lang.String.format; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIME_FORMAT; import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL; import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; import static org.apache.flink.util.Preconditions.checkArgument; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index 9c50fc66d..607d4acc8 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -52,8 +52,8 @@ import static java.lang.String.format; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIME_FORMAT; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java index b5d7ec825..e2c4f5702 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.json; import org.apache.flink.annotation.Internal; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; @@ -58,11 +59,11 @@ import java.util.Map; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.SQL_TIME_FORMAT; /** Tool class used to convert from {@link JsonNode} to {@link RowData}. * */ @Internal diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java index c30690a1c..36caaf9df 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.json; import org.apache.flink.annotation.Internal; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalData; @@ -47,11 +48,11 @@ import java.util.Arrays; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.json.TimeFormats.SQL_TIME_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; +import static org.apache.flink.formats.common.TimeFormats.SQL_TIME_FORMAT; /** Tool class used to convert from {@link RowData} to {@link JsonNode}. * */ @Internal diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java deleted file mode 100644 index c81aa20b6..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimeFormats.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import java.time.format.DateTimeFormatter; -import java.time.format.DateTimeFormatterBuilder; -import java.time.temporal.ChronoField; - -/** - * Time formats and timestamp formats respecting the RFC3339 specification, ISO-8601 specification - * and SQL specification. - */ -class TimeFormats { - - /** Formatter for RFC 3339-compliant string representation of a time value. */ - static final DateTimeFormatter RFC3339_TIME_FORMAT = - new DateTimeFormatterBuilder() - .appendPattern("HH:mm:ss") - .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) - .appendPattern("'Z'") - .toFormatter(); - - /** - * Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC - * timezone). - */ - static final DateTimeFormatter RFC3339_TIMESTAMP_FORMAT = - new DateTimeFormatterBuilder() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral('T') - .append(RFC3339_TIME_FORMAT) - .toFormatter(); - - /** Formatter for ISO8601 string representation of a timestamp value (without UTC timezone). */ - static final DateTimeFormatter ISO8601_TIMESTAMP_FORMAT = DateTimeFormatter.ISO_LOCAL_DATE_TIME; - - /** Formatter for ISO8601 string representation of a timestamp value (with UTC timezone). */ - static final DateTimeFormatter ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT = - new DateTimeFormatterBuilder() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral('T') - .append(DateTimeFormatter.ISO_LOCAL_TIME) - .appendPattern("'Z'") - .toFormatter(); - - /** Formatter for SQL string representation of a time value. */ - static final DateTimeFormatter SQL_TIME_FORMAT = - new DateTimeFormatterBuilder() - .appendPattern("HH:mm:ss") - .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) - .toFormatter(); - - /** Formatter for SQL string representation of a timestamp value (without UTC timezone). */ - static final DateTimeFormatter SQL_TIMESTAMP_FORMAT = - new DateTimeFormatterBuilder() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral(' ') - .append(SQL_TIME_FORMAT) - .toFormatter(); - - /** Formatter for SQL string representation of a timestamp value (with UTC timezone). */ - static final DateTimeFormatter SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT = - new DateTimeFormatterBuilder() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral(' ') - .append(SQL_TIME_FORMAT) - .appendPattern("'Z'") - .toFormatter(); - - private TimeFormats() {} -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java deleted file mode 100644 index 8dacfc123..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/TimestampFormat.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.Internal; - -/** Timestamp format Enums. */ -@Internal -public enum TimestampFormat { - /** - * Options to specify TIMESTAMP/TIMESTAMP_WITH_LOCAL_ZONE format. It will parse TIMESTAMP in - * "yyyy-MM-dd HH:mm:ss.s{precision}" format, TIMESTAMP_WITH_LOCAL_TIMEZONE in "yyyy-MM-dd - * HH:mm:ss.s{precision}'Z'" and output in the same format. - */ - SQL, - - /** - * Options to specify TIMESTAMP/TIMESTAMP_WITH_LOCAL_ZONE format. It will parse TIMESTAMP in - * "yyyy-MM-ddTHH:mm:ss.s{precision}" format, TIMESTAMP_WITH_LOCAL_TIMEZONE in - * "yyyy-MM-ddTHH:mm:ss.s{precision}'Z'" and output in the same format. - */ - ISO_8601 -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java index 791dd59d9..f958c8d38 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.canal.CanalJsonDeserializationSchema.MetadataConverter; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.connector.ChangelogMode; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index acdaecf58..d1bd3a37f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -21,8 +21,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.formats.json.canal.CanalJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.ArrayData; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java index e43dd1c50..a749808de 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java @@ -22,8 +22,8 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java index e80de2bff..b15fb4a26 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java @@ -19,9 +19,9 @@ package org.apache.flink.formats.json.canal; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.JsonRowDataSerializationSchema; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java index 22de3c9df..5db2f64d7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.debezium.DebeziumJsonDeserializationSchema.MetadataConverter; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.connector.ChangelogMode; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java index 628e05342..d12b7cc34 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java @@ -21,8 +21,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.formats.json.debezium.DebeziumJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.GenericRowData; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java index bed3dd645..dddb52be6 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -22,8 +22,8 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java index 9bde53db5..64e0105da 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java @@ -19,9 +19,9 @@ package org.apache.flink.formats.json.debezium; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.JsonRowDataSerializationSchema; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java index 073870895..8301210ca 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java @@ -20,8 +20,8 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java index c8d47f0a6..e03aa7397 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java @@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java index 66e534b8c..27b9da48e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java @@ -19,9 +19,9 @@ package org.apache.flink.formats.json.maxwell; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; import org.apache.flink.formats.json.JsonRowDataSerializationSchema; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index 036fdbebe..aae25c296 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 84a3c707f..6c748ce3a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.json; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.data.GenericMapData; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 8b477367b..49682940a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -20,8 +20,8 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index 3a193841a..73b34d970 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -18,8 +18,8 @@ package org.apache.flink.formats.json.canal; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.formats.json.canal.CanalJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.RowData; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index bdd014a87..466365ffa 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -20,8 +20,8 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java index db98d89d8..a628dd69c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -18,8 +18,8 @@ package org.apache.flink.formats.json.debezium; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.formats.json.debezium.DebeziumJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.RowData; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java index f2fc7d65f..7fb1ba97d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java @@ -20,8 +20,8 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java index 8ae814312..f9f17eb94 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java @@ -18,8 +18,8 @@ package org.apache.flink.formats.json.maxwell; +import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.formats.json.TimestampFormat; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; From 0fbe0cad0f206fbfcd5341dce52a8e18fde8354a Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Sat, 27 Mar 2021 16:59:23 +0800 Subject: [PATCH 138/322] [hotfix] Package format-common to flink-csv/flink-json (#15394) --- .../flink-json-debezium/pom.xml | 38 +++++++++++++++++-- 1 file changed, 35 insertions(+), 3 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index a5217b742..87d48e833 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -108,6 +108,31 @@ under the License. + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + org.apache.flink:flink-format-common + + + + + + + + + @@ -121,15 +146,21 @@ under the License. org.apache.maven.plugins - maven-jar-plugin + maven-shade-plugin package - jar + shade - sql-jar + + + org.apache.flink:flink-format-common + + + true + sql-jar @@ -138,4 +169,5 @@ under the License. + From 07ba2faf63a0650d21e57ef2bc3a20f0b10e6146 Mon Sep 17 00:00:00 2001 From: GuoWei Ma Date: Sun, 18 Apr 2021 20:19:52 +0800 Subject: [PATCH 139/322] Update version to 1.14-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 14b5ff40f..2d36f9e03 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.13-SNAPSHOT + 1.14-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index fb1fdeb85..bbb9f067c 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.13-SNAPSHOT + 1.14-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 87d48e833..454902705 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.13-SNAPSHOT + 1.14-SNAPSHOT .. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index 3e718b7e7..0f0e64eff 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.13-SNAPSHOT + 1.14-SNAPSHOT .. From 40d8e222818b87686ee05cc0929cb052cccbdcb4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8E=AB=E8=BE=9E?= Date: Thu, 3 Jun 2021 15:26:39 +0800 Subject: [PATCH 140/322] [FLINK-13538][formats] Figure out wrong field name when serializer/deserializer throw exceptions while doing serializing/deserializing for json and csv format. This closes #16068 --- .../json/JsonRowDataSerializationSchema.java | 2 +- .../formats/json/JsonToRowDataConverters.java | 9 +++- .../formats/json/RowDataToJsonConverters.java | 11 ++++- .../json/JsonRowDataSerDeSchemaTest.java | 42 ++++++++++++++++++- 4 files changed, 58 insertions(+), 6 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index 1b5ee58be..1008e6180 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -96,7 +96,7 @@ public byte[] serialize(RowData row) { runtimeConverter.convert(mapper, node, row); return mapper.writeValueAsBytes(node); } catch (Throwable t) { - throw new RuntimeException("Could not serialize row '" + row + "'. ", t); + throw new RuntimeException(String.format("Could not serialize row '%s'.", row), t); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java index e2c4f5702..f56897d20 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java @@ -347,8 +347,13 @@ public JsonToRowDataConverter createRowConverter(RowType rowType) { for (int i = 0; i < arity; i++) { String fieldName = fieldNames[i]; JsonNode field = node.get(fieldName); - Object convertedField = convertField(fieldConverters[i], fieldName, field); - row.setField(i, convertedField); + try { + Object convertedField = convertField(fieldConverters[i], fieldName, field); + row.setField(i, convertedField); + } catch (Throwable t) { + throw new JsonParseException( + String.format("Fail to deserialize at field: %s.", fieldName), t); + } } return row; }; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java index 36caaf9df..6be9d9e28 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java @@ -330,8 +330,15 @@ private RowDataToJsonConverter createRowConverter(RowType type) { RowData row = (RowData) value; for (int i = 0; i < fieldCount; i++) { String fieldName = fieldNames[i]; - Object field = fieldGetters[i].getFieldOrNull(row); - node.set(fieldName, fieldConverters[i].convert(mapper, node.get(fieldName), field)); + try { + Object field = fieldGetters[i].getFieldOrNull(row); + node.set( + fieldName, + fieldConverters[i].convert(mapper, node.get(fieldName), field)); + } catch (Throwable t) { + throw new RuntimeException( + String.format("Fail to serialize at field: %s.", fieldName), t); + } } return node; }; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 6c748ce3a..518b067a9 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.json; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.data.GenericMapData; import org.apache.flink.table.data.GenericRowData; @@ -72,6 +73,7 @@ import static org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE; import static org.apache.flink.table.api.DataTypes.TINYINT; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; @@ -562,7 +564,7 @@ public void testSerializationMapNullKey() throws Exception { serializationSchema1.serialize(rowData); Assert.fail("expecting exception message: " + errorMessage1); } catch (Throwable t) { - assertEquals(errorMessage1, t.getCause().getMessage()); + assertThat(t, FlinkMatchers.containsMessage(errorMessage1)); } // mapNullKey Mode is drop @@ -629,6 +631,44 @@ public void testJsonParse() throws Exception { } } + @Test + public void testSerializationWithTypesMismatch() { + RowType rowType = (RowType) ROW(FIELD("f0", INT()), FIELD("f1", STRING())).getLogicalType(); + GenericRowData genericRowData = new GenericRowData(2); + genericRowData.setField(0, 1); + genericRowData.setField(1, 1); + JsonRowDataSerializationSchema serializationSchema = + new JsonRowDataSerializationSchema( + rowType, + TimestampFormat.SQL, + JsonOptions.MapNullKeyMode.FAIL, + "null", + true); + String errorMessage = "Fail to serialize at field: f1."; + try { + serializationSchema.serialize(genericRowData); + fail("expecting exception message: " + errorMessage); + } catch (Throwable t) { + assertThat(t, FlinkMatchers.containsMessage(errorMessage)); + } + } + + @Test + public void testDeserializationWithTypesMismatch() { + RowType rowType = (RowType) ROW(FIELD("f0", STRING()), FIELD("f1", INT())).getLogicalType(); + String json = "{\"f0\":\"abc\", \"f1\": \"abc\"}"; + JsonRowDataDeserializationSchema deserializationSchema = + new JsonRowDataDeserializationSchema( + rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); + String errorMessage = "Fail to deserialize at field: f1."; + try { + deserializationSchema.deserialize(json.getBytes()); + fail("expecting exception message: " + errorMessage); + } catch (Throwable t) { + assertThat(t, FlinkMatchers.containsMessage(errorMessage)); + } + } + private void testIgnoreParseErrors(TestSpec spec) throws Exception { // the parsing field should be null and no exception is thrown JsonRowDataDeserializationSchema ignoreErrorsSchema = From 1b4cc852502a2ca11f092a8e5c9d2eddd4466b7d Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Tue, 1 Jun 2021 16:32:22 +0800 Subject: [PATCH 141/322] [FLINK-20926][maxwell][json] Allow to read metadata for maxwell-json format This closes #16040 --- .../maxwell/MaxwellJsonDecodingFormat.java | 200 ++++++++++++++++++ .../MaxwellJsonDeserializationSchema.java | 173 +++++++++++---- .../maxwell/MaxwellJsonFormatFactory.java | 26 +-- .../maxwell/MaxwellJsonFormatFactoryTest.java | 26 ++- .../json/maxwell/MaxwellJsonSerDerTest.java | 66 +++++- .../src/test/resources/maxwell-data.txt | 40 ++-- 6 files changed, 422 insertions(+), 109 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java new file mode 100644 index 000000000..d0403863e --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.maxwell; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.maxwell.MaxwellJsonDeserializationSchema.MetadataConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.types.RowKind; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** {@link DecodingFormat} for Maxwell using JSON encoding. */ +public class MaxwellJsonDecodingFormat implements DecodingFormat> { + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + /** The requested metadata keys. */ + private List metadataKeys; + + private final boolean ignoreParseErrors; + + private final TimestampFormat timestampFormat; + + public MaxwellJsonDecodingFormat(boolean ignoreParseErrors, TimestampFormat timestampFormat) { + this.ignoreParseErrors = ignoreParseErrors; + this.timestampFormat = timestampFormat; + this.metadataKeys = Collections.emptyList(); + } + + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType physicalDataType) { + final List readableMetadata = + metadataKeys.stream() + .map( + k -> + Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow( + () -> + new IllegalStateException( + String.format( + "Could not find the requested metadata key: %s", + k)))) + .collect(Collectors.toList()); + final List metadataFields = + readableMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList()); + final DataType producedDataType = + DataTypeUtils.appendRowFields(physicalDataType, metadataFields); + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + return new MaxwellJsonDeserializationSchema( + physicalDataType, + readableMetadata, + producedTypeInfo, + ignoreParseErrors, + timestampFormat); + } + + @Override + public Map listReadableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + Stream.of(ReadableMetadata.values()) + .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyReadableMetadata(List metadataKeys) { + this.metadataKeys = metadataKeys; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + + /** List of metadata that can be read with this format. */ + enum ReadableMetadata { + DATABASE( + "database", + DataTypes.STRING().nullable(), + DataTypes.FIELD("database", DataTypes.STRING()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getString(pos); + } + }), + + TABLE( + "table", + DataTypes.STRING().nullable(), + DataTypes.FIELD("table", DataTypes.STRING()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getString(pos); + } + }), + + PRIMARY_KEY_COLUMNS( + "primary-key-columns", + DataTypes.ARRAY(DataTypes.STRING()).nullable(), + DataTypes.FIELD("primary_key_columns", DataTypes.ARRAY(DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getArray(pos); + } + }), + + INGESTION_TIMESTAMP( + "ingestion-timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + DataTypes.FIELD("ts", DataTypes.BIGINT()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + // the `ts` field of Maxwell JSON is assigned with `timestampSeconds`. + return TimestampData.fromEpochMillis(row.getLong(pos) * 1000); + } + }); + + final String key; + + final DataType dataType; + + final DataTypes.Field requiredJsonField; + + final MetadataConverter converter; + + ReadableMetadata( + String key, + DataType dataType, + DataTypes.Field requiredJsonField, + MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.requiredJsonField = requiredJsonField; + this.converter = converter; + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java index 8301210ca..863620fd9 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java @@ -22,19 +22,25 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; +import org.apache.flink.formats.json.maxwell.MaxwellJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; import org.apache.flink.types.RowKind; import org.apache.flink.util.Collector; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; + import java.io.IOException; +import java.io.Serializable; +import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; import static java.lang.String.format; -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** * Deserialization schema from Maxwell JSON to Flink Table/SQL internal data structure {@link @@ -48,8 +54,9 @@ * @see Maxwell */ public class MaxwellJsonDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; + private static final long serialVersionUID = 2L; + private static final String FIELD_OLD = "old"; private static final String OP_INSERT = "insert"; private static final String OP_UPDATE = "update"; private static final String OP_DELETE = "delete"; @@ -57,33 +64,49 @@ public class MaxwellJsonDeserializationSchema implements DeserializationSchema resultTypeInfo; + /** Flag that indicates that an additional projection is required for metadata. */ + private final boolean hasMetadata; + + /** Metadata to be extracted for every record. */ + private final MetadataConverter[] metadataConverters; + + /** {@link TypeInformation} of the produced {@link RowData} (physical + meta data). */ + private final TypeInformation producedTypeInfo; /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ private final boolean ignoreParseErrors; - /** Number of fields. */ + /** Names of physical fields. */ + private final List fieldNames; + + /** Number of physical fields. */ private final int fieldCount; public MaxwellJsonDeserializationSchema( - RowType rowType, - TypeInformation resultTypeInfo, + DataType physicalDataType, + List requestedMetadata, + TypeInformation producedTypeInfo, boolean ignoreParseErrors, - TimestampFormat timestampFormatOption) { - this.resultTypeInfo = resultTypeInfo; - this.ignoreParseErrors = ignoreParseErrors; - this.fieldCount = rowType.getFieldCount(); + TimestampFormat timestampFormat) { + final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata); this.jsonDeserializer = new JsonRowDataDeserializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - // the result type is never used, so it's fine to pass in Canal's result - // type - resultTypeInfo, - false, // ignoreParseErrors already contains the functionality of + jsonRowType, + // the result type is never used, so it's fine to pass in the produced type + // info + producedTypeInfo, + // ignoreParseErrors already contains the functionality of // failOnMissingField + false, ignoreParseErrors, - timestampFormatOption); + timestampFormat); + this.hasMetadata = requestedMetadata.size() > 0; + this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata); + this.producedTypeInfo = producedTypeInfo; + this.ignoreParseErrors = ignoreParseErrors; + final RowType physicalRowType = ((RowType) physicalDataType.getLogicalType()); + this.fieldNames = physicalRowType.getFieldNames(); + this.fieldCount = physicalRowType.getFieldCount(); } @Override @@ -94,22 +117,27 @@ public RowData deserialize(byte[] message) throws IOException { @Override public void deserialize(byte[] message, Collector out) throws IOException { + if (message == null || message.length == 0) { + return; + } try { - RowData row = jsonDeserializer.deserialize(message); + final JsonNode root = jsonDeserializer.deserializeToJsonNode(message); + final GenericRowData row = (GenericRowData) jsonDeserializer.convertToRowData(root); String type = row.getString(2).toString(); // "type" field if (OP_INSERT.equals(type)) { // "data" field is a row, contains inserted rows - RowData insert = row.getRow(0, fieldCount); + GenericRowData insert = (GenericRowData) row.getRow(0, fieldCount); insert.setRowKind(RowKind.INSERT); - out.collect(insert); + emitRow(row, insert, out); } else if (OP_UPDATE.equals(type)) { // "data" field is a row, contains new rows - // "old" field is an array of row, contains old values + // "old" field is a row, contains old values // the underlying JSON deserialization schema always produce GenericRowData. GenericRowData after = (GenericRowData) row.getRow(0, fieldCount); // "data" field GenericRowData before = (GenericRowData) row.getRow(1, fieldCount); // "old" field + final JsonNode oldField = root.get(FIELD_OLD); for (int f = 0; f < fieldCount; f++) { - if (before.isNullAt(f)) { + if (before.isNullAt(f) && oldField.findValue(fieldNames.get(f)) == null) { // not null fields in "old" (before) means the fields are changed // null/empty fields in "old" (before) means the fields are not changed // so we just copy the not changed fields into before @@ -118,14 +146,13 @@ public void deserialize(byte[] message, Collector out) throws IOExcepti } before.setRowKind(RowKind.UPDATE_BEFORE); after.setRowKind(RowKind.UPDATE_AFTER); - out.collect(before); - out.collect(after); + emitRow(row, before, out); + emitRow(row, after, out); } else if (OP_DELETE.equals(type)) { // "data" field is a row, contains deleted rows - RowData delete = row.getRow(0, fieldCount); + GenericRowData delete = (GenericRowData) row.getRow(0, fieldCount); delete.setRowKind(RowKind.DELETE); - out.collect(delete); - + emitRow(row, delete, out); } else { if (!ignoreParseErrors) { throw new IOException( @@ -143,6 +170,26 @@ public void deserialize(byte[] message, Collector out) throws IOExcepti } } + private void emitRow( + GenericRowData rootRow, GenericRowData physicalRow, Collector out) { + // shortcut in case no output projection is required + if (!hasMetadata) { + out.collect(physicalRow); + return; + } + final int metadataArity = metadataConverters.length; + final GenericRowData producedRow = + new GenericRowData(physicalRow.getRowKind(), fieldCount + metadataArity); + for (int physicalPos = 0; physicalPos < fieldCount; physicalPos++) { + producedRow.setField(physicalPos, physicalRow.getField(physicalPos)); + } + for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { + producedRow.setField( + fieldCount + metadataPos, metadataConverters[metadataPos].convert(rootRow)); + } + out.collect(producedRow); + } + @Override public boolean isEndOfStream(RowData nextElement) { return false; @@ -150,7 +197,7 @@ public boolean isEndOfStream(RowData nextElement) { @Override public TypeInformation getProducedType() { - return resultTypeInfo; + return producedTypeInfo; } @Override @@ -162,25 +209,69 @@ public boolean equals(Object o) { return false; } MaxwellJsonDeserializationSchema that = (MaxwellJsonDeserializationSchema) o; - return ignoreParseErrors == that.ignoreParseErrors - && fieldCount == that.fieldCount - && Objects.equals(jsonDeserializer, that.jsonDeserializer) - && Objects.equals(resultTypeInfo, that.resultTypeInfo); + return Objects.equals(jsonDeserializer, that.jsonDeserializer) + && hasMetadata == that.hasMetadata + && Objects.equals(producedTypeInfo, that.producedTypeInfo) + && ignoreParseErrors == that.ignoreParseErrors + && fieldCount == that.fieldCount; } @Override public int hashCode() { - return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors, fieldCount); + return Objects.hash( + jsonDeserializer, hasMetadata, producedTypeInfo, ignoreParseErrors, fieldCount); } - private RowType createJsonRowType(DataType databaseSchema) { - // Maxwell JSON contains other information, e.g. "database", "ts" - // but we don't need them - return (RowType) + // -------------------------------------------------------------------------------------------- + + private static RowType createJsonRowType( + DataType physicalDataType, List readableMetadata) { + DataType root = DataTypes.ROW( - DataTypes.FIELD("data", databaseSchema), - DataTypes.FIELD("old", databaseSchema), - DataTypes.FIELD("type", DataTypes.STRING())) - .getLogicalType(); + DataTypes.FIELD("data", physicalDataType), + DataTypes.FIELD("old", physicalDataType), + DataTypes.FIELD("type", DataTypes.STRING())); + // append fields that are required for reading metadata in the root + final List rootMetadataFields = + readableMetadata.stream() + .map(m -> m.requiredJsonField) + .distinct() + .collect(Collectors.toList()); + return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType(); + } + + private static MetadataConverter[] createMetadataConverters( + RowType jsonRowType, List requestedMetadata) { + return requestedMetadata.stream() + .map(m -> convert(jsonRowType, m)) + .toArray(MetadataConverter[]::new); + } + + private static MetadataConverter convert(RowType jsonRowType, ReadableMetadata metadata) { + final int pos = jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); + return new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData root, int unused) { + return metadata.converter.convert(root, pos); + } + }; + } + + // -------------------------------------------------------------------------------------------- + + /** + * Converter that extracts a metadata field from the row that comes out of the JSON schema and + * converts it to the desired data type. + */ + interface MetadataConverter extends Serializable { + + // Method for top-level access. + default Object convert(GenericRowData row) { + return convert(row, -1); + } + + Object convert(GenericRowData row, int pos); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java index e03aa7397..bac2b1e91 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.formats.common.TimestampFormat; @@ -29,7 +28,6 @@ import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DeserializationFormatFactory; import org.apache.flink.table.factories.DynamicTableFactory; @@ -67,29 +65,9 @@ public DecodingFormat> createDecodingFormat( validateDecodingFormatOptions(formatOptions); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - TimestampFormat timestampFormatOption = JsonOptions.getTimestampFormat(formatOptions); + final TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - return new DecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType producedDataType) { - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation rowDataTypeInfo = - context.createTypeInformation(producedDataType); - return new MaxwellJsonDeserializationSchema( - rowType, rowDataTypeInfo, ignoreParseErrors, timestampFormatOption); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - }; + return new MaxwellJsonDecodingFormat(ignoreParseErrors, timestampFormat); } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java index 7fb1ba97d..275ee6455 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java @@ -22,10 +22,7 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; @@ -33,18 +30,21 @@ import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; +import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; +import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.junit.Assert.assertEquals; @@ -53,24 +53,22 @@ public class MaxwellJsonFormatFactoryTest extends TestLogger { @Rule public ExpectedException thrown = ExpectedException.none(); - private static final ResolvedSchema SCHEMA = - ResolvedSchema.of( - Column.physical("a", DataTypes.STRING()), - Column.physical("b", DataTypes.INT()), - Column.physical("c", DataTypes.BOOLEAN())); - - private static final RowType ROW_TYPE = - (RowType) SCHEMA.toPhysicalRowDataType().getLogicalType(); + private static final InternalTypeInfo ROW_TYPE_INFO = + InternalTypeInfo.of(PHYSICAL_TYPE); @Test public void testSeDeSchema() { final MaxwellJsonDeserializationSchema expectedDeser = new MaxwellJsonDeserializationSchema( - ROW_TYPE, InternalTypeInfo.of(ROW_TYPE), true, TimestampFormat.ISO_8601); + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + ROW_TYPE_INFO, + true, + TimestampFormat.ISO_8601); final MaxwellJsonSerializationSchema expectedSer = new MaxwellJsonSerializationSchema( - ROW_TYPE, + PHYSICAL_TYPE, TimestampFormat.ISO_8601, JsonOptions.MapNullKeyMode.LITERAL, "null", diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java index f9f17eb94..009eba15a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java @@ -20,9 +20,13 @@ import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.maxwell.MaxwellJsonDecodingFormat.ReadableMetadata; +import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; import org.apache.flink.util.Collector; import org.junit.Test; @@ -35,7 +39,9 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.function.Consumer; import java.util.stream.Collectors; import static org.apache.flink.table.api.DataTypes.FIELD; @@ -43,27 +49,67 @@ import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; /** * Tests for {@link MaxwellJsonSerializationSchema} and {@link MaxwellJsonDeserializationSchema}. */ public class MaxwellJsonSerDerTest { - private static final RowType SCHEMA = - (RowType) - ROW( - FIELD("id", INT().notNull()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", FLOAT())) - .getLogicalType(); + + private static final DataType PHYSICAL_DATA_TYPE = + ROW( + FIELD("id", INT().notNull()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", FLOAT())); + + @Test + public void testDeserializationWithMetadata() throws Exception { + // we only read the first line for keeping the test simple + final String firstLine = readLines("maxwell-data.txt").get(0); + final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); + final DataType producedDataType = + DataTypeUtils.appendRowFields( + PHYSICAL_DATA_TYPE, + requestedMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList())); + final MaxwellJsonDeserializationSchema deserializationSchema = + new MaxwellJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + requestedMetadata, + InternalTypeInfo.of(producedDataType.getLogicalType()), + false, + TimestampFormat.ISO_8601); + final SimpleCollector collector = new SimpleCollector(); + deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); + assertEquals(1, collector.list.size()); + Consumer consumer = + row -> { + assertThat(row.getInt(0), equalTo(101)); + assertThat(row.getString(1).toString(), equalTo("scooter")); + assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); + assertThat(row.getFloat(3), equalTo(3.14f)); + assertThat(row.getString(4).toString(), equalTo("test")); + assertThat(row.getString(5).toString(), equalTo("product")); + assertThat(row.getArray(6).getString(0).toString(), equalTo("id")); + assertThat(row.getTimestamp(7, 3).getMillisecond(), equalTo(1596684883000L)); + }; + consumer.accept(collector.list.get(0)); + } @Test public void testSerializationDeserialization() throws Exception { List lines = readLines("maxwell-data.txt"); MaxwellJsonDeserializationSchema deserializationSchema = new MaxwellJsonDeserializationSchema( - SCHEMA, InternalTypeInfo.of(SCHEMA), false, TimestampFormat.ISO_8601); + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), + false, + TimestampFormat.ISO_8601); SimpleCollector collector = new SimpleCollector(); for (String line : lines) { @@ -135,7 +181,7 @@ public void testSerializationDeserialization() throws Exception { MaxwellJsonSerializationSchema serializationSchema = new MaxwellJsonSerializationSchema( - SCHEMA, + (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), TimestampFormat.SQL, JsonOptions.MapNullKeyMode.LITERAL, "null", diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt index ecba573aa..2d33ff7d7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt @@ -1,20 +1,20 @@ -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":0,"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14}} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":1,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1}} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":2,"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8}} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":3,"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75}} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":4,"data":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875}} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":5,"data":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0}} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":6,"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3}} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":7,"data":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1}} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"commit":true,"data":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2}} -{"database":"test","table":"product","type":"update","ts":1596684893,"xid":7152,"commit":true,"data":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"old":{"description":"16oz carpenter's hammer"}} -{"database":"test","table":"product","type":"update","ts":1596684897,"xid":7169,"commit":true,"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"old":{"weight":5.3}} -{"database":"test","table":"product","type":"insert","ts":1596684900,"xid":7186,"commit":true,"data":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2}} -{"database":"test","table":"product","type":"insert","ts":1596684904,"xid":7201,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18}} -{"database":"test","table":"product","type":"update","ts":1596684906,"xid":7216,"commit":true,"data":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"old":{"description":"water resistent white wind breaker","weight":0.2}} -{"database":"test","table":"product","type":"update","ts":1596684912,"xid":7235,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"old":{"weight":5.18}} -{"database":"test","table":"product","type":"delete","ts":1596684914,"xid":7250,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17}} -{"database":"test","table":"product","type":"update","ts":1596684928,"xid":7291,"xoffset":0,"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":5.17},"old":{"weight":3.14}} -{"database":"test","table":"product","type":"update","ts":1596684928,"xid":7291,"commit":true,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":5.17},"old":{"weight":8.1}} -{"database":"test","table":"product","type":"delete","ts":1596684938,"xid":7322,"xoffset":0,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":5.17}} -{"database":"test","table":"product","type":"delete","ts":1596684938,"xid":7322,"commit":true,"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8}} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":0,"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":1,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":2,"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":3,"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":4,"data":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":5,"data":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":6,"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":7,"data":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"commit":true,"data":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"update","ts":1596684893,"xid":7152,"commit":true,"data":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"old":{"description":"16oz carpenter's hammer"},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"update","ts":1596684897,"xid":7169,"commit":true,"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"old":{"weight":5.3},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"insert","ts":1596684900,"xid":7186,"commit":true,"data":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"insert","ts":1596684904,"xid":7201,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"update","ts":1596684906,"xid":7216,"commit":true,"data":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"old":{"description":"water resistent white wind breaker","weight":0.2},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"update","ts":1596684912,"xid":7235,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"old":{"weight":5.18},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"delete","ts":1596684914,"xid":7250,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"update","ts":1596684928,"xid":7291,"xoffset":0,"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":5.17},"old":{"weight":3.14},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"update","ts":1596684928,"xid":7291,"commit":true,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":5.17},"old":{"weight":8.1},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"delete","ts":1596684938,"xid":7322,"xoffset":0,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":5.17},"primary_key_columns": ["id"]} +{"database":"test","table":"product","type":"delete","ts":1596684938,"xid":7322,"commit":true,"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"primary_key_columns": ["id"]} From 329193f8e4ac63465521684464bf2c7561597230 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 22 Jun 2021 09:39:40 +0200 Subject: [PATCH 142/322] [FLINK-21229][avro-confluent-registry] Shorten RegistryAvroOptions --- .../connectors/table/formats/avro-confluent.md | 18 +++++++++--------- .../docs/connectors/table/formats/debezium.md | 4 ++-- .../connectors/table/formats/avro-confluent.md | 18 +++++++++--------- .../docs/connectors/table/formats/debezium.md | 4 ++-- .../confluent/RegistryAvroFormatFactory.java | 16 ++++++++-------- .../confluent/RegistryAvroOptions.java | 10 ++++++---- .../debezium/DebeziumAvroFormatFactory.java | 16 ++++++++-------- .../RegistryAvroFormatFactoryTest.java | 9 ++++----- .../DebeziumAvroFormatFactoryTest.java | 4 ++-- 9 files changed, 50 insertions(+), 49 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md index 845ca1f90..a2faaec23 100644 --- a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md @@ -35,7 +35,7 @@ Avro Schema Registry (``avro-confluent``) 格式能让你读取被 ``io.confluen 当以这种格式读取(反序列化)记录时,将根据记录中编码的 schema 版本 id 从配置的 Confluent Schema Registry 中获取 Avro writer schema ,而从 table schema 中推断出 reader schema。 -当以这种格式写入(序列化)记录时,Avro schema 是从 table schema 中推断出来的,并会用来检索要与数据一起编码的 schema id。我们会在配置的 Confluent Schema Registry 中配置的 [subject](https://docs.confluent.io/current/schema-registry/index.html#schemas-subjects-and-topics) 下,检索 schema id。subject 通过 `avro-confluent.schema-registry.subject` 参数来制定。 +当以这种格式写入(序列化)记录时,Avro schema 是从 table schema 中推断出来的,并会用来检索要与数据一起编码的 schema id。我们会在配置的 Confluent Schema Registry 中配置的 [subject](https://docs.confluent.io/current/schema-registry/index.html#schemas-subjects-and-topics) 下,检索 schema id。subject 通过 `avro-confluent.subject` 参数来制定。 Avro Schema Registry 格式只能与 [Apache Kafka SQL 连接器]({{< ref "docs/connectors/table/kafka" >}})或 [Upsert Kafka SQL 连接器]({{< ref "docs/connectors/table/upsert-kafka" >}})一起使用。 @@ -76,7 +76,7 @@ CREATE TABLE user_created ( 'key.fields' = 'the_kafka_key', 'value.format' = 'avro-confluent', - 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.avro-confluent.url' = 'http://localhost:8082', 'value.fields-include' = 'EXCEPT_KEY' ) ``` @@ -117,7 +117,7 @@ CREATE TABLE user_created ( -- 注意:由于哈希分区,在 Kafka key 的上下文中,schema 升级几乎从不向后也不向前兼容。 'key.format' = 'avro-confluent', - 'key.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'key.avro-confluent.url' = 'http://localhost:8082', 'key.fields' = 'kafka_key_id', -- 在本例中,我们希望 Kafka 的 key 和 value 的 Avro 类型都包含 'id' 字段 @@ -125,12 +125,12 @@ CREATE TABLE user_created ( 'key.fields-prefix' = 'kafka_key_', 'value.format' = 'avro-confluent', - 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.avro-confluent.url' = 'http://localhost:8082', 'value.fields-include' = 'EXCEPT_KEY', -- 自 Flink 1.13 起,subjects 具有一个默认值, 但是可以被覆盖: - 'key.avro-confluent.schema-registry.subject' = 'user_events_example2-key2', - 'value.avro-confluent.schema-registry.subject' = 'user_events_example2-value2' + 'key.avro-confluent.subject' = 'user_events_example2-key2', + 'value.avro-confluent.subject' = 'user_events_example2-value2' ) ``` @@ -166,7 +166,7 @@ CREATE TABLE user_created ( 'key.fields-prefix' = 'kafka_key_', 'value.format' = 'avro-confluent', - 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.avro-confluent.url' = 'http://localhost:8082', 'value.fields-include' = 'EXCEPT_KEY' ) ``` @@ -195,14 +195,14 @@ Format 参数 指定要使用的格式,这里应该是 'avro-confluent'。 -

avro-confluent.schema-registry.url
+
avro-confluent.url
必选 (none) String 用于获取/注册 schemas 的 Confluent Schema Registry 的URL。 -
avro-confluent.schema-registry.subject
+
avro-confluent.subject
可选 (none) String diff --git a/docs/content.zh/docs/connectors/table/formats/debezium.md b/docs/content.zh/docs/connectors/table/formats/debezium.md index d0283934b..e5886e820 100644 --- a/docs/content.zh/docs/connectors/table/formats/debezium.md +++ b/docs/content.zh/docs/connectors/table/formats/debezium.md @@ -265,14 +265,14 @@ Flink 提供了 `debezium-avro-confluent` 和 `debezium-json` 两种 format 来 指定要使用的格式,此处应为 'debezium-avro-confluent'。 -
debezium-avro-confluent.schema-registry.url
+
debezium-avro-confluent.url
必选 (none) String 用于获取/注册 schemas 的 Confluent Schema Registry 的 URL。 -
debezium-avro-confluent.schema-registry.subject
+
debezium-avro-confluent.subject
可选 (none) String diff --git a/docs/content/docs/connectors/table/formats/avro-confluent.md b/docs/content/docs/connectors/table/formats/avro-confluent.md index e784ce74c..8d6c54cf5 100644 --- a/docs/content/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content/docs/connectors/table/formats/avro-confluent.md @@ -33,7 +33,7 @@ The Avro Schema Registry (``avro-confluent``) format allows you to read records When reading (deserializing) a record with this format the Avro writer schema is fetched from the configured Confluent Schema Registry based on the schema version id encoded in the record while the reader schema is inferred from table schema. -When writing (serializing) a record with this format the Avro schema is inferred from the table schema and used to retrieve a schema id to be encoded with the data. The lookup is performed with in the configured Confluent Schema Registry under the [subject](https://docs.confluent.io/current/schema-registry/index.html#schemas-subjects-and-topics) given in `avro-confluent.schema-registry.subject`. +When writing (serializing) a record with this format the Avro schema is inferred from the table schema and used to retrieve a schema id to be encoded with the data. The lookup is performed with in the configured Confluent Schema Registry under the [subject](https://docs.confluent.io/current/schema-registry/index.html#schemas-subjects-and-topics) given in `avro-confluent.subject`. The Avro Schema Registry format can only be used in conjunction with the [Apache Kafka SQL connector]({{< ref "docs/connectors/table/kafka" >}}) or the [Upsert Kafka SQL Connector]({{< ref "docs/connectors/table/upsert-kafka" >}}). @@ -69,7 +69,7 @@ CREATE TABLE user_created ( 'key.fields' = 'the_kafka_key', 'value.format' = 'avro-confluent', - 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.avro-confluent.url' = 'http://localhost:8082', 'value.fields-include' = 'EXCEPT_KEY' ) ``` @@ -111,7 +111,7 @@ CREATE TABLE user_created ( -- Watch out: schema evolution in the context of a Kafka key is almost never backward nor -- forward compatible due to hash partitioning. 'key.format' = 'avro-confluent', - 'key.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'key.avro-confluent.url' = 'http://localhost:8082', 'key.fields' = 'kafka_key_id', -- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id' @@ -119,12 +119,12 @@ CREATE TABLE user_created ( 'key.fields-prefix' = 'kafka_key_', 'value.format' = 'avro-confluent', - 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.avro-confluent.url' = 'http://localhost:8082', 'value.fields-include' = 'EXCEPT_KEY', -- subjects have a default value since Flink 1.13, though can be overriden: - 'key.avro-confluent.schema-registry.subject' = 'user_events_example2-key2', - 'value.avro-confluent.schema-registry.subject' = 'user_events_example2-value2' + 'key.avro-confluent.subject' = 'user_events_example2-key2', + 'value.avro-confluent.subject' = 'user_events_example2-value2' ) ``` @@ -160,7 +160,7 @@ CREATE TABLE user_created ( 'key.fields-prefix' = 'kafka_key_', 'value.format' = 'avro-confluent', - 'value.avro-confluent.schema-registry.url' = 'http://localhost:8082', + 'value.avro-confluent.url' = 'http://localhost:8082', 'value.fields-include' = 'EXCEPT_KEY' ) ``` @@ -188,14 +188,14 @@ Format Options Specify what format to use, here should be 'avro-confluent'. -
avro-confluent.schema-registry.url
+
avro-confluent.url
required (none) String The URL of the Confluent Schema Registry to fetch/register schemas. -
avro-confluent.schema-registry.subject
+
avro-confluent.subject
optional (none) String diff --git a/docs/content/docs/connectors/table/formats/debezium.md b/docs/content/docs/connectors/table/formats/debezium.md index c64debd32..fa9c9ab01 100644 --- a/docs/content/docs/connectors/table/formats/debezium.md +++ b/docs/content/docs/connectors/table/formats/debezium.md @@ -259,14 +259,14 @@ Use format `debezium-avro-confluent` to interpret Debezium Avro messages and for Specify what format to use, here should be 'debezium-avro-confluent'. -
debezium-avro-confluent.schema-registry.url
+
debezium-avro-confluent.url
required (none) String The URL of the Confluent Schema Registry to fetch/register schemas. -
debezium-avro-confluent.schema-registry.subject
+
debezium-avro-confluent.subject
optional (none) String diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index cff1870c6..049fdbaa0 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -46,8 +46,8 @@ import java.util.Optional; import java.util.Set; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_SUBJECT; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_URL; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SUBJECT; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.URL; /** * Table format factory for providing configured instances of Schema Registry Avro to RowData {@link @@ -63,7 +63,7 @@ public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); + String schemaRegistryURL = formatOptions.get(URL); return new DecodingFormat>() { @Override public DeserializationSchema createRuntimeDecoder( @@ -90,13 +90,13 @@ public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); - Optional subject = formatOptions.getOptional(SCHEMA_REGISTRY_SUBJECT); + String schemaRegistryURL = formatOptions.get(URL); + Optional subject = formatOptions.getOptional(SUBJECT); if (!subject.isPresent()) { throw new ValidationException( String.format( "Option %s.%s is required for serialization", - IDENTIFIER, SCHEMA_REGISTRY_SUBJECT.key())); + IDENTIFIER, SUBJECT.key())); } return new EncodingFormat>() { @@ -128,14 +128,14 @@ public String factoryIdentifier() { @Override public Set> requiredOptions() { Set> options = new HashSet<>(); - options.add(SCHEMA_REGISTRY_URL); + options.add(URL); return options; } @Override public Set> optionalOptions() { Set> options = new HashSet<>(); - options.add(SCHEMA_REGISTRY_SUBJECT); + options.add(SUBJECT); return options; } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java index c94ebc19e..0d3c8f0f8 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java @@ -25,16 +25,18 @@ public class RegistryAvroOptions { private RegistryAvroOptions() {} - public static final ConfigOption SCHEMA_REGISTRY_URL = - ConfigOptions.key("schema-registry.url") + public static final ConfigOption URL = + ConfigOptions.key("url") .stringType() .noDefaultValue() + .withFallbackKeys("schema-registry.url") .withDescription("Required URL to connect to schema registry service"); - public static final ConfigOption SCHEMA_REGISTRY_SUBJECT = - ConfigOptions.key("schema-registry.subject") + public static final ConfigOption SUBJECT = + ConfigOptions.key("subject") .stringType() .noDefaultValue() + .withFallbackKeys("schema-registry.subject") .withDescription( "Subject name to write to the Schema Registry service, required for sink"); } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java index d67994e2a..f566bfbaf 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java @@ -42,8 +42,8 @@ import java.util.Optional; import java.util.Set; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_SUBJECT; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SCHEMA_REGISTRY_URL; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SUBJECT; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.URL; /** * Format factory for providing configured instances of Debezium Avro to RowData {@link @@ -59,7 +59,7 @@ public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); + String schemaRegistryURL = formatOptions.get(URL); return new DecodingFormat>() { @Override @@ -89,13 +89,13 @@ public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - String schemaRegistryURL = formatOptions.get(SCHEMA_REGISTRY_URL); - Optional subject = formatOptions.getOptional(SCHEMA_REGISTRY_SUBJECT); + String schemaRegistryURL = formatOptions.get(URL); + Optional subject = formatOptions.getOptional(SUBJECT); if (!subject.isPresent()) { throw new ValidationException( String.format( "Option '%s.%s' is required for serialization", - IDENTIFIER, SCHEMA_REGISTRY_SUBJECT.key())); + IDENTIFIER, SUBJECT.key())); } return new EncodingFormat>() { @@ -127,14 +127,14 @@ public String factoryIdentifier() { @Override public Set> requiredOptions() { Set> options = new HashSet<>(); - options.add(SCHEMA_REGISTRY_URL); + options.add(URL); return options; } @Override public Set> optionalOptions() { Set> options = new HashSet<>(); - options.add(SCHEMA_REGISTRY_SUBJECT); + options.add(SUBJECT); return options; } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java index 3171c82c6..d923bf824 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java @@ -118,11 +118,10 @@ public void testMissingSubjectForSink() { thrown.expect( containsCause( new ValidationException( - "Option avro-confluent.schema-registry.subject " - + "is required for serialization"))); + "Option avro-confluent.subject is required for serialization"))); final Map options = - getModifiedOptions(opts -> opts.remove("avro-confluent.schema-registry.subject")); + getModifiedOptions(opts -> opts.remove("avro-confluent.subject")); createTableSink(SCHEMA, options); } @@ -149,8 +148,8 @@ private Map getDefaultOptions() { options.put("buffer-size", "1000"); options.put("format", RegistryAvroFormatFactory.IDENTIFIER); - options.put("avro-confluent.schema-registry.subject", SUBJECT); - options.put("avro-confluent.schema-registry.url", REGISTRY_URL); + options.put("avro-confluent.subject", SUBJECT); + options.put("avro-confluent.url", REGISTRY_URL); return options; } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java index 132c925fb..5bb81820e 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java @@ -86,8 +86,8 @@ private Map getAllOptions() { options.put("buffer-size", "1000"); options.put("format", DebeziumAvroFormatFactory.IDENTIFIER); - options.put("debezium-avro-confluent.schema-registry.url", REGISTRY_URL); - options.put("debezium-avro-confluent.schema-registry.subject", SUBJECT); + options.put("debezium-avro-confluent.url", REGISTRY_URL); + options.put("debezium-avro-confluent.subject", SUBJECT); return options; } From 0b915bfa293b7e023bef2ee64187a07f1c769fe9 Mon Sep 17 00:00:00 2001 From: Lars Bachmann Date: Thu, 29 Apr 2021 09:31:51 +0200 Subject: [PATCH 143/322] [FLINK-21229][avro-confluent-registry] Add Confluent schema registry SSL support --- .../confluent/CachedSchemaCoderProvider.java | 21 ++- ...uentRegistryAvroDeserializationSchema.java | 110 +++++++++++-- ...fluentRegistryAvroSerializationSchema.java | 65 +++++++- .../confluent/RegistryAvroFormatFactory.java | 81 +++++++++- .../confluent/RegistryAvroOptions.java | 68 ++++++++ .../CachedSchemaCoderProviderTest.java | 145 ++++++++++++++++++ .../RegistryAvroFormatFactoryTest.java | 95 ++++++++++++ .../src/test/resources/test-keystore.jks | Bin 0 -> 2327 bytes 8 files changed, 552 insertions(+), 33 deletions(-) create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/test-keystore.jks diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java index b5f32008d..53be09c74 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java @@ -25,10 +25,11 @@ import javax.annotation.Nullable; +import java.util.Map; import java.util.Objects; /** - * A {@link SchemaCoder.SchemaCoderProvider} that uses a cached schema registry client underlying. * + * A {@link SchemaCoder.SchemaCoderProvider} that uses a cached schema registry client underneath. */ @Internal class CachedSchemaCoderProvider implements SchemaCoder.SchemaCoderProvider { @@ -37,21 +38,28 @@ class CachedSchemaCoderProvider implements SchemaCoder.SchemaCoderProvider { private final String subject; private final String url; private final int identityMapCapacity; + private final @Nullable Map registryConfigs; CachedSchemaCoderProvider(String url, int identityMapCapacity) { - this(null, url, identityMapCapacity); + this(null, url, identityMapCapacity, null); } - CachedSchemaCoderProvider(@Nullable String subject, String url, int identityMapCapacity) { + CachedSchemaCoderProvider( + @Nullable String subject, + String url, + int identityMapCapacity, + @Nullable Map registryConfigs) { this.subject = subject; this.url = Objects.requireNonNull(url); this.identityMapCapacity = identityMapCapacity; + this.registryConfigs = registryConfigs; } @Override public SchemaCoder get() { return new ConfluentSchemaRegistryCoder( - this.subject, new CachedSchemaRegistryClient(url, identityMapCapacity)); + this.subject, + new CachedSchemaRegistryClient(url, identityMapCapacity, registryConfigs)); } @Override @@ -65,11 +73,12 @@ public boolean equals(Object o) { CachedSchemaCoderProvider that = (CachedSchemaCoderProvider) o; return identityMapCapacity == that.identityMapCapacity && Objects.equals(subject, that.subject) - && url.equals(that.url); + && url.equals(that.url) + && Objects.equals(registryConfigs, that.registryConfigs); } @Override public int hashCode() { - return Objects.hash(subject, url, identityMapCapacity); + return Objects.hash(subject, url, identityMapCapacity, registryConfigs); } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java index 7becf5013..c448a91b8 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java @@ -28,6 +28,8 @@ import javax.annotation.Nullable; +import java.util.Map; + /** * Deserialization schema that deserializes from Avro binary format using {@link SchemaCoder} that * uses Confluent Schema Registry. @@ -60,8 +62,10 @@ private ConfluentRegistryAvroDeserializationSchema( /** * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link - * GenericRecord} using provided reader schema and looks up writer schema in Confluent Schema - * Registry. + * GenericRecord} using the provided reader schema and looks up the writer schema in the + * Confluent Schema Registry. + * + *

By default, this method supports up to 1000 cached schema versions. * * @param schema schema of produced records * @param url url of schema registry to connect @@ -74,25 +78,65 @@ public static ConfluentRegistryAvroDeserializationSchema forGener /** * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link - * GenericRecord} using provided reader schema and looks up writer schema in Confluent Schema - * Registry. + * GenericRecord} using the provided reader schema and looks up the writer schema in the + * Confluent Schema Registry. * * @param schema schema of produced records * @param url url of schema registry to connect - * @param identityMapCapacity maximum number of cached schema versions (default: 1000) + * @param identityMapCapacity maximum number of cached schema versions * @return deserialized record in form of {@link GenericRecord} */ public static ConfluentRegistryAvroDeserializationSchema forGeneric( Schema schema, String url, int identityMapCapacity) { + return forGeneric(schema, url, identityMapCapacity, null); + } + + /** + * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link + * GenericRecord} using the provided reader schema and looks up the writer schema in the + * Confluent Schema Registry. + * + *

By default, this method supports up to 1000 cached schema versions. + * + * @param schema schema of produced records + * @param url URL of schema registry to connect + * @param registryConfigs map with additional schema registry configs (for example SSL + * properties) + * @return deserialized record in form of {@link GenericRecord} + */ + public static ConfluentRegistryAvroDeserializationSchema forGeneric( + Schema schema, String url, @Nullable Map registryConfigs) { + return forGeneric(schema, url, DEFAULT_IDENTITY_MAP_CAPACITY, registryConfigs); + } + + /** + * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link + * GenericRecord} using the provided reader schema and looks up the writer schema in the + * Confluent Schema Registry. + * + * @param schema schema of produced records + * @param url URL of schema registry to connect + * @param identityMapCapacity maximum number of cached schema versions + * @param registryConfigs map with additional schema registry configs (for example SSL + * properties) + * @return deserialized record in form of {@link GenericRecord} + */ + public static ConfluentRegistryAvroDeserializationSchema forGeneric( + Schema schema, + String url, + int identityMapCapacity, + @Nullable Map registryConfigs) { return new ConfluentRegistryAvroDeserializationSchema<>( GenericRecord.class, schema, - new CachedSchemaCoderProvider(url, identityMapCapacity)); + new CachedSchemaCoderProvider(null, url, identityMapCapacity, registryConfigs)); } /** - * Creates {@link AvroDeserializationSchema} that produces classes that were generated from avro - * schema and looks up writer schema in Confluent Schema Registry. + * Creates {@link AvroDeserializationSchema} that produces classes that were generated from Avro + * schema and looks up the writer schema in the Confluent Schema Registry. + * + *

By default, this method supports up to 1000 cached schema versions. * * @param tClass class of record to be produced * @param url url of schema registry to connect @@ -100,22 +144,62 @@ public static ConfluentRegistryAvroDeserializationSchema forGener */ public static ConfluentRegistryAvroDeserializationSchema forSpecific(Class tClass, String url) { - return forSpecific(tClass, url, DEFAULT_IDENTITY_MAP_CAPACITY); + return forSpecific(tClass, url, DEFAULT_IDENTITY_MAP_CAPACITY, null); } /** - * Creates {@link AvroDeserializationSchema} that produces classes that were generated from avro - * schema and looks up writer schema in Confluent Schema Registry. + * Creates {@link AvroDeserializationSchema} that produces classes that were generated from Avro + * schema and looks up the writer schema in the Confluent Schema Registry. * * @param tClass class of record to be produced * @param url url of schema registry to connect - * @param identityMapCapacity maximum number of cached schema versions (default: 1000) + * @param identityMapCapacity maximum number of cached schema versions * @return deserialized record */ public static ConfluentRegistryAvroDeserializationSchema forSpecific( Class tClass, String url, int identityMapCapacity) { + return forSpecific(tClass, url, identityMapCapacity, null); + } + + /** + * Creates {@link AvroDeserializationSchema} that produces classes that were generated from Avro + * schema and looks up the writer schema in the Confluent Schema Registry. + * + *

By default, this method supports up to 1000 cached schema versions. + * + * @param tClass class of record to be produced + * @param url URL of schema registry to connect + * @param registryConfigs map with additional schema registry configs (for example SSL + * properties) + * @return deserialized record + */ + public static + ConfluentRegistryAvroDeserializationSchema forSpecific( + Class tClass, String url, @Nullable Map registryConfigs) { + return forSpecific(tClass, url, DEFAULT_IDENTITY_MAP_CAPACITY, registryConfigs); + } + + /** + * Creates {@link AvroDeserializationSchema} that produces classes that were generated from Avro + * schema and looks up the writer schema in the Confluent Schema Registry. + * + * @param tClass class of record to be produced + * @param url URL of schema registry to connect + * @param identityMapCapacity maximum number of cached schema versions + * @param registryConfigs map with additional schema registry configs (for example SSL + * properties) + * @return deserialized record + */ + public static + ConfluentRegistryAvroDeserializationSchema forSpecific( + Class tClass, + String url, + int identityMapCapacity, + @Nullable Map registryConfigs) { return new ConfluentRegistryAvroDeserializationSchema<>( - tClass, null, new CachedSchemaCoderProvider(url, identityMapCapacity)); + tClass, + null, + new CachedSchemaCoderProvider(null, url, identityMapCapacity, registryConfigs)); } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java index f06193fbd..b3b574c27 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java @@ -26,6 +26,10 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.specific.SpecificRecord; +import javax.annotation.Nullable; + +import java.util.Map; + /** * Serialization schema that serializes to Avro binary format that uses Confluent Schema Registry. * @@ -57,38 +61,83 @@ private ConfluentRegistryAvroSerializationSchema( /** * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from - * avro schema and writes the writer schema to Confluent Schema Registry. + * Avro schema and writes the writer schema to Confluent Schema Registry. * * @param tClass the type to be serialized * @param subject subject of schema registry to produce - * @param schemaRegistryUrl url of schema registry to connect - * @return Serialized record + * @param schemaRegistryUrl URL of schema registry to connect + * @return serialized record */ public static ConfluentRegistryAvroSerializationSchema forSpecific( Class tClass, String subject, String schemaRegistryUrl) { + return forSpecific(tClass, subject, schemaRegistryUrl, null); + } + + /** + * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from + * Avro schema and writes the writer schema to Confluent Schema Registry. + * + * @param tClass the type to be serialized + * @param subject subject of schema registry to produce + * @param schemaRegistryUrl URL of schema registry to connect + * @param registryConfigs map with additional schema registry configs (for example SSL + * properties) + * @return serialized record + */ + public static + ConfluentRegistryAvroSerializationSchema forSpecific( + Class tClass, + String subject, + String schemaRegistryUrl, + @Nullable Map registryConfigs) { return new ConfluentRegistryAvroSerializationSchema<>( tClass, null, new CachedSchemaCoderProvider( - subject, schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); + subject, + schemaRegistryUrl, + DEFAULT_IDENTITY_MAP_CAPACITY, + registryConfigs)); } /** * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from - * avro schema and writes the writer schema to Confluent Schema Registry. + * Avro schema and writes the writer schema to Confluent Schema Registry. * * @param subject subject of schema registry to produce * @param schema schema that will be used for serialization - * @param schemaRegistryUrl url of schema registry to connect - * @return Serialized record in form of byte array + * @param schemaRegistryUrl URL of schema registry to connect + * @return serialized record */ public static ConfluentRegistryAvroSerializationSchema forGeneric( String subject, Schema schema, String schemaRegistryUrl) { + return forGeneric(subject, schema, schemaRegistryUrl, null); + } + + /** + * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from + * Avro schema and writes the writer schema to Confluent Schema Registry. + * + * @param subject subject of schema registry to produce + * @param schema schema that will be used for serialization + * @param schemaRegistryUrl URL of schema registry to connect + * @param registryConfigs map with additional schema registry configs (for example SSL + * properties) + * @return serialized record + */ + public static ConfluentRegistryAvroSerializationSchema forGeneric( + String subject, + Schema schema, + String schemaRegistryUrl, + @Nullable Map registryConfigs) { return new ConfluentRegistryAvroSerializationSchema<>( GenericRecord.class, schema, new CachedSchemaCoderProvider( - subject, schemaRegistryUrl, DEFAULT_IDENTITY_MAP_CAPACITY)); + subject, + schemaRegistryUrl, + DEFAULT_IDENTITY_MAP_CAPACITY, + registryConfigs)); } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index 049fdbaa0..d8465e7ab 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -42,10 +42,21 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Optional; import java.util.Set; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.BASIC_AUTH_CREDENTIALS_SOURCE; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.BASIC_AUTH_USER_INFO; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.BEARER_AUTH_CREDENTIALS_SOURCE; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.BEARER_AUTH_TOKEN; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.PROPERTIES; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SSL_KEYSTORE_LOCATION; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SSL_KEYSTORE_PASSWORD; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SSL_TRUSTSTORE_LOCATION; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SSL_TRUSTSTORE_PASSWORD; import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SUBJECT; import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.URL; @@ -64,6 +75,8 @@ public DecodingFormat> createDecodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); String schemaRegistryURL = formatOptions.get(URL); + Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); + return new DecodingFormat>() { @Override public DeserializationSchema createRuntimeDecoder( @@ -72,8 +85,14 @@ public DeserializationSchema createRuntimeDecoder( final TypeInformation rowDataTypeInfo = context.createTypeInformation(producedDataType); return new AvroRowDataDeserializationSchema( - ConfluentRegistryAvroDeserializationSchema.forGeneric( - AvroSchemaConverter.convertToSchema(rowType), schemaRegistryURL), + optionalPropertiesMap.isEmpty() + ? ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(rowType), + schemaRegistryURL) + : ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(rowType), + schemaRegistryURL, + optionalPropertiesMap), AvroToRowDataConverters.createRowConverter(rowType), rowDataTypeInfo); } @@ -92,6 +111,8 @@ public EncodingFormat> createEncodingFormat( String schemaRegistryURL = formatOptions.get(URL); Optional subject = formatOptions.getOptional(SUBJECT); + Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); + if (!subject.isPresent()) { throw new ValidationException( String.format( @@ -106,10 +127,16 @@ public SerializationSchema createRuntimeEncoder( final RowType rowType = (RowType) consumedDataType.getLogicalType(); return new AvroRowDataSerializationSchema( rowType, - ConfluentRegistryAvroSerializationSchema.forGeneric( - subject.get(), - AvroSchemaConverter.convertToSchema(rowType), - schemaRegistryURL), + optionalPropertiesMap.isEmpty() + ? ConfluentRegistryAvroSerializationSchema.forGeneric( + subject.get(), + AvroSchemaConverter.convertToSchema(rowType), + schemaRegistryURL) + : ConfluentRegistryAvroSerializationSchema.forGeneric( + subject.get(), + AvroSchemaConverter.convertToSchema(rowType), + schemaRegistryURL, + optionalPropertiesMap), RowDataToAvroConverters.createConverter(rowType)); } @@ -136,6 +163,48 @@ public Set> requiredOptions() { public Set> optionalOptions() { Set> options = new HashSet<>(); options.add(SUBJECT); + options.add(PROPERTIES); + options.add(SSL_KEYSTORE_LOCATION); + options.add(SSL_KEYSTORE_PASSWORD); + options.add(SSL_TRUSTSTORE_LOCATION); + options.add(SSL_TRUSTSTORE_PASSWORD); + options.add(BASIC_AUTH_CREDENTIALS_SOURCE); + options.add(BASIC_AUTH_USER_INFO); + options.add(BEARER_AUTH_CREDENTIALS_SOURCE); + options.add(BEARER_AUTH_TOKEN); return options; } + + private Map buildOptionalPropertiesMap(ReadableConfig formatOptions) { + final Map properties = new HashMap<>(); + + formatOptions.getOptional(PROPERTIES).ifPresent(properties::putAll); + + formatOptions + .getOptional(SSL_KEYSTORE_LOCATION) + .ifPresent(v -> properties.put("schema.registry.ssl.keystore.location", v)); + formatOptions + .getOptional(SSL_KEYSTORE_PASSWORD) + .ifPresent(v -> properties.put("schema.registry.ssl.keystore.password", v)); + formatOptions + .getOptional(SSL_TRUSTSTORE_LOCATION) + .ifPresent(v -> properties.put("schema.registry.ssl.truststore.location", v)); + formatOptions + .getOptional(SSL_TRUSTSTORE_PASSWORD) + .ifPresent(v -> properties.put("schema.registry.ssl.truststore.password", v)); + formatOptions + .getOptional(BASIC_AUTH_CREDENTIALS_SOURCE) + .ifPresent(v -> properties.put("basic.auth.credentials.source", v)); + formatOptions + .getOptional(BASIC_AUTH_USER_INFO) + .ifPresent(v -> properties.put("basic.auth.user.info", v)); + formatOptions + .getOptional(BEARER_AUTH_CREDENTIALS_SOURCE) + .ifPresent(v -> properties.put("bearer.auth.credentials.source", v)); + formatOptions + .getOptional(BEARER_AUTH_TOKEN) + .ifPresent(v -> properties.put("bearer.auth.token", v)); + + return properties; + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java index 0d3c8f0f8..7bbaba597 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java @@ -21,6 +21,8 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; +import java.util.Map; + /** Options for Schema Registry Avro format. */ public class RegistryAvroOptions { private RegistryAvroOptions() {} @@ -39,4 +41,70 @@ private RegistryAvroOptions() {} .withFallbackKeys("schema-registry.subject") .withDescription( "Subject name to write to the Schema Registry service, required for sink"); + + // -------------------------------------------------------------------------------------------- + // Commonly used options maintained by Flink for convenience + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SSL_KEYSTORE_LOCATION = + ConfigOptions.key("ssl.keystore.location") + .stringType() + .noDefaultValue() + .withDescription("Location / File of SSL keystore"); + + public static final ConfigOption SSL_KEYSTORE_PASSWORD = + ConfigOptions.key("ssl.keystore.password") + .stringType() + .noDefaultValue() + .withDescription("Password for SSL keystore"); + + public static final ConfigOption SSL_TRUSTSTORE_LOCATION = + ConfigOptions.key("ssl.truststore.location") + .stringType() + .noDefaultValue() + .withDescription("Location / File of SSL truststore"); + + public static final ConfigOption SSL_TRUSTSTORE_PASSWORD = + ConfigOptions.key("ssl.truststore.password") + .stringType() + .noDefaultValue() + .withDescription("Password for SSL truststore"); + + public static final ConfigOption BASIC_AUTH_CREDENTIALS_SOURCE = + ConfigOptions.key("basic-auth.credentials-source") + .stringType() + .noDefaultValue() + .withDescription("Basic auth credentials source for Schema Registry"); + + public static final ConfigOption BASIC_AUTH_USER_INFO = + ConfigOptions.key("basic-auth.user-info") + .stringType() + .noDefaultValue() + .withDescription("Basic auth user info for schema registry"); + + public static final ConfigOption BEARER_AUTH_CREDENTIALS_SOURCE = + ConfigOptions.key("bearer-auth.credentials-source") + .stringType() + .noDefaultValue() + .withDescription("Bearer auth credentials source for Schema Registry"); + + public static final ConfigOption BEARER_AUTH_TOKEN = + ConfigOptions.key("bearer-auth.token") + .stringType() + .noDefaultValue() + .withDescription("Bearer auth token for Schema Registry"); + + // -------------------------------------------------------------------------------------------- + // Fallback properties + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption> PROPERTIES = + ConfigOptions.key("properties") + .mapType() + .noDefaultValue() + .withDescription( + "Properties map that is forwarded to the underlying Schema Registry. " + + "This is useful for options that are not officially exposed " + + "via Flink config options. However, note that Flink options " + + "have higher precedence."); } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java new file mode 100644 index 000000000..db877d8ed --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.avro.registry.confluent; + +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.RestService; +import io.confluent.kafka.schemaregistry.client.security.basicauth.BasicAuthCredentialProvider; +import io.confluent.kafka.schemaregistry.client.security.bearerauth.BearerAuthCredentialProvider; +import org.junit.Test; +import org.powermock.reflect.Whitebox; + +import javax.net.ssl.SSLSocketFactory; + +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +/** + * Tests for properties set by {@link RegistryAvroFormatFactory} in {@link + * CachedSchemaCoderProvider}. + */ +public class CachedSchemaCoderProviderTest { + + @Test + public void testThatSslIsNotInitializedForNoSslProperties() { + CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(new HashMap<>()); + SSLSocketFactory sslSocketFactory = getSslSocketFactoryFromProvider(provider); + + assertNull(sslSocketFactory); + } + + @Test + public void testThatSslIsInitializedForSslProperties() throws URISyntaxException { + String keystoreFile = getAbsolutePath("/test-keystore.jks"); + String keystorePassword = "123456"; + Map configs = new HashMap<>(); + configs.put("schema.registry.ssl.keystore.location", keystoreFile); + configs.put("schema.registry.ssl.keystore.password", keystorePassword); + configs.put("schema.registry.ssl.truststore.location", keystoreFile); + configs.put("schema.registry.ssl.truststore.password", keystorePassword); + + CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(configs); + SSLSocketFactory sslSocketFactory = getSslSocketFactoryFromProvider(provider); + + assertNotNull(sslSocketFactory); + } + + @Test + public void testThatBasicAuthIsNotInitializedForNoBasicAuthProperties() { + CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(new HashMap<>()); + BasicAuthCredentialProvider basicAuthCredentialProvider = + getBasicAuthFromProvider(provider); + + assertNull(basicAuthCredentialProvider); + } + + @Test + public void testThatBasicAuthIsInitializedForBasicAuthProperties() { + String userPassword = "user:pwd"; + Map configs = new HashMap<>(); + configs.put("basic.auth.credentials.source", "USER_INFO"); + configs.put("basic.auth.user.info", userPassword); + + CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(configs); + BasicAuthCredentialProvider basicAuthCredentialProvider = + getBasicAuthFromProvider(provider); + + assertNotNull(basicAuthCredentialProvider); + assertEquals(basicAuthCredentialProvider.getUserInfo(null), userPassword); + } + + @Test + public void testThatBearerAuthIsNotInitializedForNoBearerAuthProperties() { + CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(new HashMap<>()); + BearerAuthCredentialProvider bearerAuthCredentialProvider = + getBearerAuthFromProvider(provider); + + assertNull(bearerAuthCredentialProvider); + } + + @Test + public void testThatBearerAuthIsInitializedForBearerAuthProperties() { + String token = "123456"; + Map configs = new HashMap<>(); + configs.put("bearer.auth.credentials.source", "STATIC_TOKEN"); + configs.put("bearer.auth.token", token); + + CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(configs); + BearerAuthCredentialProvider bearerAuthCredentialProvider = + getBearerAuthFromProvider(provider); + + assertNotNull(bearerAuthCredentialProvider); + assertEquals(bearerAuthCredentialProvider.getBearerToken(null), token); + } + + private String getAbsolutePath(String path) throws URISyntaxException { + return CachedSchemaCoderProviderTest.class.getResource(path).toURI().getPath(); + } + + private CachedSchemaCoderProvider initCachedSchemaCoderProvider(Map config) { + return new CachedSchemaCoderProvider("test", "someUrl", 1000, config); + } + + private SSLSocketFactory getSslSocketFactoryFromProvider(CachedSchemaCoderProvider provider) { + return getInternalStateFromRestService("sslSocketFactory", provider); + } + + private BasicAuthCredentialProvider getBasicAuthFromProvider( + CachedSchemaCoderProvider provider) { + return getInternalStateFromRestService("basicAuthCredentialProvider", provider); + } + + private BearerAuthCredentialProvider getBearerAuthFromProvider( + CachedSchemaCoderProvider provider) { + return getInternalStateFromRestService("bearerAuthCredentialProvider", provider); + } + + private T getInternalStateFromRestService(String name, CachedSchemaCoderProvider provider) { + CachedSchemaRegistryClient cachedSchemaRegistryClient = + Whitebox.getInternalState(provider.get(), "schemaRegistryClient"); + RestService restService = + Whitebox.getInternalState(cachedSchemaRegistryClient, "restService"); + return Whitebox.getInternalState(restService, name); + } +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java index d923bf824..feed68bae 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java @@ -67,6 +67,20 @@ public class RegistryAvroFormatFactoryTest { private static final String SUBJECT = "test-subject"; private static final String REGISTRY_URL = "http://localhost:8081"; + private static final Map EXPECTED_OPTIONAL_PROPERTIES = new HashMap<>(); + + static { + EXPECTED_OPTIONAL_PROPERTIES.put( + "schema.registry.ssl.keystore.location", getAbsolutePath("/test-keystore.jks")); + EXPECTED_OPTIONAL_PROPERTIES.put("schema.registry.ssl.keystore.password", "123456"); + EXPECTED_OPTIONAL_PROPERTIES.put( + "schema.registry.ssl.truststore.location", getAbsolutePath("/test-keystore.jks")); + EXPECTED_OPTIONAL_PROPERTIES.put("schema.registry.ssl.truststore.password", "123456"); + EXPECTED_OPTIONAL_PROPERTIES.put("basic.auth.credentials.source", "USER_INFO"); + EXPECTED_OPTIONAL_PROPERTIES.put("basic.auth.user.info", "user:pwd"); + EXPECTED_OPTIONAL_PROPERTIES.put("bearer.auth.token", "CUSTOM"); + } + @Rule public ExpectedException thrown = ExpectedException.none(); @Test @@ -126,6 +140,52 @@ public void testMissingSubjectForSink() { createTableSink(SCHEMA, options); } + @Test + public void testDeserializationSchemaWithOptionalProperties() { + final AvroRowDataDeserializationSchema expectedDeser = + new AvroRowDataDeserializationSchema( + ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(ROW_TYPE), + REGISTRY_URL, + EXPECTED_OPTIONAL_PROPERTIES), + AvroToRowDataConverters.createRowConverter(ROW_TYPE), + InternalTypeInfo.of(ROW_TYPE)); + + final DynamicTableSource actualSource = createTableSource(SCHEMA, getOptionalProperties()); + assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); + TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = + (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; + + DeserializationSchema actualDeser = + scanSourceMock.valueFormat.createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); + + assertEquals(expectedDeser, actualDeser); + } + + @Test + public void testSerializationSchemaWithOptionalProperties() { + final AvroRowDataSerializationSchema expectedSer = + new AvroRowDataSerializationSchema( + ROW_TYPE, + ConfluentRegistryAvroSerializationSchema.forGeneric( + SUBJECT, + AvroSchemaConverter.convertToSchema(ROW_TYPE), + REGISTRY_URL, + EXPECTED_OPTIONAL_PROPERTIES), + RowDataToAvroConverters.createConverter(ROW_TYPE)); + + final DynamicTableSink actualSink = createTableSink(SCHEMA, getOptionalProperties()); + assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = + sinkMock.valueFormat.createRuntimeEncoder(null, SCHEMA.toPhysicalRowDataType()); + + assertEquals(expectedSer, actualSer); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -152,4 +212,39 @@ private Map getDefaultOptions() { options.put("avro-confluent.url", REGISTRY_URL); return options; } + + private Map getOptionalProperties() { + final Map properties = new HashMap<>(); + // defined via Flink maintained options + properties.put( + RegistryAvroOptions.SSL_KEYSTORE_LOCATION.key(), + getAbsolutePath("/test-keystore.jks")); + properties.put(RegistryAvroOptions.SSL_KEYSTORE_PASSWORD.key(), "123456"); + properties.put( + RegistryAvroOptions.SSL_TRUSTSTORE_LOCATION.key(), + getAbsolutePath("/test-keystore.jks")); + properties.put(RegistryAvroOptions.SSL_TRUSTSTORE_PASSWORD.key(), "123456"); + properties.put(RegistryAvroOptions.BASIC_AUTH_CREDENTIALS_SOURCE.key(), "USER_INFO"); + properties.put(RegistryAvroOptions.BASIC_AUTH_USER_INFO.key(), "user:pwd"); + // defined via general property map + properties.put("properties.bearer.auth.token", "CUSTOM"); + + return getModifiedOptions( + opts -> + properties.forEach( + (k, v) -> + opts.put( + String.format( + "%s.%s", + RegistryAvroFormatFactory.IDENTIFIER, k), + v))); + } + + private static String getAbsolutePath(String path) { + try { + return CachedSchemaCoderProviderTest.class.getResource(path).toURI().getPath(); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/test-keystore.jks b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/test-keystore.jks new file mode 100644 index 0000000000000000000000000000000000000000..b682158d9180cb6b2268eef858f6be3be0011ca8 GIT binary patch literal 2327 zcmY+DdpHvcAIEoZHm2B-%cz!VZncqXHtLW>7nPB=*WJt|9k)}ivqF|jJ4Y@}?r{#K zk`Z&u%G@s@LJ~=;k+s|}ulIf4_nhba@q2#H^ZEXM-@m?J0!vCB1OyXU@J^VNR{V+h zB^lrjU>*xz4`IP;wlR;u0!#fv0xQ2WZ)W&dcPxu)mqFwrrrO$Iq#RpyT|Irl$=ZUgEQZ7qW-+lpQJ3D zeHH3^E$_%zqh0Aw`|Ed)`W>xsHj;#tK~{@|H_1=fAMERz{2zj|H#1O&aQ7x<9du8)Fr_-*z&$voFt`SH& z+@9lmm%{rd`;13og_V}o3ADmxB2Vr=8?g}ZdCA&K-f$u29jq62NQgDi&Whpo+I_Z; z3Hp@yvN*F5vhp-Q?_~N;TC~7>jX6HIux@#?6E-2b^@L~B7U8SpbII#`k1X230~WaP z(p4F2<#OF@30C2;=JRk@qBT!vC6&q~SOD|S_xAw?n;PQJj7lrk+jEMcb zzp_La)A}frEqI2`i40Oe7yL?lG=O!c>NHGC2~|Ps{iS(>w4=yQ?hi?38He)~Om}k_ zq^$F0n1a+HCB^DfLyj-@d$bSKA`vmV38&CsdOl&>{eXVAp2{nkNPL2!w|P6+S1$9v zY`(H8-5YD>uS!Db z5iNeWTFA1tT-QD+a)7o)^FR|Vq&E8RDQv5s`H;uba{IGuL_TU>l8sU%OH)bz?)= z8F%LuotIlAr`QcYW+kTXlNuA}sz(~uO;!5V%2d2@ZrXA~m-rZ#+}ECrko7I2GuK{?YTuCNaUT zu}1aN7d+Okj}?5iQorYp(|6=p>94CqRD`!r!tj#X(CY@=gIs`NUcw(1$tg^MTG+=9 zW3#Fl0>eJdE<0ZFg)a*~R><4aXkV)ikuPs~vxW5IVh1`0EZ2ns(a8{Av71A;W7jQJVbqgJ9s3*Zb;37q=cd)i^c!3iP#1h?rFvZ{7d^$j#_ zLM|-et)Jbk&T_z5t@rirhw0@z3)41lIl;qRj-*wlg37+AR)5;CcNIro(h@3{cD2zP zn<6pdBFE2ReRfYN_igbS8mw}K-)X>i>hilDVx5-wK20sL z^BsmiRbzft8vHr@K23WhaCj9&r+O(bRv66t>yE<&f1ZHk${qRMYzD}@2@i5-qv}Od zYwO5WPfRM`jiN;%jz#FxYSVpF4El^gFiN#uC+MT|Zt4cOu}a&s^17HhNDn@DjOczX z4vAud3}>txkbH;CIOBa%sWuX_`e?CH0uSJ@=k1HPz17VY!58W9rdrz8&kJ6i`pUm0 zH`Hgg5$@#97%!I|GH&G`&EFllH)RDkYb#uGDQg5{zjX(rCptgb?+Gd$mstUthWr8W z_b&;?Am?+=_{v(lnZ3aQd%0xAnT9f362H~D5NqOQGndmcseuc7FrCvMr=-```qDqP zce)kgv6aXv2^(E;qe4tWSCJPTLRWYcov`G0{NC@kofx!tS80fXMzJ2rvvsgY=Ynih zsN0W9ilOV&z0Tw&1Y_WZ06(kxQ*JsQ=`%G{`hIC9{StnfP%a#qNho*L{E{vGK?x69 zJiZ*ojXiiybpGqAtJFo+V-QD;)&?j5_3iAA!F&B#-k)C-XPK8+?kQR`sHLZ0iUr1o zt6V$I1R;|!LvisR1$J3;IBcK-ZW-VpeB3VqlIgA&xiW->QG^Hi=jRWF>Zm+gYHr4( znz0IFB~<|>!~fv0$I<2Jj7_WG+}+06hrC;6k2C5^$F2Qz6+jfp#`NYeWL`J_0M2tW zzeB(41H)_~x3)?QyJzc~hV?o@ygEmXIvKxTuk*n0^TK>(*PVGIn3VOTspT{mQg3So zLMAYDBi==zw={8BPt-%%`Ccg;qK+vEkwqz+1|e^iiNbkE*^v{4TxY4vK_a(_AFk`p zz4C!+cr}1-U``Fzg#AM+8>GD8olYPV_7S9DP;C?#qznZ>&=@_@Pc;ZleNTnf2U*1h k-4yEm76{rle`cxUCE~`C_PiC*|C`ynX6Pyi42;kFH{XCk00000 literal 0 HcmV?d00001 From 900a19858331b23e7dce9cb9fd64b3b844882181 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 23 Jun 2021 08:40:56 +0200 Subject: [PATCH 144/322] [FLINK-21229][docs] Update avro-confluent docs This closes #15808. --- .../table/formats/avro-confluent.md | 105 +++++++++++++---- .../docs/connectors/table/formats/debezium.md | 105 +++++++++++++---- .../table/formats/avro-confluent.md | 105 +++++++++++++---- .../docs/connectors/table/formats/debezium.md | 107 ++++++++++++++---- .../confluent/RegistryAvroOptions.java | 10 +- 5 files changed, 345 insertions(+), 87 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md index a2faaec23..61248d185 100644 --- a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md @@ -187,27 +187,90 @@ Format 参数 - -

format
- 必选 - (none) - String - 指定要使用的格式,这里应该是 'avro-confluent'。 - - -
avro-confluent.url
- 必选 - (none) - String - 用于获取/注册 schemas 的 Confluent Schema Registry 的URL。 - - -
avro-confluent.subject
- 可选 - (none) - String - Confluent Schema Registry 主题,用于在序列化期间注册此格式使用的 schema。默认 kafka 和 upsert-kafka 连接器会使用 "<topic_name>-value" 或者 "<topic_name>-key" 作为 subject 名字。但是对于其他连接器(如 filesystem)则在当做 sink 使用时需要显式指定 subject 名字。 - + +
format
+ required + (none) + String + Specify what format to use, here should be 'avro-confluent'. + + +
avro-confluent.basic-auth.credentials-source
+ optional + (none) + String + Basic auth credentials source for Schema Registry + + +
avro-confluent.basic-auth.user-info
+ optional + (none) + String + Basic auth user info for schema registry + + +
avro-confluent.bearer-auth.credentials-source
+ optional + (none) + String + Bearer auth credentials source for Schema Registry + + +
avro-confluent.bearer-auth.token
+ optional + (none) + String + Bearer auth token for Schema Registry + + +
avro-confluent.properties
+ optional + (none) + Map + Properties map that is forwarded to the underlying Schema Registry. This is useful for options that are not officially exposed via Flink config options. However, note that Flink options have higher precedence. + + +
avro-confluent.ssl.keystore.location
+ optional + (none) + String + Location / File of SSL keystore + + +
avro-confluent.ssl.keystore.password
+ optional + (none) + String + Password for SSL keystore + + +
avro-confluent.ssl.truststore.location
+ optional + (none) + String + Location / File of SSL truststore + + +
avro-confluent.ssl.truststore.password
+ optional + (none) + String + Password for SSL truststore + + +
avro-confluent.subject
+ optional + (none) + String + The Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, 'kafka' and 'upsert-kafka' connectors use '<topic_name>-value' or '<topic_name>-key' as the default subject name if this format is used as the value or key format. But for other connectors (e.g. 'filesystem'), the subject option is required when used as sink. + + +
avro-confluent.url
+ required + (none) + String + The URL of the Confluent Schema Registry to fetch/register schemas. + diff --git a/docs/content.zh/docs/connectors/table/formats/debezium.md b/docs/content.zh/docs/connectors/table/formats/debezium.md index e5886e820..375968d16 100644 --- a/docs/content.zh/docs/connectors/table/formats/debezium.md +++ b/docs/content.zh/docs/connectors/table/formats/debezium.md @@ -257,27 +257,90 @@ Flink 提供了 `debezium-avro-confluent` 和 `debezium-json` 两种 format 来 - -
format
- 必选 - (none) - String - 指定要使用的格式,此处应为 'debezium-avro-confluent'。 - - -
debezium-avro-confluent.url
- 必选 - (none) - String - 用于获取/注册 schemas 的 Confluent Schema Registry 的 URL。 - - -
debezium-avro-confluent.subject
- 可选 - (none) - String - Confluent Schema Registry主题,用于在序列化期间注册此格式使用的 schema。默认 kafka 连接器会使用 "<topic_name>-value" 作为默认的 subject 名字,但是对于其他连接器(如 filesystem)则在当做 sink 使用时需要显式指定 subject 名字。 - + +
format
+ required + (none) + String + Specify what format to use, here should be 'debezium-avro-confluent'. + + +
debezium-avro-confluent.basic-auth.credentials-source
+ optional + (none) + String + Basic auth credentials source for Schema Registry + + +
debezium-avro-confluent.basic-auth.user-info
+ optional + (none) + String + Basic auth user info for schema registry + + +
debezium-avro-confluent.bearer-auth.credentials-source
+ optional + (none) + String + Bearer auth credentials source for Schema Registry + + +
debezium-avro-confluent.bearer-auth.token
+ optional + (none) + String + Bearer auth token for Schema Registry + + +
debezium-avro-confluent.properties
+ optional + (none) + Map + Properties map that is forwarded to the underlying Schema Registry. This is useful for options that are not officially exposed via Flink config options. However, note that Flink options have higher precedence. + + +
debezium-avro-confluent.ssl.keystore.location
+ optional + (none) + String + Location / File of SSL keystore + + +
debezium-avro-confluent.ssl.keystore.password
+ optional + (none) + String + Password for SSL keystore + + +
debezium-avro-confluent.ssl.truststore.location
+ optional + (none) + String + Location / File of SSL truststore + + +
debezium-avro-confluent.ssl.truststore.password
+ optional + (none) + String + Password for SSL truststore + + +
debezium-avro-confluent.subject
+ optional + (none) + String + The Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, 'kafka' and 'upsert-kafka' connectors use '<topic_name>-value' or '<topic_name>-key' as the default subject name if this format is used as the value or key format. But for other connectors (e.g. 'filesystem'), the subject option is required when used as sink. + + +
debezium-avro-confluent.url
+ required + (none) + String + The URL of the Confluent Schema Registry to fetch/register schemas. + diff --git a/docs/content/docs/connectors/table/formats/avro-confluent.md b/docs/content/docs/connectors/table/formats/avro-confluent.md index 8d6c54cf5..219b35599 100644 --- a/docs/content/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content/docs/connectors/table/formats/avro-confluent.md @@ -180,27 +180,90 @@ Format Options - -
format
- required - (none) - String - Specify what format to use, here should be 'avro-confluent'. - - -
avro-confluent.url
- required - (none) - String - The URL of the Confluent Schema Registry to fetch/register schemas. - - -
avro-confluent.subject
- optional - (none) - String - The Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, kafka and upsert-kafka connectors use "<topic_name>-value" or "<topic_name>-key" as the default subject name if avro-confluent is used as the value or key format. But for other connectors (e.g. filesystem), the subject option is required when used as sink. - + +
format
+ required + (none) + String + Specify what format to use, here should be 'avro-confluent'. + + +
avro-confluent.basic-auth.credentials-source
+ optional + (none) + String + Basic auth credentials source for Schema Registry + + +
avro-confluent.basic-auth.user-info
+ optional + (none) + String + Basic auth user info for schema registry + + +
avro-confluent.bearer-auth.credentials-source
+ optional + (none) + String + Bearer auth credentials source for Schema Registry + + +
avro-confluent.bearer-auth.token
+ optional + (none) + String + Bearer auth token for Schema Registry + + +
avro-confluent.properties
+ optional + (none) + Map + Properties map that is forwarded to the underlying Schema Registry. This is useful for options that are not officially exposed via Flink config options. However, note that Flink options have higher precedence. + + +
avro-confluent.ssl.keystore.location
+ optional + (none) + String + Location / File of SSL keystore + + +
avro-confluent.ssl.keystore.password
+ optional + (none) + String + Password for SSL keystore + + +
avro-confluent.ssl.truststore.location
+ optional + (none) + String + Location / File of SSL truststore + + +
avro-confluent.ssl.truststore.password
+ optional + (none) + String + Password for SSL truststore + + +
avro-confluent.subject
+ optional + (none) + String + The Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, 'kafka' and 'upsert-kafka' connectors use '<topic_name>-value' or '<topic_name>-key' as the default subject name if this format is used as the value or key format. But for other connectors (e.g. 'filesystem'), the subject option is required when used as sink. + + +
avro-confluent.url
+ required + (none) + String + The URL of the Confluent Schema Registry to fetch/register schemas. + diff --git a/docs/content/docs/connectors/table/formats/debezium.md b/docs/content/docs/connectors/table/formats/debezium.md index fa9c9ab01..e03a1535f 100644 --- a/docs/content/docs/connectors/table/formats/debezium.md +++ b/docs/content/docs/connectors/table/formats/debezium.md @@ -251,27 +251,90 @@ Use format `debezium-avro-confluent` to interpret Debezium Avro messages and for - -
format
- required - (none) - String - Specify what format to use, here should be 'debezium-avro-confluent'. - - -
debezium-avro-confluent.url
- required - (none) - String - The URL of the Confluent Schema Registry to fetch/register schemas. - - -
debezium-avro-confluent.subject
- optional - (none) - String - The Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, kafka connector use "<topic_name>-value" as the default subject name when debezium-avro-confluent is used as the value format. But for other connectors (e.g. filesystem), the subject option is required when used as sink. - + +
format
+ required + (none) + String + Specify what format to use, here should be 'debezium-avro-confluent'. + + +
debezium-avro-confluent.basic-auth.credentials-source
+ optional + (none) + String + Basic auth credentials source for Schema Registry + + +
debezium-avro-confluent.basic-auth.user-info
+ optional + (none) + String + Basic auth user info for schema registry + + +
debezium-avro-confluent.bearer-auth.credentials-source
+ optional + (none) + String + Bearer auth credentials source for Schema Registry + + +
debezium-avro-confluent.bearer-auth.token
+ optional + (none) + String + Bearer auth token for Schema Registry + + +
debezium-avro-confluent.properties
+ optional + (none) + Map + Properties map that is forwarded to the underlying Schema Registry. This is useful for options that are not officially exposed via Flink config options. However, note that Flink options have higher precedence. + + +
debezium-avro-confluent.ssl.keystore.location
+ optional + (none) + String + Location / File of SSL keystore + + +
debezium-avro-confluent.ssl.keystore.password
+ optional + (none) + String + Password for SSL keystore + + +
debezium-avro-confluent.ssl.truststore.location
+ optional + (none) + String + Location / File of SSL truststore + + +
debezium-avro-confluent.ssl.truststore.password
+ optional + (none) + String + Password for SSL truststore + + +
debezium-avro-confluent.subject
+ optional + (none) + String + The Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, 'kafka' and 'upsert-kafka' connectors use '<topic_name>-value' or '<topic_name>-key' as the default subject name if this format is used as the value or key format. But for other connectors (e.g. 'filesystem'), the subject option is required when used as sink. + + +
debezium-avro-confluent.url
+ required + (none) + String + The URL of the Confluent Schema Registry to fetch/register schemas. + @@ -354,7 +417,7 @@ Use format `debezium-avro-confluent` to interpret Debezium Avro messages and for - +{{< /tab >}} Caveats ---------------- diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java index 7bbaba597..0ed875589 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java @@ -32,7 +32,8 @@ private RegistryAvroOptions() {} .stringType() .noDefaultValue() .withFallbackKeys("schema-registry.url") - .withDescription("Required URL to connect to schema registry service"); + .withDescription( + "The URL of the Confluent Schema Registry to fetch/register schemas."); public static final ConfigOption SUBJECT = ConfigOptions.key("subject") @@ -40,7 +41,12 @@ private RegistryAvroOptions() {} .noDefaultValue() .withFallbackKeys("schema-registry.subject") .withDescription( - "Subject name to write to the Schema Registry service, required for sink"); + "The Confluent Schema Registry subject under which to register the " + + "schema used by this format during serialization. By default, " + + "'kafka' and 'upsert-kafka' connectors use '-value' " + + "or '-key' as the default subject name if this format " + + "is used as the value or key format. But for other connectors (e.g. 'filesystem'), " + + "the subject option is required when used as sink."); // -------------------------------------------------------------------------------------------- // Commonly used options maintained by Flink for convenience From 58893ed14418f42d719f036ac0fa40a3f4b81a95 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 24 Jun 2021 09:56:28 +0200 Subject: [PATCH 145/322] [FLINK-22880][table] Remove 'blink' term from code base This removes all mentionings of the term "blink" in the code base. In order to reduce user confusion, do not use this term anymore but refer to as "Flink SQL" or "Flink Table API". This closes #16374. --- .../flink/formats/json/JsonRowDataSerDeSchemaTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 518b067a9..fd398e53e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -105,10 +105,10 @@ public void testSerDe() throws Exception { .toInstant(); Map map = new HashMap<>(); - map.put("flink", 123L); + map.put("element", 123L); Map multiSet = new HashMap<>(); - multiSet.put("blink", 2); + multiSet.put("element", 2); Map> nestedMap = new HashMap<>(); Map innerMap = new HashMap<>(); @@ -135,8 +135,8 @@ public void testSerDe() throws Exception { root.put("timestamp3", "1990-10-14T12:12:43.123"); root.put("timestamp9", "1990-10-14T12:12:43.123456789"); root.put("timestampWithLocalZone", "1990-10-14T12:12:43.123456789Z"); - root.putObject("map").put("flink", 123); - root.putObject("multiSet").put("blink", 2); + root.putObject("map").put("element", 123); + root.putObject("multiSet").put("element", 2); root.putObject("map2map").putObject("inner_map").put("key", 234); byte[] serializedJson = objectMapper.writeValueAsBytes(root); From e3a499b32d06f13d0a4c41fe884c49ecd308eb35 Mon Sep 17 00:00:00 2001 From: Yangyang ZHANG Date: Thu, 24 Jun 2021 15:53:38 +0800 Subject: [PATCH 146/322] [FLINK-21448] Add dependency for ChangelogStateBackend ITTests --- flink-formats-kafka/flink-json-debezium/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 454902705..a19d89bbc 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -64,6 +64,13 @@ under the License. provided + + org.apache.flink + flink-statebackend-changelog_${scala.binary.version} + ${project.version} + test + + From c611980c94f3b7611f89f7df40b39198fb1190ed Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 5 Jul 2021 16:47:02 +0200 Subject: [PATCH 147/322] [FLINK-22879][table] Rename flink-table-runtime-blink to flink-table-runtime It might be required to update job dependencies. --- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index bbb9f067c..d7c1be920 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -101,7 +101,7 @@ under the License. org.apache.flink - flink-table-runtime-blink_${scala.binary.version} + flink-table-runtime_${scala.binary.version} ${project.version} test From 7f2ff966d0e993841972af2948872f539ffcda50 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 6 Jul 2021 14:22:24 +0200 Subject: [PATCH 148/322] [FLINK-22879][table] Rename flink-table-planner-blink to flink-table-planner It might be required to update job dependencies. Note that flink-table-planner used to contain the legacy planner before Flink 1.14 and now cotains the only officially supported planner (i.e. previously known as 'Blink' planner). This closes #16386. --- flink-formats-kafka/flink-json-debezium/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index a19d89bbc..a30c199d5 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -85,7 +85,7 @@ under the License. org.apache.flink - flink-table-planner-blink_${scala.binary.version} + flink-table-planner_${scala.binary.version} ${project.version} test @@ -93,7 +93,7 @@ under the License. org.apache.flink - flink-table-planner-blink_${scala.binary.version} + flink-table-planner_${scala.binary.version} ${project.version} test test-jar From 6df5e27435b1ce00cf47f12e4682bf8745106e5c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ingo=20B=C3=BCrk?= Date: Wed, 30 Jun 2021 12:15:34 +0200 Subject: [PATCH 149/322] [FLINK-23064][format-json] Make JsonOptions PublicEvolving --- .../flink/formats/json/JsonFormatFactory.java | 25 +++--- .../flink/formats/json/JsonFormatOptions.java | 88 +++++++++++++++++++ ...ptions.java => JsonFormatOptionsUtil.java} | 83 ++++------------- .../json/JsonRowDataSerializationSchema.java | 4 +- .../formats/json/RowDataToJsonConverters.java | 6 +- .../json/canal/CanalJsonFormatFactory.java | 37 +++++--- ...tions.java => CanalJsonFormatOptions.java} | 31 +++---- .../canal/CanalJsonSerializationSchema.java | 4 +- .../debezium/DebeziumJsonFormatFactory.java | 44 +++++++--- ...ns.java => DebeziumJsonFormatOptions.java} | 42 +++------ .../DebeziumJsonSerializationSchema.java | 4 +- .../maxwell/MaxwellJsonFormatFactory.java | 33 ++++--- ...ons.java => MaxwellJsonFormatOptions.java} | 31 +++---- .../MaxwellJsonSerializationSchema.java | 4 +- .../formats/json/JsonFormatFactoryTest.java | 2 +- .../json/JsonRowDataSerDeSchemaTest.java | 20 ++--- .../canal/CanalJsonFormatFactoryTest.java | 6 +- .../json/canal/CanalJsonSerDeSchemaTest.java | 4 +- .../DebeziumJsonFormatFactoryTest.java | 4 +- .../debezium/DebeziumJsonSerDeSchemaTest.java | 4 +- .../maxwell/MaxwellJsonFormatFactoryTest.java | 4 +- .../json/maxwell/MaxwellJsonSerDerTest.java | 4 +- 22 files changed, 262 insertions(+), 222 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java rename flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/{JsonOptions.java => JsonFormatOptionsUtil.java} (60%) rename flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/{CanalJsonOptions.java => CanalJsonFormatOptions.java} (71%) rename flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/{DebeziumJsonOptions.java => DebeziumJsonFormatOptions.java} (57%) rename flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/{MaxwellJsonOptions.java => MaxwellJsonFormatOptions.java} (57%) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index 8d69ec254..5d57f422e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -41,14 +41,12 @@ import java.util.HashSet; import java.util.Set; -import static org.apache.flink.formats.json.JsonOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; -import static org.apache.flink.formats.json.JsonOptions.FAIL_ON_MISSING_FIELD; -import static org.apache.flink.formats.json.JsonOptions.IGNORE_PARSE_ERRORS; -import static org.apache.flink.formats.json.JsonOptions.MAP_NULL_KEY_LITERAL; -import static org.apache.flink.formats.json.JsonOptions.MAP_NULL_KEY_MODE; -import static org.apache.flink.formats.json.JsonOptions.TIMESTAMP_FORMAT; -import static org.apache.flink.formats.json.JsonOptions.validateDecodingFormatOptions; -import static org.apache.flink.formats.json.JsonOptions.validateEncodingFormatOptions; +import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; +import static org.apache.flink.formats.json.JsonFormatOptions.FAIL_ON_MISSING_FIELD; +import static org.apache.flink.formats.json.JsonFormatOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.json.JsonFormatOptions.MAP_NULL_KEY_LITERAL; +import static org.apache.flink.formats.json.JsonFormatOptions.MAP_NULL_KEY_MODE; +import static org.apache.flink.formats.json.JsonFormatOptions.TIMESTAMP_FORMAT; /** * Table format factory for providing configured instances of JSON to RowData {@link @@ -62,11 +60,11 @@ public class JsonFormatFactory implements DeserializationFormatFactory, Serializ public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - validateDecodingFormatOptions(formatOptions); + JsonFormatOptionsUtil.validateDecodingFormatOptions(formatOptions); final boolean failOnMissingField = formatOptions.get(FAIL_ON_MISSING_FIELD); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - TimestampFormat timestampOption = JsonOptions.getTimestampFormat(formatOptions); + TimestampFormat timestampOption = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); return new DecodingFormat>() { @Override @@ -94,10 +92,11 @@ public ChangelogMode getChangelogMode() { public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - validateEncodingFormatOptions(formatOptions); + JsonFormatOptionsUtil.validateEncodingFormatOptions(formatOptions); - TimestampFormat timestampOption = JsonOptions.getTimestampFormat(formatOptions); - JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + TimestampFormat timestampOption = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); + JsonFormatOptions.MapNullKeyMode mapNullKeyMode = + JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); String mapNullKeyLiteral = formatOptions.get(MAP_NULL_KEY_LITERAL); final boolean encodeDecimalAsPlainNumber = diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java new file mode 100644 index 000000000..74d567ab9 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** Options for the JSON format. */ +@PublicEvolving +public class JsonFormatOptions { + + public static final ConfigOption FAIL_ON_MISSING_FIELD = + ConfigOptions.key("fail-on-missing-field") + .booleanType() + .defaultValue(false) + .withDescription( + "Optional flag to specify whether to fail if a field is missing or not, false by default."); + + public static final ConfigOption IGNORE_PARSE_ERRORS = + ConfigOptions.key("ignore-parse-errors") + .booleanType() + .defaultValue(false) + .withDescription( + "Optional flag to skip fields and rows with parse errors instead of failing;\n" + + "fields are set to null in case of errors, false by default."); + + public static final ConfigOption MAP_NULL_KEY_MODE = + ConfigOptions.key("map-null-key.mode") + .stringType() + .defaultValue("FAIL") + .withDescription( + "Optional flag to control the handling mode when serializing null key for map data, FAIL by default." + + " Option DROP will drop null key entries for map data." + + " Option LITERAL will use 'map-null-key.literal' as key literal."); + + public static final ConfigOption MAP_NULL_KEY_LITERAL = + ConfigOptions.key("map-null-key.literal") + .stringType() + .defaultValue("null") + .withDescription( + "Optional flag to specify string literal for null keys when 'map-null-key.mode' is LITERAL, \"null\" by default."); + + public static final ConfigOption TIMESTAMP_FORMAT = + ConfigOptions.key("timestamp-format.standard") + .stringType() + .defaultValue("SQL") + .withDescription( + "Optional flag to specify timestamp format, SQL by default." + + " Option ISO-8601 will parse input timestamp in \"yyyy-MM-ddTHH:mm:ss.s{precision}\" format and output timestamp in the same format." + + " Option SQL will parse input timestamp in \"yyyy-MM-dd HH:mm:ss.s{precision}\" format and output timestamp in the same format."); + + public static final ConfigOption ENCODE_DECIMAL_AS_PLAIN_NUMBER = + ConfigOptions.key("encode.decimal-as-plain-number") + .booleanType() + .defaultValue(false) + .withDescription( + "Optional flag to specify whether to encode all decimals as plain numbers instead of possible scientific notations, false by default."); + + // -------------------------------------------------------------------------------------------- + // Enums + // -------------------------------------------------------------------------------------------- + + /** Handling mode for map data with null key. */ + public enum MapNullKeyMode { + FAIL, + DROP, + LITERAL + } + + private JsonFormatOptions() {} +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptionsUtil.java similarity index 60% rename from flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java rename to flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptionsUtil.java index ae6243072..60a953d08 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptionsUtil.java @@ -18,8 +18,7 @@ package org.apache.flink.formats.json; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.api.TableException; @@ -31,55 +30,14 @@ import java.util.Set; import java.util.stream.Collectors; -/** This class holds configuration constants used by json format. */ -public class JsonOptions { - - public static final ConfigOption FAIL_ON_MISSING_FIELD = - ConfigOptions.key("fail-on-missing-field") - .booleanType() - .defaultValue(false) - .withDescription( - "Optional flag to specify whether to fail if a field is missing or not, false by default."); - - public static final ConfigOption IGNORE_PARSE_ERRORS = - ConfigOptions.key("ignore-parse-errors") - .booleanType() - .defaultValue(false) - .withDescription( - "Optional flag to skip fields and rows with parse errors instead of failing;\n" - + "fields are set to null in case of errors, false by default."); - - public static final ConfigOption MAP_NULL_KEY_MODE = - ConfigOptions.key("map-null-key.mode") - .stringType() - .defaultValue("FAIL") - .withDescription( - "Optional flag to control the handling mode when serializing null key for map data, FAIL by default." - + " Option DROP will drop null key entries for map data." - + " Option LITERAL will use 'map-null-key.literal' as key literal."); - - public static final ConfigOption MAP_NULL_KEY_LITERAL = - ConfigOptions.key("map-null-key.literal") - .stringType() - .defaultValue("null") - .withDescription( - "Optional flag to specify string literal for null keys when 'map-null-key.mode' is LITERAL, \"null\" by default."); - - public static final ConfigOption TIMESTAMP_FORMAT = - ConfigOptions.key("timestamp-format.standard") - .stringType() - .defaultValue("SQL") - .withDescription( - "Optional flag to specify timestamp format, SQL by default." - + " Option ISO-8601 will parse input timestamp in \"yyyy-MM-ddTHH:mm:ss.s{precision}\" format and output timestamp in the same format." - + " Option SQL will parse input timestamp in \"yyyy-MM-dd HH:mm:ss.s{precision}\" format and output timestamp in the same format."); - - public static final ConfigOption ENCODE_DECIMAL_AS_PLAIN_NUMBER = - ConfigOptions.key("encode.decimal-as-plain-number") - .booleanType() - .defaultValue(false) - .withDescription( - "Optional flag to specify whether to encode all decimals as plain numbers instead of possible scientific notations, false by default."); +import static org.apache.flink.formats.json.JsonFormatOptions.FAIL_ON_MISSING_FIELD; +import static org.apache.flink.formats.json.JsonFormatOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.json.JsonFormatOptions.MAP_NULL_KEY_MODE; +import static org.apache.flink.formats.json.JsonFormatOptions.TIMESTAMP_FORMAT; + +/** Utilities for {@link JsonFormatOptions}. */ +@Internal +public class JsonFormatOptionsUtil { // -------------------------------------------------------------------------------------------- // Option enumerations @@ -121,15 +79,15 @@ public static TimestampFormat getTimestampFormat(ReadableConfig config) { *

See {@link #JSON_MAP_NULL_KEY_MODE_FAIL}, {@link #JSON_MAP_NULL_KEY_MODE_DROP}, and {@link * #JSON_MAP_NULL_KEY_MODE_LITERAL} for more information. */ - public static MapNullKeyMode getMapNullKeyMode(ReadableConfig config) { + public static JsonFormatOptions.MapNullKeyMode getMapNullKeyMode(ReadableConfig config) { String mapNullKeyMode = config.get(MAP_NULL_KEY_MODE); switch (mapNullKeyMode.toUpperCase()) { case JSON_MAP_NULL_KEY_MODE_FAIL: - return MapNullKeyMode.FAIL; + return JsonFormatOptions.MapNullKeyMode.FAIL; case JSON_MAP_NULL_KEY_MODE_DROP: - return MapNullKeyMode.DROP; + return JsonFormatOptions.MapNullKeyMode.DROP; case JSON_MAP_NULL_KEY_MODE_LITERAL: - return MapNullKeyMode.LITERAL; + return JsonFormatOptions.MapNullKeyMode.LITERAL; default: throw new TableException( String.format( @@ -138,17 +96,6 @@ public static MapNullKeyMode getMapNullKeyMode(ReadableConfig config) { } } - // -------------------------------------------------------------------------------------------- - // Inner classes - // -------------------------------------------------------------------------------------------- - - /** Handling mode for map data with null key. */ - public enum MapNullKeyMode { - FAIL, - DROP, - LITERAL - } - // -------------------------------------------------------------------------------------------- // Validation // -------------------------------------------------------------------------------------------- @@ -171,7 +118,7 @@ public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { // validator for {@link MAP_NULL_KEY_MODE} Set nullKeyModes = - Arrays.stream(MapNullKeyMode.values()) + Arrays.stream(JsonFormatOptions.MapNullKeyMode.values()) .map(Objects::toString) .collect(Collectors.toSet()); if (!nullKeyModes.contains(tableOptions.get(MAP_NULL_KEY_MODE).toUpperCase())) { @@ -195,4 +142,6 @@ static void validateTimestampFormat(ReadableConfig tableOptions) { timestampFormat, TIMESTAMP_FORMAT.key())); } } + + private JsonFormatOptionsUtil() {} } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index 1008e6180..1b77aab1c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -59,7 +59,7 @@ public class JsonRowDataSerializationSchema implements SerializationSchema> createDecodingFormat( final String database = formatOptions.getOptional(DATABASE_INCLUDE).orElse(null); final String table = formatOptions.getOptional(TABLE_INCLUDE).orElse(null); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - final TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + final TimestampFormat timestampFormat = + JsonFormatOptionsUtil.getTimestampFormat(formatOptions); return new CanalJsonDecodingFormat(database, table, ignoreParseErrors, timestampFormat); } @@ -81,8 +81,9 @@ public EncodingFormat> createEncodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); validateEncodingFormatOptions(formatOptions); - TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); + JsonFormatOptions.MapNullKeyMode mapNullKeyMode = + JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); final boolean encodeDecimalAsPlainNumber = @@ -135,4 +136,14 @@ public Set> optionalOptions() { options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); return options; } + + /** Validator for canal decoding format. */ + private static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions); + } + + /** Validator for canal encoding format. */ + private static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatOptions.java similarity index 71% rename from flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java rename to flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatOptions.java index 24ad21fb2..6fa11bcb9 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatOptions.java @@ -18,22 +18,25 @@ package org.apache.flink.formats.json.canal; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; /** Option utils for canal-json format. */ -public class CanalJsonOptions { +@PublicEvolving +public class CanalJsonFormatOptions { - public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + public static final ConfigOption IGNORE_PARSE_ERRORS = + JsonFormatOptions.IGNORE_PARSE_ERRORS; - public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + public static final ConfigOption TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT; - public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; + public static final ConfigOption JSON_MAP_NULL_KEY_MODE = + JsonFormatOptions.MAP_NULL_KEY_MODE; public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = - JsonOptions.MAP_NULL_KEY_LITERAL; + JsonFormatOptions.MAP_NULL_KEY_LITERAL; public static final ConfigOption DATABASE_INCLUDE = ConfigOptions.key("database.include") @@ -51,17 +54,5 @@ public class CanalJsonOptions { "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the Canal record." + "The pattern string is compatible with Java's Pattern."); - // -------------------------------------------------------------------------------------------- - // Validation - // -------------------------------------------------------------------------------------------- - - /** Validator for canal decoding format. */ - public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateDecodingFormatOptions(tableOptions); - } - - /** Validator for canal encoding format. */ - public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateEncodingFormatOptions(tableOptions); - } + private CanalJsonFormatOptions() {} } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java index b15fb4a26..90302d174 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; import org.apache.flink.formats.json.JsonRowDataSerializationSchema; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.ArrayData; @@ -57,7 +57,7 @@ public class CanalJsonSerializationSchema implements SerializationSchema> createDecodingFormat( final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - final TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + final TimestampFormat timestampFormat = + JsonFormatOptionsUtil.getTimestampFormat(formatOptions); return new DebeziumJsonDecodingFormat(schemaInclude, ignoreParseErrors, timestampFormat); } @@ -82,8 +83,9 @@ public EncodingFormat> createEncodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); validateEncodingFormatOptions(formatOptions); - TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); + JsonFormatOptions.MapNullKeyMode mapNullKeyMode = + JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); final boolean encodeDecimalAsPlainNumber = @@ -136,4 +138,22 @@ public Set> optionalOptions() { options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); return options; } + + /** Validator for debezium decoding format. */ + private static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions); + } + + /** Validator for debezium encoding format. */ + private static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions); + + // validator for {@link SCHEMA_INCLUDE} + if (tableOptions.get(SCHEMA_INCLUDE)) { + throw new ValidationException( + String.format( + "Debezium JSON serialization doesn't support '%s.%s' option been set to true.", + IDENTIFIER, SCHEMA_INCLUDE.key())); + } + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatOptions.java similarity index 57% rename from flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java rename to flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatOptions.java index fa1c1e1bb..bf338a9f9 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatOptions.java @@ -18,16 +18,14 @@ package org.apache.flink.formats.json.debezium; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.json.JsonOptions; -import org.apache.flink.table.api.ValidationException; - -import static org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory.IDENTIFIER; +import org.apache.flink.formats.json.JsonFormatOptions; /** Option utils for debezium-json format. */ -public class DebeziumJsonOptions { +@PublicEvolving +public class DebeziumJsonFormatOptions { public static final ConfigOption SCHEMA_INCLUDE = ConfigOptions.key("schema-include") @@ -39,34 +37,16 @@ public class DebeziumJsonOptions { + "This option indicates the Debezium JSON data include the schema in the message or not. " + "Default is false."); - public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + public static final ConfigOption IGNORE_PARSE_ERRORS = + JsonFormatOptions.IGNORE_PARSE_ERRORS; - public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + public static final ConfigOption TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT; - public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; + public static final ConfigOption JSON_MAP_NULL_KEY_MODE = + JsonFormatOptions.MAP_NULL_KEY_MODE; public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = - JsonOptions.MAP_NULL_KEY_LITERAL; - - // -------------------------------------------------------------------------------------------- - // Validation - // -------------------------------------------------------------------------------------------- - - /** Validator for debezium decoding format. */ - public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateDecodingFormatOptions(tableOptions); - } - - /** Validator for debezium encoding format. */ - public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateEncodingFormatOptions(tableOptions); + JsonFormatOptions.MAP_NULL_KEY_LITERAL; - // validator for {@link SCHEMA_INCLUDE} - if (tableOptions.get(SCHEMA_INCLUDE)) { - throw new ValidationException( - String.format( - "Debezium JSON serialization doesn't support '%s.%s' option been set to true.", - IDENTIFIER, SCHEMA_INCLUDE.key())); - } - } + private DebeziumJsonFormatOptions() {} } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java index 64e0105da..f6e37073d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; import org.apache.flink.formats.json.JsonRowDataSerializationSchema; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.GenericRowData; @@ -54,7 +54,7 @@ public class DebeziumJsonSerializationSchema implements SerializationSchema> createDecodingFormat( validateDecodingFormatOptions(formatOptions); final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - final TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + final TimestampFormat timestampFormat = + JsonFormatOptionsUtil.getTimestampFormat(formatOptions); return new MaxwellJsonDecodingFormat(ignoreParseErrors, timestampFormat); } @@ -76,8 +76,9 @@ public EncodingFormat> createEncodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); validateEncodingFormatOptions(formatOptions); - TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); - JsonOptions.MapNullKeyMode mapNullKeyMode = JsonOptions.getMapNullKeyMode(formatOptions); + TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); + JsonFormatOptions.MapNullKeyMode mapNullKeyMode = + JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); final boolean encodeDecimalAsPlainNumber = @@ -129,4 +130,14 @@ public Set> optionalOptions() { options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); return options; } + + /** Validator for maxwell decoding format. */ + private static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions); + } + + /** Validator for maxwell encoding format. */ + private static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions); + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatOptions.java similarity index 57% rename from flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java rename to flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatOptions.java index 6d66c28e7..011fa7b79 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonOptions.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatOptions.java @@ -18,33 +18,24 @@ package org.apache.flink.formats.json.maxwell; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; /** Option utils for maxwell-json format. */ -public class MaxwellJsonOptions { +@PublicEvolving +public class MaxwellJsonFormatOptions { - public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS; + public static final ConfigOption IGNORE_PARSE_ERRORS = + JsonFormatOptions.IGNORE_PARSE_ERRORS; - public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT; + public static final ConfigOption TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT; - public static final ConfigOption JSON_MAP_NULL_KEY_MODE = JsonOptions.MAP_NULL_KEY_MODE; + public static final ConfigOption JSON_MAP_NULL_KEY_MODE = + JsonFormatOptions.MAP_NULL_KEY_MODE; public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = - JsonOptions.MAP_NULL_KEY_LITERAL; + JsonFormatOptions.MAP_NULL_KEY_LITERAL; - // -------------------------------------------------------------------------------------------- - // Validation - // -------------------------------------------------------------------------------------------- - - /** Validator for maxwell decoding format. */ - public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateDecodingFormatOptions(tableOptions); - } - - /** Validator for maxwell encoding format. */ - public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - JsonOptions.validateEncodingFormatOptions(tableOptions); - } + private MaxwellJsonFormatOptions() {} } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java index 27b9da48e..8dfac669c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; import org.apache.flink.formats.json.JsonRowDataSerializationSchema; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.GenericRowData; @@ -54,7 +54,7 @@ public class MaxwellJsonSerializationSchema implements SerializationSchema options) { new JsonRowDataSerializationSchema( PHYSICAL_TYPE, TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index fd398e53e..e14849b74 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -197,7 +197,7 @@ public void testSerDe() throws Exception { new JsonRowDataSerializationSchema( schema, TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); @@ -287,7 +287,7 @@ public void testSerDeMultiRows() throws Exception { new JsonRowDataSerializationSchema( rowType, TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); @@ -372,7 +372,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { new JsonRowDataSerializationSchema( rowType, TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); @@ -487,7 +487,7 @@ public void testSerDeSQLTimestampFormat() throws Exception { new JsonRowDataSerializationSchema( rowType, TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); @@ -530,7 +530,7 @@ public void testSerializationMapNullKey() throws Exception { new JsonRowDataSerializationSchema( rowType, TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.FAIL, + JsonFormatOptions.MapNullKeyMode.FAIL, "null", true); // expect message for serializationSchema1 @@ -542,7 +542,7 @@ public void testSerializationMapNullKey() throws Exception { new JsonRowDataSerializationSchema( rowType, TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.DROP, + JsonFormatOptions.MapNullKeyMode.DROP, "null", true); // expect result for serializationSchema2 @@ -552,7 +552,7 @@ public void testSerializationMapNullKey() throws Exception { new JsonRowDataSerializationSchema( rowType, TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "nullKey", true); // expect result for serializationSchema3 @@ -596,14 +596,14 @@ public void testSerializationDecimalEncode() throws Exception { new JsonRowDataSerializationSchema( schema, TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); JsonRowDataSerializationSchema scientificDecimalSerializer = new JsonRowDataSerializationSchema( schema, TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", false); @@ -641,7 +641,7 @@ public void testSerializationWithTypesMismatch() { new JsonRowDataSerializationSchema( rowType, TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.FAIL, + JsonFormatOptions.MapNullKeyMode.FAIL, "null", true); String errorMessage = "Fail to serialize at field: f1."; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 49682940a..8c8f8443d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; @@ -75,7 +75,7 @@ public void testDefaultOptions() { new CanalJsonSerializationSchema( PHYSICAL_TYPE, TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.FAIL, + JsonFormatOptions.MapNullKeyMode.FAIL, "null", false); SerializationSchema actualSer = createSerializationSchema(options); @@ -110,7 +110,7 @@ public void testUserDefinedOptions() { new CanalJsonSerializationSchema( PHYSICAL_TYPE, TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "nullKey", true); SerializationSchema actualSer = createSerializationSchema(options); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index 73b34d970..d7f70ecc0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -19,7 +19,7 @@ package org.apache.flink.formats.json.canal; import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; import org.apache.flink.formats.json.canal.CanalJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.RowData; @@ -218,7 +218,7 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali new CanalJsonSerializationSchema( (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); serializationSchema.open(null); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 466365ffa..1eadf9c59 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; @@ -83,7 +83,7 @@ public void testSeDeSchema() { new DebeziumJsonSerializationSchema( (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java index a628dd69c..2422dbd3a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -19,7 +19,7 @@ package org.apache.flink.formats.json.debezium; import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; import org.apache.flink.formats.json.debezium.DebeziumJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.RowData; @@ -256,7 +256,7 @@ private void testSerializationDeserialization(String resourceFile, boolean schem new DebeziumJsonSerializationSchema( (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java index 275ee6455..3c517bac0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; @@ -70,7 +70,7 @@ public void testSeDeSchema() { new MaxwellJsonSerializationSchema( PHYSICAL_TYPE, TimestampFormat.ISO_8601, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java index 009eba15a..9fb55dfce 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java @@ -19,7 +19,7 @@ package org.apache.flink.formats.json.maxwell; import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonOptions; +import org.apache.flink.formats.json.JsonFormatOptions; import org.apache.flink.formats.json.maxwell.MaxwellJsonDecodingFormat.ReadableMetadata; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.RowData; @@ -183,7 +183,7 @@ public void testSerializationDeserialization() throws Exception { new MaxwellJsonSerializationSchema( (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), TimestampFormat.SQL, - JsonOptions.MapNullKeyMode.LITERAL, + JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); serializationSchema.open(null); From a9bca2bf3ec31ff98f442522366fa7a658f589b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ingo=20B=C3=BCrk?= Date: Wed, 30 Jun 2021 12:25:35 +0200 Subject: [PATCH 150/322] [FLINK-23064][format-avro][format-avro-confluent-registry] Expose config options as PublicEvolving --- ...s.java => AvroConfluentFormatOptions.java} | 7 ++++-- .../confluent/RegistryAvroFormatFactory.java | 22 +++++++++---------- .../debezium/DebeziumAvroFormatFactory.java | 4 ++-- .../RegistryAvroFormatFactoryTest.java | 12 +++++----- 4 files changed, 24 insertions(+), 21 deletions(-) rename flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/{RegistryAvroOptions.java => AvroConfluentFormatOptions.java} (97%) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java similarity index 97% rename from flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java rename to flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java index 0ed875589..9d06a6340 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroOptions.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java @@ -18,14 +18,15 @@ package org.apache.flink.formats.avro.registry.confluent; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import java.util.Map; /** Options for Schema Registry Avro format. */ -public class RegistryAvroOptions { - private RegistryAvroOptions() {} +@PublicEvolving +public class AvroConfluentFormatOptions { public static final ConfigOption URL = ConfigOptions.key("url") @@ -113,4 +114,6 @@ private RegistryAvroOptions() {} + "This is useful for options that are not officially exposed " + "via Flink config options. However, note that Flink options " + "have higher precedence."); + + private AvroConfluentFormatOptions() {} } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index d8465e7ab..b78613caa 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -48,17 +48,17 @@ import java.util.Optional; import java.util.Set; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.BASIC_AUTH_CREDENTIALS_SOURCE; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.BASIC_AUTH_USER_INFO; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.BEARER_AUTH_CREDENTIALS_SOURCE; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.BEARER_AUTH_TOKEN; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.PROPERTIES; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SSL_KEYSTORE_LOCATION; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SSL_KEYSTORE_PASSWORD; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SSL_TRUSTSTORE_LOCATION; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SSL_TRUSTSTORE_PASSWORD; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SUBJECT; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.URL; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_CREDENTIALS_SOURCE; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_CREDENTIALS_SOURCE; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_TOKEN; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.PROPERTIES; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_LOCATION; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_PASSWORD; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_LOCATION; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_PASSWORD; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SUBJECT; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.URL; /** * Table format factory for providing configured instances of Schema Registry Avro to RowData {@link diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java index f566bfbaf..8f2fc6c76 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java @@ -42,8 +42,8 @@ import java.util.Optional; import java.util.Set; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.SUBJECT; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroOptions.URL; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SUBJECT; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.URL; /** * Format factory for providing configured instances of Debezium Avro to RowData {@link diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java index feed68bae..7c5b42666 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java @@ -217,15 +217,15 @@ private Map getOptionalProperties() { final Map properties = new HashMap<>(); // defined via Flink maintained options properties.put( - RegistryAvroOptions.SSL_KEYSTORE_LOCATION.key(), + AvroConfluentFormatOptions.SSL_KEYSTORE_LOCATION.key(), getAbsolutePath("/test-keystore.jks")); - properties.put(RegistryAvroOptions.SSL_KEYSTORE_PASSWORD.key(), "123456"); + properties.put(AvroConfluentFormatOptions.SSL_KEYSTORE_PASSWORD.key(), "123456"); properties.put( - RegistryAvroOptions.SSL_TRUSTSTORE_LOCATION.key(), + AvroConfluentFormatOptions.SSL_TRUSTSTORE_LOCATION.key(), getAbsolutePath("/test-keystore.jks")); - properties.put(RegistryAvroOptions.SSL_TRUSTSTORE_PASSWORD.key(), "123456"); - properties.put(RegistryAvroOptions.BASIC_AUTH_CREDENTIALS_SOURCE.key(), "USER_INFO"); - properties.put(RegistryAvroOptions.BASIC_AUTH_USER_INFO.key(), "user:pwd"); + properties.put(AvroConfluentFormatOptions.SSL_TRUSTSTORE_PASSWORD.key(), "123456"); + properties.put(AvroConfluentFormatOptions.BASIC_AUTH_CREDENTIALS_SOURCE.key(), "USER_INFO"); + properties.put(AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO.key(), "user:pwd"); // defined via general property map properties.put("properties.bearer.auth.token", "CUSTOM"); From 47a10d64fcd5b40db2f8ba7ca90e6f652685b816 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 29 Jun 2021 22:03:28 +0200 Subject: [PATCH 151/322] [FLINK-18783] Load Akka with separate classloader --- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index a30c199d5..9c230c500 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -66,7 +66,7 @@ under the License. org.apache.flink - flink-statebackend-changelog_${scala.binary.version} + flink-statebackend-changelog ${project.version} test From e75bcdbfb99ead032617513422f6a265eb97df04 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 14 Jul 2021 09:10:37 +0200 Subject: [PATCH 152/322] Revert "[FLINK-18783] Load Akka with separate classloader" This reverts commit 54db3d6fa9e5d6ce14630b338c66b2ad28e8b4fa. --- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 9c230c500..a30c199d5 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -66,7 +66,7 @@ under the License. org.apache.flink - flink-statebackend-changelog + flink-statebackend-changelog_${scala.binary.version} ${project.version} test From 1bce2e2ed3be2d074152230860dc92ea2dc63b7d Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 29 Jun 2021 22:03:28 +0200 Subject: [PATCH 153/322] [FLINK-18783] Load Akka with separate classloader --- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index a30c199d5..9c230c500 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -66,7 +66,7 @@ under the License. org.apache.flink - flink-statebackend-changelog_${scala.binary.version} + flink-statebackend-changelog ${project.version} test From 1649c9c8cf43694af6c7e525a79ef900eb104e64 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 21 Jul 2021 14:39:15 +0200 Subject: [PATCH 154/322] [FLINK-23450][avro-confluent-registry] Set properties map for DebeziumAvroFormat This closes #16565. --- .../confluent/RegistryAvroFormatFactory.java | 35 +++++++++---------- .../DebeziumAvroDeserializationSchema.java | 11 ++++-- .../debezium/DebeziumAvroFormatFactory.java | 27 ++++++++++++-- .../DebeziumAvroSerializationSchema.java | 13 +++++-- .../DebeziumAvroFormatFactoryTest.java | 11 ++++-- 5 files changed, 70 insertions(+), 27 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index b78613caa..2b37e3649 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -42,6 +42,8 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import javax.annotation.Nullable; + import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -85,14 +87,10 @@ public DeserializationSchema createRuntimeDecoder( final TypeInformation rowDataTypeInfo = context.createTypeInformation(producedDataType); return new AvroRowDataDeserializationSchema( - optionalPropertiesMap.isEmpty() - ? ConfluentRegistryAvroDeserializationSchema.forGeneric( - AvroSchemaConverter.convertToSchema(rowType), - schemaRegistryURL) - : ConfluentRegistryAvroDeserializationSchema.forGeneric( - AvroSchemaConverter.convertToSchema(rowType), - schemaRegistryURL, - optionalPropertiesMap), + ConfluentRegistryAvroDeserializationSchema.forGeneric( + AvroSchemaConverter.convertToSchema(rowType), + schemaRegistryURL, + optionalPropertiesMap), AvroToRowDataConverters.createRowConverter(rowType), rowDataTypeInfo); } @@ -127,16 +125,11 @@ public SerializationSchema createRuntimeEncoder( final RowType rowType = (RowType) consumedDataType.getLogicalType(); return new AvroRowDataSerializationSchema( rowType, - optionalPropertiesMap.isEmpty() - ? ConfluentRegistryAvroSerializationSchema.forGeneric( - subject.get(), - AvroSchemaConverter.convertToSchema(rowType), - schemaRegistryURL) - : ConfluentRegistryAvroSerializationSchema.forGeneric( - subject.get(), - AvroSchemaConverter.convertToSchema(rowType), - schemaRegistryURL, - optionalPropertiesMap), + ConfluentRegistryAvroSerializationSchema.forGeneric( + subject.get(), + AvroSchemaConverter.convertToSchema(rowType), + schemaRegistryURL, + optionalPropertiesMap), RowDataToAvroConverters.createConverter(rowType)); } @@ -175,7 +168,8 @@ public Set> optionalOptions() { return options; } - private Map buildOptionalPropertiesMap(ReadableConfig formatOptions) { + public static @Nullable Map buildOptionalPropertiesMap( + ReadableConfig formatOptions) { final Map properties = new HashMap<>(); formatOptions.getOptional(PROPERTIES).ifPresent(properties::putAll); @@ -205,6 +199,9 @@ private Map buildOptionalPropertiesMap(ReadableConfig formatOpti .getOptional(BEARER_AUTH_TOKEN) .ifPresent(v -> properties.put("bearer.auth.token", v)); + if (properties.isEmpty()) { + return null; + } return properties; } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java index 28d253196..9ae5133ee 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java @@ -34,7 +34,10 @@ import org.apache.flink.types.RowKind; import org.apache.flink.util.Collector; +import javax.annotation.Nullable; + import java.io.IOException; +import java.util.Map; import java.util.Objects; import static java.lang.String.format; @@ -74,7 +77,10 @@ public final class DebeziumAvroDeserializationSchema implements DeserializationS private final TypeInformation producedTypeInfo; public DebeziumAvroDeserializationSchema( - RowType rowType, TypeInformation producedTypeInfo, String schemaRegistryUrl) { + RowType rowType, + TypeInformation producedTypeInfo, + String schemaRegistryUrl, + @Nullable Map registryConfigs) { this.producedTypeInfo = producedTypeInfo; RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); @@ -82,7 +88,8 @@ public DebeziumAvroDeserializationSchema( new AvroRowDataDeserializationSchema( ConfluentRegistryAvroDeserializationSchema.forGeneric( AvroSchemaConverter.convertToSchema(debeziumAvroRowType), - schemaRegistryUrl), + schemaRegistryUrl, + registryConfigs), AvroToRowDataConverters.createRowConverter(debeziumAvroRowType), producedTypeInfo); } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java index 8f2fc6c76..dbb079049 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java @@ -39,11 +39,22 @@ import org.apache.flink.types.RowKind; import java.util.HashSet; +import java.util.Map; import java.util.Optional; import java.util.Set; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_CREDENTIALS_SOURCE; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_CREDENTIALS_SOURCE; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_TOKEN; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.PROPERTIES; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_LOCATION; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_PASSWORD; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_LOCATION; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_PASSWORD; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SUBJECT; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.URL; +import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroFormatFactory.buildOptionalPropertiesMap; /** * Format factory for providing configured instances of Debezium Avro to RowData {@link @@ -60,6 +71,7 @@ public DecodingFormat> createDecodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); String schemaRegistryURL = formatOptions.get(URL); + Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); return new DecodingFormat>() { @Override @@ -69,7 +81,7 @@ public DeserializationSchema createRuntimeDecoder( final TypeInformation producedTypeInfo = context.createTypeInformation(producedDataType); return new DebeziumAvroDeserializationSchema( - rowType, producedTypeInfo, schemaRegistryURL); + rowType, producedTypeInfo, schemaRegistryURL, optionalPropertiesMap); } @Override @@ -91,6 +103,8 @@ public EncodingFormat> createEncodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); String schemaRegistryURL = formatOptions.get(URL); Optional subject = formatOptions.getOptional(SUBJECT); + Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); + if (!subject.isPresent()) { throw new ValidationException( String.format( @@ -114,7 +128,7 @@ public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); return new DebeziumAvroSerializationSchema( - rowType, schemaRegistryURL, subject.get()); + rowType, schemaRegistryURL, subject.get(), optionalPropertiesMap); } }; } @@ -135,6 +149,15 @@ public Set> requiredOptions() { public Set> optionalOptions() { Set> options = new HashSet<>(); options.add(SUBJECT); + options.add(PROPERTIES); + options.add(SSL_KEYSTORE_LOCATION); + options.add(SSL_KEYSTORE_PASSWORD); + options.add(SSL_TRUSTSTORE_LOCATION); + options.add(SSL_TRUSTSTORE_PASSWORD); + options.add(BASIC_AUTH_CREDENTIALS_SOURCE); + options.add(BASIC_AUTH_USER_INFO); + options.add(BEARER_AUTH_CREDENTIALS_SOURCE); + options.add(BEARER_AUTH_TOKEN); return options; } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java index 07ab0e73e..4cecceff2 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.avro.registry.confluent.debezium; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; @@ -31,6 +32,9 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import javax.annotation.Nullable; + +import java.util.Map; import java.util.Objects; import static java.lang.String.format; @@ -40,6 +44,7 @@ * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium * Avro. */ +@Internal public class DebeziumAvroSerializationSchema implements SerializationSchema { private static final long serialVersionUID = 1L; @@ -54,7 +59,10 @@ public class DebeziumAvroSerializationSchema implements SerializationSchema registryConfigs) { RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); this.avroSerializer = @@ -63,7 +71,8 @@ public DebeziumAvroSerializationSchema( ConfluentRegistryAvroSerializationSchema.forGeneric( schemaRegistrySubject, AvroSchemaConverter.convertToSchema(debeziumAvroRowType), - schemaRegistryUrl), + schemaRegistryUrl, + registryConfigs), RowDataToAvroConverters.createConverter(debeziumAvroRowType)); } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java index 5bb81820e..eb91ccbed 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java @@ -67,14 +67,19 @@ public class DebeziumAvroFormatFactoryTest extends TestLogger { public void testSeDeSchema() { final Map options = getAllOptions(); + final Map registryConfigs = new HashMap<>(); + registryConfigs.put("basic.auth.user.info", "something1"); + registryConfigs.put("basic.auth.credentials.source", "something2"); + DebeziumAvroDeserializationSchema expectedDeser = new DebeziumAvroDeserializationSchema( - ROW_TYPE, InternalTypeInfo.of(ROW_TYPE), REGISTRY_URL); + ROW_TYPE, InternalTypeInfo.of(ROW_TYPE), REGISTRY_URL, registryConfigs); DeserializationSchema actualDeser = createDeserializationSchema(options); assertEquals(expectedDeser, actualDeser); DebeziumAvroSerializationSchema expectedSer = - new DebeziumAvroSerializationSchema(ROW_TYPE, REGISTRY_URL, SUBJECT); + new DebeziumAvroSerializationSchema( + ROW_TYPE, REGISTRY_URL, SUBJECT, registryConfigs); SerializationSchema actualSer = createSerializationSchema(options); Assert.assertEquals(expectedSer, actualSer); } @@ -88,6 +93,8 @@ private Map getAllOptions() { options.put("format", DebeziumAvroFormatFactory.IDENTIFIER); options.put("debezium-avro-confluent.url", REGISTRY_URL); options.put("debezium-avro-confluent.subject", SUBJECT); + options.put("debezium-avro-confluent.basic-auth.user-info", "something1"); + options.put("debezium-avro-confluent.basic-auth.credentials-source", "something2"); return options; } From f08f6e7f913977ac2d947dfb63fc3311c3020886 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ingo=20B=C3=BCrk?= Date: Fri, 30 Jul 2021 08:27:30 +0200 Subject: [PATCH 155/322] [FLINK-23513] Remove format descriptors --- .../formats/json/JsonRowFormatFactory.java | 114 ----------- .../apache/flink/table/descriptors/Json.java | 158 --------------- .../table/descriptors/JsonValidator.java | 71 ------- ....apache.flink.table.factories.TableFactory | 16 -- .../json/JsonRowFormatFactoryTest.java | 181 ------------------ .../flink/table/descriptors/JsonTest.java | 147 -------------- 6 files changed, 687 deletions(-) delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java deleted file mode 100644 index 00506e921..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.descriptors.JsonValidator; -import org.apache.flink.table.factories.DeserializationSchemaFactory; -import org.apache.flink.table.factories.SerializationSchemaFactory; -import org.apache.flink.table.factories.TableFormatFactoryBase; -import org.apache.flink.types.Row; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** - * Table format factory for providing configured instances of JSON-to-row {@link - * SerializationSchema} and {@link DeserializationSchema}. - */ -public class JsonRowFormatFactory extends TableFormatFactoryBase - implements SerializationSchemaFactory, DeserializationSchemaFactory { - - public JsonRowFormatFactory() { - super(JsonValidator.FORMAT_TYPE_VALUE, 1, true); - } - - @Override - protected List supportedFormatProperties() { - final List properties = new ArrayList<>(); - properties.add(JsonValidator.FORMAT_JSON_SCHEMA); - properties.add(JsonValidator.FORMAT_SCHEMA); - properties.add(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD); - properties.add(JsonValidator.FORMAT_IGNORE_PARSE_ERRORS); - return properties; - } - - @Override - public DeserializationSchema createDeserializationSchema(Map properties) { - final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - - // create and configure - final JsonRowDeserializationSchema.Builder schema = - new JsonRowDeserializationSchema.Builder( - createTypeInformation(descriptorProperties)); - - descriptorProperties - .getOptionalBoolean(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD) - .ifPresent( - flag -> { - if (flag) { - schema.failOnMissingField(); - } - }); - descriptorProperties - .getOptionalBoolean(JsonValidator.FORMAT_IGNORE_PARSE_ERRORS) - .ifPresent( - flag -> { - if (flag) { - schema.ignoreParseErrors(); - } - }); - return schema.build(); - } - - @Override - public SerializationSchema createSerializationSchema(Map properties) { - final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - - // create and configure - return new JsonRowSerializationSchema.Builder(createTypeInformation(descriptorProperties)) - .build(); - } - - private TypeInformation createTypeInformation(DescriptorProperties descriptorProperties) { - if (descriptorProperties.containsKey(JsonValidator.FORMAT_SCHEMA)) { - return (RowTypeInfo) descriptorProperties.getType(JsonValidator.FORMAT_SCHEMA); - } else if (descriptorProperties.containsKey(JsonValidator.FORMAT_JSON_SCHEMA)) { - return JsonRowSchemaConverter.convert( - descriptorProperties.getString(JsonValidator.FORMAT_JSON_SCHEMA)); - } else { - return deriveSchema(descriptorProperties.asMap()).toRowType(); - } - } - - private static DescriptorProperties getValidatedProperties(Map propertiesMap) { - final DescriptorProperties descriptorProperties = new DescriptorProperties(); - descriptorProperties.putProperties(propertiesMap); - - // validate - new JsonValidator().validate(descriptorProperties); - - return descriptorProperties; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java deleted file mode 100644 index 7f1d500b2..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/Json.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.descriptors; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.utils.TypeStringUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; - -import java.util.Map; - -import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA; -import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD; -import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_IGNORE_PARSE_ERRORS; -import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_JSON_SCHEMA; -import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_SCHEMA; -import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_TYPE_VALUE; - -/** Format descriptor for JSON. */ -public class Json extends FormatDescriptor { - - private Boolean failOnMissingField; - private Boolean deriveSchema; - private Boolean ignoreParseErrors; - private String jsonSchema; - private String schema; - - /** Format descriptor for JSON. */ - public Json() { - super(FORMAT_TYPE_VALUE, 1); - } - - /** - * Sets flag whether to fail if a field is missing or not. - * - * @param failOnMissingField If set to true, the operation fails if there is a missing field. If - * set to false, a missing field is set to null. - */ - public Json failOnMissingField(boolean failOnMissingField) { - this.failOnMissingField = failOnMissingField; - return this; - } - - /** - * Sets flag whether to fail when parsing json fails. - * - * @param ignoreParseErrors If set to true, the operation will ignore parse errors. If set to - * false, the operation fails when parsing json fails. - */ - public Json ignoreParseErrors(boolean ignoreParseErrors) { - this.ignoreParseErrors = ignoreParseErrors; - return this; - } - - /** - * Sets the JSON schema string with field names and the types according to the JSON schema - * specification [[http://json-schema.org/specification.html]]. - * - *

The schema might be nested. - * - * @param jsonSchema JSON schema - * @deprecated {@link Json} supports derive schema from table schema by default, it is no longer - * necessary to explicitly declare the format schema. This method will be removed in the - * future. - */ - @Deprecated - public Json jsonSchema(String jsonSchema) { - Preconditions.checkNotNull(jsonSchema); - this.jsonSchema = jsonSchema; - this.schema = null; - this.deriveSchema = null; - return this; - } - - /** - * Sets the schema using type information. - * - *

JSON objects are represented as ROW types. - * - *

The schema might be nested. - * - * @param schemaType type information that describes the schema - * @deprecated {@link Json} supports derive schema from table schema by default, it is no longer - * necessary to explicitly declare the format schema. This method will be removed in the - * future. - */ - @Deprecated - public Json schema(TypeInformation schemaType) { - Preconditions.checkNotNull(schemaType); - this.schema = TypeStringUtils.writeTypeInfo(schemaType); - this.jsonSchema = null; - this.deriveSchema = null; - return this; - } - - /** - * Derives the format schema from the table's schema described. - * - *

This allows for defining schema information only once. - * - *

The names, types, and fields' order of the format are determined by the table's schema. - * Time attributes are ignored if their origin is not a field. A "from" definition is - * interpreted as a field renaming in the format. - * - * @deprecated Derivation format schema from table's schema is the default behavior now. So - * there is no need to explicitly declare to derive schema. - */ - @Deprecated - public Json deriveSchema() { - this.deriveSchema = true; - this.schema = null; - this.jsonSchema = null; - return this; - } - - @Override - protected Map toFormatProperties() { - final DescriptorProperties properties = new DescriptorProperties(); - - if (deriveSchema != null) { - properties.putBoolean(FORMAT_DERIVE_SCHEMA, deriveSchema); - } - - if (jsonSchema != null) { - properties.putString(FORMAT_JSON_SCHEMA, jsonSchema); - } - - if (schema != null) { - properties.putString(FORMAT_SCHEMA, schema); - } - - if (failOnMissingField != null) { - properties.putBoolean(FORMAT_FAIL_ON_MISSING_FIELD, failOnMissingField); - } - - if (ignoreParseErrors != null) { - properties.putBoolean(FORMAT_IGNORE_PARSE_ERRORS, ignoreParseErrors); - } - - return properties.asMap(); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java deleted file mode 100644 index 8fcdaf517..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.descriptors; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.api.ValidationException; - -/** Validator for {@link Json}. */ -@Internal -public class JsonValidator extends FormatDescriptorValidator { - - public static final String FORMAT_TYPE_VALUE = "json"; - public static final String FORMAT_SCHEMA = "format.schema"; - public static final String FORMAT_JSON_SCHEMA = "format.json-schema"; - public static final String FORMAT_FAIL_ON_MISSING_FIELD = "format.fail-on-missing-field"; - public static final String FORMAT_IGNORE_PARSE_ERRORS = "format.ignore-parse-errors"; - - @Override - public void validate(DescriptorProperties properties) { - super.validate(properties); - properties.validateBoolean(FORMAT_DERIVE_SCHEMA, true); - final boolean deriveSchema = - properties.getOptionalBoolean(FORMAT_DERIVE_SCHEMA).orElse(true); - final boolean hasSchema = properties.containsKey(FORMAT_SCHEMA); - final boolean hasSchemaString = properties.containsKey(FORMAT_JSON_SCHEMA); - // if a schema is defined, no matter derive schema is set or not, will use the defined - // schema - if (!deriveSchema && hasSchema && hasSchemaString) { - throw new ValidationException( - "A definition of both a schema and JSON schema is not allowed."); - } else if (!deriveSchema && !hasSchema && !hasSchemaString) { - throw new ValidationException( - "A definition of a schema or JSON schema is required " - + "if derivation from table's schema is disabled."); - } else if (hasSchema) { - properties.validateType(FORMAT_SCHEMA, false, true); - } else if (hasSchemaString) { - properties.validateString(FORMAT_JSON_SCHEMA, false, 1); - } - - properties.validateBoolean(FORMAT_FAIL_ON_MISSING_FIELD, true); - properties.validateBoolean(FORMAT_IGNORE_PARSE_ERRORS, true); - boolean failOnMissingField = - properties.getOptionalBoolean(FORMAT_FAIL_ON_MISSING_FIELD).orElse(false); - boolean ignoreParseErrors = - properties.getOptionalBoolean(FORMAT_IGNORE_PARSE_ERRORS).orElse(false); - if (ignoreParseErrors && failOnMissingField) { - throw new ValidationException( - FORMAT_FAIL_ON_MISSING_FIELD - + " and " - + FORMAT_IGNORE_PARSE_ERRORS - + " shouldn't both be true."); - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory deleted file mode 100644 index aec584688..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -org.apache.flink.formats.json.JsonRowFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java deleted file mode 100644 index 5c0ecf640..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.api.Types; -import org.apache.flink.table.descriptors.Descriptor; -import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.descriptors.Json; -import org.apache.flink.table.descriptors.Schema; -import org.apache.flink.table.factories.DeserializationSchemaFactory; -import org.apache.flink.table.factories.SerializationSchemaFactory; -import org.apache.flink.table.factories.TableFactoryService; -import org.apache.flink.types.Row; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.util.Map; - -import static org.junit.Assert.assertEquals; - -/** Tests for the {@link JsonRowFormatFactory}. */ -public class JsonRowFormatFactoryTest extends TestLogger { - - private static final String JSON_SCHEMA = - "{" - + " 'title': 'Fruit'," - + " 'type': 'object'," - + " 'properties': {" - + " 'name': {" - + " 'type': 'string'" - + " }," - + " 'count': {" - + " 'type': 'integer'" - + " }," - + " 'time': {" - + " 'description': 'row time'," - + " 'type': 'string'," - + " 'format': 'date-time'" - + " }" - + " }," - + " 'required': ['name', 'count', 'time']" - + "}"; - - private static final TypeInformation SCHEMA = - Types.ROW( - new String[] {"field1", "field2"}, - new TypeInformation[] {Types.BOOLEAN(), Types.INT()}); - - @Test - public void testSchema() { - final Map properties = - toMap(new Json().schema(SCHEMA).failOnMissingField(false)); - - testSchemaSerializationSchema(properties); - - testSchemaDeserializationSchema(properties); - } - - @Test - public void testSchemaIgnoreParseErrors() { - final Map properties = - toMap(new Json().schema(SCHEMA).ignoreParseErrors(true)); - - testSchemaSerializationSchema(properties); - - final DeserializationSchema actual2 = - TableFactoryService.find(DeserializationSchemaFactory.class, properties) - .createDeserializationSchema(properties); - final JsonRowDeserializationSchema expected2 = - new JsonRowDeserializationSchema.Builder(SCHEMA).ignoreParseErrors().build(); - assertEquals(expected2, actual2); - } - - @Test - public void testJsonSchema() { - final Map properties = - toMap(new Json().jsonSchema(JSON_SCHEMA).failOnMissingField(true)); - - testJsonSchemaSerializationSchema(properties); - - testJsonSchemaDeserializationSchema(properties); - } - - @Test - public void testSchemaDerivation() { - final Map properties = - toMap( - new Schema() - .field("field1", Types.BOOLEAN()) - .field("field2", Types.INT()) - .field("proctime", Types.SQL_TIMESTAMP()) - .proctime(), - new Json().deriveSchema()); - - testSchemaSerializationSchema(properties); - - testSchemaDeserializationSchema(properties); - } - - @Test - public void testSchemaDerivationByDefault() { - final Map properties = - toMap( - new Schema() - .field("field1", Types.BOOLEAN()) - .field("field2", Types.INT()) - .field("proctime", Types.SQL_TIMESTAMP()) - .proctime(), - new Json()); - - testSchemaSerializationSchema(properties); - - testSchemaDeserializationSchema(properties); - } - - private void testSchemaDeserializationSchema(Map properties) { - final DeserializationSchema actual2 = - TableFactoryService.find(DeserializationSchemaFactory.class, properties) - .createDeserializationSchema(properties); - final JsonRowDeserializationSchema expected2 = - new JsonRowDeserializationSchema.Builder(SCHEMA).build(); - assertEquals(expected2, actual2); - } - - private void testSchemaSerializationSchema(Map properties) { - final SerializationSchema actual1 = - TableFactoryService.find(SerializationSchemaFactory.class, properties) - .createSerializationSchema(properties); - final SerializationSchema expected1 = - new JsonRowSerializationSchema.Builder(SCHEMA).build(); - assertEquals(expected1, actual1); - } - - private void testJsonSchemaDeserializationSchema(Map properties) { - final DeserializationSchema actual2 = - TableFactoryService.find(DeserializationSchemaFactory.class, properties) - .createDeserializationSchema(properties); - final JsonRowDeserializationSchema expected2 = - new JsonRowDeserializationSchema.Builder(JSON_SCHEMA).failOnMissingField().build(); - assertEquals(expected2, actual2); - } - - private void testJsonSchemaSerializationSchema(Map properties) { - final SerializationSchema actual1 = - TableFactoryService.find(SerializationSchemaFactory.class, properties) - .createSerializationSchema(properties); - final SerializationSchema expected1 = - JsonRowSerializationSchema.builder() - .withTypeInfo(JsonRowSchemaConverter.convert(JSON_SCHEMA)) - .build(); - assertEquals(expected1, actual1); - } - - private static Map toMap(Descriptor... desc) { - final DescriptorProperties descriptorProperties = new DescriptorProperties(); - for (Descriptor d : desc) { - descriptorProperties.putProperties(d.toProperties()); - } - return descriptorProperties.asMap(); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java deleted file mode 100644 index 89ed2135e..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/table/descriptors/JsonTest.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.descriptors; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.api.Types; -import org.apache.flink.table.api.ValidationException; - -import org.junit.Test; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** Tests for the {@link Json} descriptor. */ -public class JsonTest extends DescriptorTestBase { - - private static final String JSON_SCHEMA = - "{" - + " 'title': 'Person'," - + " 'type': 'object'," - + " 'properties': {" - + " 'firstName': {" - + " 'type': 'string'" - + " }," - + " 'lastName': {" - + " 'type': 'string'" - + " }," - + " 'age': {" - + " 'description': 'Age in years'," - + " 'type': 'integer'," - + " 'minimum': 0" - + " }" - + " }," - + " 'required': ['firstName', 'lastName']" - + "}"; - - @Test(expected = ValidationException.class) - public void testInvalidMissingField() { - addPropertyAndVerify(descriptors().get(0), "format.fail-on-missing-field", "DDD"); - } - - @Test(expected = ValidationException.class) - public void testDuplicateSchema() { - // we add an additional non-json schema - addPropertyAndVerify(descriptors().get(0), "format.schema", "DDD"); - } - - @Test(expected = ValidationException.class) - public void testInvalidIgnoreParseErrors() { - addPropertyAndVerify(descriptors().get(0), "format.ignore-parse-errors", "DDD"); - } - - // -------------------------------------------------------------------------------------------- - - @Override - public List descriptors() { - final Descriptor desc1 = new Json().jsonSchema("test"); - - final Descriptor desc2 = new Json().jsonSchema(JSON_SCHEMA).failOnMissingField(true); - - final Descriptor desc3 = - new Json() - .schema( - Types.ROW( - new String[] {"test1", "test2"}, - new TypeInformation[] { - Types.STRING(), Types.SQL_TIMESTAMP() - })) - .failOnMissingField(true); - - final Descriptor desc4 = new Json().deriveSchema(); - - final Descriptor desc5 = new Json().failOnMissingField(false); - - final Descriptor desc6 = new Json().jsonSchema(JSON_SCHEMA).ignoreParseErrors(false); - - final Descriptor desc7 = new Json().ignoreParseErrors(true); - - return Arrays.asList(desc1, desc2, desc3, desc4, desc5, desc6, desc7); - } - - @Override - public List> properties() { - final Map props1 = new HashMap<>(); - props1.put("format.type", "json"); - props1.put("format.property-version", "1"); - props1.put("format.json-schema", "test"); - - final Map props2 = new HashMap<>(); - props2.put("format.type", "json"); - props2.put("format.property-version", "1"); - props2.put("format.json-schema", JSON_SCHEMA); - props2.put("format.fail-on-missing-field", "true"); - - final Map props3 = new HashMap<>(); - props3.put("format.type", "json"); - props3.put("format.property-version", "1"); - props3.put("format.schema", "ROW"); - props3.put("format.fail-on-missing-field", "true"); - - final Map props4 = new HashMap<>(); - props4.put("format.type", "json"); - props4.put("format.property-version", "1"); - props4.put("format.derive-schema", "true"); - - final Map props5 = new HashMap<>(); - props5.put("format.type", "json"); - props5.put("format.property-version", "1"); - props5.put("format.fail-on-missing-field", "false"); - - final Map props6 = new HashMap<>(); - props6.put("format.type", "json"); - props6.put("format.property-version", "1"); - props6.put("format.json-schema", JSON_SCHEMA); - props6.put("format.ignore-parse-errors", "false"); - - final Map props7 = new HashMap<>(); - props7.put("format.type", "json"); - props7.put("format.property-version", "1"); - props7.put("format.ignore-parse-errors", "true"); - - return Arrays.asList(props1, props2, props3, props4, props5, props6, props7); - } - - @Override - public DescriptorValidator validator() { - return new JsonValidator(); - } -} From 54ab781261a3c5d45d9bc4fc56c5e71de79c698e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ingo=20B=C3=BCrk?= Date: Wed, 4 Aug 2021 10:46:06 +0200 Subject: [PATCH 156/322] [FLINK-23192][table] Move connector & format options to consistent packages We should advertise the following package structure: For connector options: org.apache.flink.connector.XX.table.XXConnectorOptions For format options: org.apache.flink.formats.XX.XXFormatOptions [FLINK-23192][connector-hbase] Move HBase table connector options class [FLINK-23192][table] Move DataGen connector to a proper package [FLINK-23192][table] Move print sink and split config options [FLINK-23192][table] Move blackhole connector [FLINK-23192][table] Move raw format [FLINK-23192][formats] Mark factories as internal This closes #16708. --- .../avro/registry/confluent/RegistryAvroFormatFactory.java | 2 ++ .../registry/confluent/debezium/DebeziumAvroFormatFactory.java | 2 ++ .../java/org/apache/flink/formats/json/JsonFormatFactory.java | 2 ++ .../apache/flink/formats/json/canal/CanalJsonFormatFactory.java | 2 ++ .../flink/formats/json/debezium/DebeziumJsonFormatFactory.java | 2 ++ .../flink/formats/json/maxwell/MaxwellJsonFormatFactory.java | 2 ++ 6 files changed, 12 insertions(+) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index 2b37e3649..c1d0ff814 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.avro.registry.confluent; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -66,6 +67,7 @@ * Table format factory for providing configured instances of Schema Registry Avro to RowData {@link * SerializationSchema} and {@link DeserializationSchema}. */ +@Internal public class RegistryAvroFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java index dbb079049..eaf069a9d 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.avro.registry.confluent.debezium; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -60,6 +61,7 @@ * Format factory for providing configured instances of Debezium Avro to RowData {@link * DeserializationSchema}. */ +@Internal public class DebeziumAvroFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index 5d57f422e..692973f5a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -52,6 +53,7 @@ * Table format factory for providing configured instances of JSON to RowData {@link * SerializationSchema} and {@link DeserializationSchema}. */ +@Internal public class JsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { public static final String IDENTIFIER = "json"; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java index 0da3419fe..9f3ccad2d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json.canal; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigOption; @@ -54,6 +55,7 @@ * Format factory for providing configured instances of Canal JSON to RowData {@link * DeserializationSchema}. */ +@Internal public class CanalJsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java index d261cb30c..d72fcd23d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json.debezium; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigOption; @@ -54,6 +55,7 @@ * Format factory for providing configured instances of Debezium JSON to RowData {@link * DeserializationSchema}. */ +@Internal public class DebeziumJsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java index d37222bcc..1bbbec844 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json.maxwell; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigOption; @@ -52,6 +53,7 @@ * Format factory for providing configured instances of Maxwell JSON to RowData {@link * DeserializationSchema}. */ +@Internal public class MaxwellJsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { From 918235ff3980dd2fa518057c1e3ed78b6f2ded3f Mon Sep 17 00:00:00 2001 From: hapihu Date: Wed, 18 Aug 2021 17:53:00 +0800 Subject: [PATCH 157/322] [FLINK-23044][docs] Fix typos --- docs/content/docs/connectors/table/formats/avro-confluent.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/docs/connectors/table/formats/avro-confluent.md b/docs/content/docs/connectors/table/formats/avro-confluent.md index 219b35599..7ab4c988a 100644 --- a/docs/content/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content/docs/connectors/table/formats/avro-confluent.md @@ -122,7 +122,7 @@ CREATE TABLE user_created ( 'value.avro-confluent.url' = 'http://localhost:8082', 'value.fields-include' = 'EXCEPT_KEY', - -- subjects have a default value since Flink 1.13, though can be overriden: + -- subjects have a default value since Flink 1.13, though can be overridden: 'key.avro-confluent.subject' = 'user_events_example2-key2', 'value.avro-confluent.subject' = 'user_events_example2-value2' ) From dcae1cc50f09d18278921d928d6dd92f4cccbcdd Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Sat, 28 Aug 2021 08:33:55 +0800 Subject: [PATCH 158/322] Update version to 1.15-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 2d36f9e03..ae8751480 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.14-SNAPSHOT + 1.15-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index d7c1be920..548998c6d 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.14-SNAPSHOT + 1.15-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 9c230c500..3a37040f4 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.14-SNAPSHOT + 1.15-SNAPSHOT .. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index 0f0e64eff..e76c3bac2 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.14-SNAPSHOT + 1.15-SNAPSHOT .. From ec97614a8c3d493e10c1ee427879f3f7f9ebcc33 Mon Sep 17 00:00:00 2001 From: MartijnVisser Date: Wed, 8 Sep 2021 13:23:09 +0200 Subject: [PATCH 159/322] [FLINK-24034] Upgrade apache commons dependencies --- .../src/main/resources/META-INF/NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 92f02871e..d5f18bd4d 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -10,7 +10,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-core:2.12.1 - com.fasterxml.jackson.core:jackson-databind:2.12.1 - com.fasterxml.jackson.core:jackson-annotations:2.12.1 -- org.apache.commons:commons-compress:1.20 +- org.apache.commons:commons-compress:1.21 - io.confluent:kafka-schema-registry-client:5.5.2 - org.apache.kafka:kafka-clients:5.5.2-ccs - io.confluent:common-config:5.5.2 From bd82131cc7237e059d6302f1ec3a7f8a6e00a40f Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Wed, 25 Aug 2021 11:13:26 +0200 Subject: [PATCH 160/322] [FLINK-23607][state/changelog] Make Changelog Backend a transitive dependency ...of flink-test-utils instead of direct dependency of each module. --- flink-formats-kafka/flink-json-debezium/pom.xml | 7 ------- 1 file changed, 7 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 3a37040f4..16d0b2717 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -64,13 +64,6 @@ under the License. provided - - org.apache.flink - flink-statebackend-changelog - ${project.version} - test - - From 6561133837b5f131e6e69d248b9605df66bb541e Mon Sep 17 00:00:00 2001 From: Fabian Paul Date: Tue, 14 Sep 2021 16:04:25 +0200 Subject: [PATCH 161/322] [FLINK-24281][connectors/kafka] Migrate all format tests from FlinkKafkaProducer to KafkaSink --- .../test/TestAvroConsumerConfluent.java | 48 ++++++++++++------- 1 file changed, 32 insertions(+), 16 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java index 840aa9e52..e3f414978 100644 --- a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java +++ b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java @@ -20,16 +20,18 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; +import org.apache.flink.connector.kafka.sink.KafkaSink; import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema; import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; import example.avro.User; import org.apache.avro.specific.SpecificRecordBase; +import org.apache.kafka.clients.producer.ProducerConfig; import java.util.Properties; @@ -74,22 +76,36 @@ public static void main(String[] args) throws Exception { SingleOutputStreamOperator mapToString = input.map((MapFunction) SpecificRecordBase::toString); - FlinkKafkaProducer stringFlinkKafkaProducer = - new FlinkKafkaProducer<>( - parameterTool.getRequired("output-string-topic"), - new SimpleStringSchema(), - config); - mapToString.addSink(stringFlinkKafkaProducer); + KafkaSink stringSink = + KafkaSink.builder() + .setBootstrapServers( + config.getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) + .setRecordSerializer( + KafkaRecordSerializationSchema.builder() + .setValueSerializationSchema(new SimpleStringSchema()) + .setTopic(parameterTool.getRequired("output-string-topic")) + .build()) + .setKafkaProducerConfig(config) + .build(); + mapToString.sinkTo(stringSink); - FlinkKafkaProducer avroFlinkKafkaProducer = - new FlinkKafkaProducer<>( - parameterTool.getRequired("output-avro-topic"), - ConfluentRegistryAvroSerializationSchema.forSpecific( - User.class, - parameterTool.getRequired("output-subject"), - schemaRegistryUrl), - config); - input.addSink(avroFlinkKafkaProducer); + KafkaSink avroSink = + KafkaSink.builder() + .setBootstrapServers( + config.getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) + .setRecordSerializer( + KafkaRecordSerializationSchema.builder() + .setValueSerializationSchema( + ConfluentRegistryAvroSerializationSchema + .forSpecific( + User.class, + parameterTool.getRequired( + "output-subject"), + schemaRegistryUrl)) + .setTopic(parameterTool.getRequired("output-avro-topic")) + .build()) + .build(); + input.sinkTo(avroSink); env.execute("Kafka Confluent Schema Registry AVRO Example"); } From e6d1ef62fe6f70f247f86f101c330f75869b2258 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 26 Oct 2021 00:48:06 +0200 Subject: [PATCH 162/322] [FLINK-24018][build] Remove Scala dependencies from Java APIs --- flink-confluent-schema-registry-e2e-tests/pom.xml | 4 ++-- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index ae8751480..55fbbf2bb 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -43,14 +43,14 @@ under the License. org.apache.flink - flink-streaming-java_${scala.binary.version} + flink-streaming-java ${project.version} provided org.apache.flink - flink-connector-kafka_${scala.binary.version} + flink-connector-kafka ${project.version} diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 16d0b2717..fca8d9b54 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -95,7 +95,7 @@ under the License. org.apache.flink - flink-test-utils_${scala.binary.version} + flink-test-utils ${project.version} test From b01a973739bbfd004bcbb6b74a92461b47773de0 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Fri, 22 Oct 2021 18:17:06 +0200 Subject: [PATCH 163/322] [FLINK-24599][table] Replace static methods with convenient methods in LogicalType Make the code of calling `hasRoot`/`hasFamily` less verbose by replacing the static utility methods with member methods `LogicalType#is()`. This closes #17550. --- .../org/apache/flink/formats/json/JsonToRowDataConverters.java | 3 +-- .../org/apache/flink/formats/json/RowDataToJsonConverters.java | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java index f56897d20..c89b5f583 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java @@ -36,7 +36,6 @@ import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.MultisetType; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; import org.apache.flink.table.types.logical.utils.LogicalTypeUtils; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; @@ -310,7 +309,7 @@ private JsonToRowDataConverter createArrayConverter(ArrayType arrayType) { private JsonToRowDataConverter createMapConverter( String typeSummary, LogicalType keyType, LogicalType valueType) { - if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { + if (!keyType.is(LogicalTypeFamily.CHARACTER_STRING)) { throw new UnsupportedOperationException( "JSON format doesn't support non-string as key type of map. " + "The type is: " diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java index 302f6d38a..2a1cd076e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java @@ -33,7 +33,6 @@ import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.MultisetType; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -249,7 +248,7 @@ private RowDataToJsonConverter createArrayConverter(ArrayType type) { private RowDataToJsonConverter createMapConverter( String typeSummary, LogicalType keyType, LogicalType valueType) { - if (!LogicalTypeChecks.hasFamily(keyType, LogicalTypeFamily.CHARACTER_STRING)) { + if (!keyType.is(LogicalTypeFamily.CHARACTER_STRING)) { throw new UnsupportedOperationException( "JSON format doesn't support non-string as key type of map. " + "The type is: " From 20cc1d88171e7e737422734ee65366c8b2d5e0c6 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Thu, 28 Oct 2021 11:41:04 +0200 Subject: [PATCH 164/322] [FLINK-24673][formats] Deprecate old Row SerializationSchema/DeserializationSchema Signed-off-by: slinkydeveloper This closes #17589. --- .../flink/formats/json/JsonRowDeserializationSchema.java | 5 +++++ .../flink/formats/json/JsonRowSerializationSchema.java | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index 6cfd84c1a..a337a78af 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -77,8 +77,13 @@ *

Deserializes a byte[] message as a JSON object and reads the specified fields. * *

Failures during deserialization are forwarded as wrapped IOExceptions. + * + * @deprecated The format was developed for the Table API users and will not be maintained for + * DataStream API users anymore. Either use Table API or switch to Data Stream, defining your + * own {@link DeserializationSchema}. */ @PublicEvolving +@Deprecated public class JsonRowDeserializationSchema implements DeserializationSchema { private static final long serialVersionUID = -228294330688809195L; diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index 607d4acc8..3dd8be1fd 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -64,8 +64,13 @@ * *

Result byte[] messages can be deserialized using {@link * JsonRowDeserializationSchema}. + * + * @deprecated The format was developed for the Table API users and will not be maintained for + * DataStream API users anymore. Either use Table API or switch to Data Stream, defining your + * own {@link SerializationSchema}. */ @PublicEvolving +@Deprecated public class JsonRowSerializationSchema implements SerializationSchema { private static final long serialVersionUID = -2885556750743978636L; From 13a86e6266ba651671a84dc69db7b1f8581025ff Mon Sep 17 00:00:00 2001 From: martijnvisser Date: Wed, 3 Nov 2021 09:26:29 +0100 Subject: [PATCH 165/322] [FLINK-24715][connectors][filesystems][formats] Update multiple Jackson dependencies to latest version --- .../src/main/resources/META-INF/NOTICE | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index d5f18bd4d..ddbec159d 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -7,9 +7,9 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - org.apache.avro:avro:1.10.0 -- com.fasterxml.jackson.core:jackson-core:2.12.1 -- com.fasterxml.jackson.core:jackson-databind:2.12.1 -- com.fasterxml.jackson.core:jackson-annotations:2.12.1 +- com.fasterxml.jackson.core:jackson-core:2.13.0 +- com.fasterxml.jackson.core:jackson-databind:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.0 - org.apache.commons:commons-compress:1.21 - io.confluent:kafka-schema-registry-client:5.5.2 - org.apache.kafka:kafka-clients:5.5.2-ccs From 9aa0925e51ca1a2b84824146cf8f10bbfff744c3 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Thu, 11 Nov 2021 10:53:43 +0100 Subject: [PATCH 166/322] [FLINK-24776][table] Clarify DecodingFormat and introduce ProjectableDecodingFormat Clearly separates projectable formats from non-projectable formats. Before this, the semantics were not 100% clear which led to inconsistent connector and format implementations. The FileSystemTableSource has been updated to distinguish between those two interfaces now. Users that implemented custom formats for FileSystemTableSource might need to verify the implementation. For convienience, we introduces helper classes such as ProjectedRowData and Projection to ease the implementation. This closes #17768. --- .../registry/confluent/RegistryAvroFormatFactory.java | 8 ++++++-- .../confluent/debezium/DebeziumAvroFormatFactory.java | 8 ++++++-- .../org/apache/flink/formats/json/JsonFormatFactory.java | 9 +++++++-- .../formats/json/canal/CanalJsonDecodingFormat.java | 7 +++++-- .../json/debezium/DebeziumJsonDecodingFormat.java | 7 +++++-- .../formats/json/maxwell/MaxwellJsonDecodingFormat.java | 8 ++++++-- 6 files changed, 35 insertions(+), 12 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index c1d0ff814..d4771fe3e 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -33,6 +33,7 @@ import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.format.ProjectableDecodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; @@ -81,10 +82,13 @@ public DecodingFormat> createDecodingFormat( String schemaRegistryURL = formatOptions.get(URL); Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); - return new DecodingFormat>() { + return new ProjectableDecodingFormat>() { @Override public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType producedDataType) { + DynamicTableSource.Context context, + DataType producedDataType, + int[][] projections) { + producedDataType = DataType.projectFields(producedDataType, projections); final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = context.createTypeInformation(producedDataType); diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java index eaf069a9d..d889dee03 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java @@ -28,6 +28,7 @@ import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.format.ProjectableDecodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; @@ -75,10 +76,13 @@ public DecodingFormat> createDecodingFormat( String schemaRegistryURL = formatOptions.get(URL); Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); - return new DecodingFormat>() { + return new ProjectableDecodingFormat>() { @Override public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType producedDataType) { + DynamicTableSource.Context context, + DataType producedDataType, + int[][] projections) { + producedDataType = DataType.projectFields(producedDataType, projections); final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation producedTypeInfo = context.createTypeInformation(producedDataType); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index 692973f5a..b78c0240e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -28,6 +28,7 @@ import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.format.ProjectableDecodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; @@ -68,10 +69,14 @@ public DecodingFormat> createDecodingFormat( final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); TimestampFormat timestampOption = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - return new DecodingFormat>() { + return new ProjectableDecodingFormat>() { @Override public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType producedDataType) { + DynamicTableSource.Context context, + DataType physicalDataType, + int[][] projections) { + final DataType producedDataType = + DataType.projectFields(physicalDataType, projections); final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = context.createTypeInformation(producedDataType); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java index f958c8d38..76f4e2eae 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java @@ -25,6 +25,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.ProjectableDecodingFormat; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; @@ -43,7 +44,8 @@ import java.util.stream.Stream; /** {@link DecodingFormat} for Canal using JSON encoding. */ -public class CanalJsonDecodingFormat implements DecodingFormat> { +public class CanalJsonDecodingFormat + implements ProjectableDecodingFormat> { // -------------------------------------------------------------------------------------------- // Mutable attributes @@ -77,7 +79,8 @@ public CanalJsonDecodingFormat( @Override public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType physicalDataType) { + DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) { + physicalDataType = DataType.projectFields(physicalDataType, projections); final List readableMetadata = metadataKeys.stream() .map( diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java index 5db2f64d7..472596a7c 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java @@ -25,6 +25,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.ProjectableDecodingFormat; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.GenericMapData; import org.apache.flink.table.data.GenericRowData; @@ -43,7 +44,8 @@ import java.util.stream.Stream; /** {@link DecodingFormat} for Debezium using JSON encoding. */ -public class DebeziumJsonDecodingFormat implements DecodingFormat> { +public class DebeziumJsonDecodingFormat + implements ProjectableDecodingFormat> { // -------------------------------------------------------------------------------------------- // Mutable attributes @@ -71,7 +73,8 @@ public DebeziumJsonDecodingFormat( @Override public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType physicalDataType) { + DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) { + physicalDataType = DataType.projectFields(physicalDataType, projections); final List readableMetadata = metadataKeys.stream() diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java index d0403863e..8aae68f0a 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java @@ -25,6 +25,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.ProjectableDecodingFormat; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; @@ -41,7 +42,8 @@ import java.util.stream.Stream; /** {@link DecodingFormat} for Maxwell using JSON encoding. */ -public class MaxwellJsonDecodingFormat implements DecodingFormat> { +public class MaxwellJsonDecodingFormat + implements ProjectableDecodingFormat> { // -------------------------------------------------------------------------------------------- // Mutable attributes @@ -62,7 +64,9 @@ public MaxwellJsonDecodingFormat(boolean ignoreParseErrors, TimestampFormat time @Override public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType physicalDataType) { + DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) { + physicalDataType = DataType.projectFields(physicalDataType, projections); + final List readableMetadata = metadataKeys.stream() .map( From 868da97d80a34bd8fa632acc132e186f45ebffce Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 25 Nov 2021 11:55:32 +0100 Subject: [PATCH 167/322] [FLINK-25039][build] Disable shading of test jar by default --- flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index e76c3bac2..b00aa61a8 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -55,7 +55,6 @@ under the License. shade - false io.confluent:* From d50867d5d1745e4d6b2ac21b56dca4ca9e1dad49 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Thu, 25 Nov 2021 10:49:38 +0100 Subject: [PATCH 168/322] [FLINK-25060][table-common] Replace projection methods of FLINK-24399 with the new Projection util This closes #17906. --- .../avro/registry/confluent/RegistryAvroFormatFactory.java | 3 ++- .../registry/confluent/debezium/DebeziumAvroFormatFactory.java | 3 ++- .../java/org/apache/flink/formats/json/JsonFormatFactory.java | 3 ++- .../flink/formats/json/canal/CanalJsonDecodingFormat.java | 3 ++- .../formats/json/debezium/DebeziumJsonDecodingFormat.java | 3 ++- .../flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java | 3 ++- 6 files changed, 12 insertions(+), 6 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index d4771fe3e..659160b77 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -31,6 +31,7 @@ import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.format.ProjectableDecodingFormat; @@ -88,7 +89,7 @@ public DeserializationSchema createRuntimeDecoder( DynamicTableSource.Context context, DataType producedDataType, int[][] projections) { - producedDataType = DataType.projectFields(producedDataType, projections); + producedDataType = Projection.of(projections).project(producedDataType); final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = context.createTypeInformation(producedDataType); diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java index d889dee03..c9773dcf4 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java @@ -26,6 +26,7 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.format.ProjectableDecodingFormat; @@ -82,7 +83,7 @@ public DeserializationSchema createRuntimeDecoder( DynamicTableSource.Context context, DataType producedDataType, int[][] projections) { - producedDataType = DataType.projectFields(producedDataType, projections); + producedDataType = Projection.of(projections).project(producedDataType); final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation producedTypeInfo = context.createTypeInformation(producedDataType); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index b78c0240e..09322e010 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -26,6 +26,7 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.format.ProjectableDecodingFormat; @@ -76,7 +77,7 @@ public DeserializationSchema createRuntimeDecoder( DataType physicalDataType, int[][] projections) { final DataType producedDataType = - DataType.projectFields(physicalDataType, projections); + Projection.of(projections).project(physicalDataType); final RowType rowType = (RowType) producedDataType.getLogicalType(); final TypeInformation rowDataTypeInfo = context.createTypeInformation(producedDataType); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java index 76f4e2eae..4c981d5a3 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java @@ -24,6 +24,7 @@ import org.apache.flink.formats.json.canal.CanalJsonDeserializationSchema.MetadataConverter; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.ProjectableDecodingFormat; import org.apache.flink.table.connector.source.DynamicTableSource; @@ -80,7 +81,7 @@ public CanalJsonDecodingFormat( @Override public DeserializationSchema createRuntimeDecoder( DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) { - physicalDataType = DataType.projectFields(physicalDataType, projections); + physicalDataType = Projection.of(projections).project(physicalDataType); final List readableMetadata = metadataKeys.stream() .map( diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java index 472596a7c..706d01a81 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java @@ -24,6 +24,7 @@ import org.apache.flink.formats.json.debezium.DebeziumJsonDeserializationSchema.MetadataConverter; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.ProjectableDecodingFormat; import org.apache.flink.table.connector.source.DynamicTableSource; @@ -74,7 +75,7 @@ public DebeziumJsonDecodingFormat( @Override public DeserializationSchema createRuntimeDecoder( DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) { - physicalDataType = DataType.projectFields(physicalDataType, projections); + physicalDataType = Projection.of(projections).project(physicalDataType); final List readableMetadata = metadataKeys.stream() diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java index 8aae68f0a..71d1ee9cc 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java @@ -24,6 +24,7 @@ import org.apache.flink.formats.json.maxwell.MaxwellJsonDeserializationSchema.MetadataConverter; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.ProjectableDecodingFormat; import org.apache.flink.table.connector.source.DynamicTableSource; @@ -65,7 +66,7 @@ public MaxwellJsonDecodingFormat(boolean ignoreParseErrors, TimestampFormat time @Override public DeserializationSchema createRuntimeDecoder( DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) { - physicalDataType = DataType.projectFields(physicalDataType, projections); + physicalDataType = Projection.of(projections).project(physicalDataType); final List readableMetadata = metadataKeys.stream() From 90ddb4f4af87fe4718dddc4a2af3fb0911b5ad0b Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Wed, 24 Nov 2021 15:13:07 +0100 Subject: [PATCH 169/322] [FLINK-24687][table][connectors] Move FileSystemTableSink, FileSystemTableSource to flink-connector-files and columnar support to flink-table-common Now table packages don't depend on flink-connector-files anymore. Fix orc and parquet format to use only common classes and not planner nor runtime classes. - [connector-files] Add @Internal to all public classes and interfaces - [orc][parquet][hive] Drop scala suffix from flink-orc and flink-parquet - [architecture-tests] Updated the violations file - [connector-elasticsearch-base] Add flink-connector-base as dependency, which was previously brought in through flink-table-api-java-bridge -> flink-table-api-java -> flink-table-common -> flink-connector-files -> flink-connector-base. - [orc][parquet] Add issue link for partition keys handling - [table-uber][dist] Now flink-connector-files is not shaded inside table-uber anymore but it's loaded in /lib in the distribution as flink-connector-files - [docs] Update sql_connectors.yml This closes #17897. --- .../flink-avro-confluent-registry/pom.xml | 12 ++++++++++++ flink-formats-kafka/flink-json-debezium/pom.xml | 11 +++++++++++ 2 files changed, 23 insertions(+) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 548998c6d..59fc1d1f4 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -84,11 +84,23 @@ under the License. ${project.version} provided + + + org.apache.flink flink-table-common ${project.version} provided + true + + + + org.apache.flink + flink-connector-files + ${project.version} + provided + true diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index fca8d9b54..0e60961e0 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -57,11 +57,22 @@ under the License. provided + + org.apache.flink flink-table-common ${project.version} provided + true + + + + org.apache.flink + flink-connector-files + ${project.version} + provided + true From d084a7c99c6314c8b6a5444933d5aaf28af051e2 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Wed, 1 Dec 2021 14:54:36 +0100 Subject: [PATCH 170/322] [FLINK-25114][table-runtime] Remove flink-scala dependency and scala suffix This closes #18011. --- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 59fc1d1f4..6dda87354 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -113,7 +113,7 @@ under the License. org.apache.flink - flink-table-runtime_${scala.binary.version} + flink-table-runtime ${project.version} test From 862df07619f6f38976a3b27e2712d274758ce0db Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Thu, 9 Dec 2021 14:55:05 +0100 Subject: [PATCH 171/322] [FLINK-25229][table] Introduce flink-table-api-bridge-base This closes #18065. --- flink-formats-kafka/flink-json-debezium/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 0e60961e0..a04294917 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -95,6 +95,12 @@ under the License. + + org.apache.flink + flink-table-api-scala-bridge_${scala.binary.version} + ${project.version} + test + org.apache.flink flink-table-planner_${scala.binary.version} From ede46ea832679253eed000639b3a3aac9274a7f8 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 17 Dec 2021 11:50:06 +0100 Subject: [PATCH 172/322] [FLINK-25362][docs] fix maven instructions for "Confluent Avro Format" --- docs/content.zh/docs/connectors/table/formats/avro-confluent.md | 2 ++ docs/content/docs/connectors/table/formats/avro-confluent.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md index 61248d185..ddb3c0880 100644 --- a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md @@ -44,6 +44,8 @@ Avro Schema Registry 格式只能与 [Apache Kafka SQL 连接器]({{< ref "docs/ {{< sql_download_table "avro-confluent" >}} +For Maven, SBT, Gradle, or other build automation tools, please also ensure that Confluent's maven repository at `https://packages.confluent.io/maven/` is configured in your project's build files. + 如何创建使用 Avro-Confluent 格式的表 ---------------- diff --git a/docs/content/docs/connectors/table/formats/avro-confluent.md b/docs/content/docs/connectors/table/formats/avro-confluent.md index 7ab4c988a..cf2fe7023 100644 --- a/docs/content/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content/docs/connectors/table/formats/avro-confluent.md @@ -42,6 +42,8 @@ Dependencies {{< sql_download_table "avro-confluent" >}} +For Maven, SBT, Gradle, or other build automation tools, please also ensure that Confluent's maven repository at `https://packages.confluent.io/maven/` is configured in your project's build files. + How to create tables with Avro-Confluent format -------------- From ebaa1695941c5f4c024dfd8fb577ff2d0e575a5d Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Thu, 6 Jan 2022 11:05:31 +0100 Subject: [PATCH 173/322] [FLINK-25390][connector-kafka][json] Show usage of helper to forward and merge options This closes #18283. --- .../flink/formats/json/JsonFormatFactory.java | 11 ++ .../formats/json/JsonFormatFactoryTest.java | 137 ++++++++++-------- 2 files changed, 91 insertions(+), 57 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index 09322e010..bf2e2878b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -151,4 +151,15 @@ public Set> optionalOptions() { options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); return options; } + + @Override + public Set> forwardOptions() { + Set> options = new HashSet<>(); + options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); + options.add(MAP_NULL_KEY_MODE); + options.add(MAP_NULL_KEY_LITERAL); + options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); + return options; + } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index 8e39a3163..e22a67aeb 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -26,37 +26,33 @@ import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.factories.utils.FactoryMocks; import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.util.TestLogger; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.assertj.core.api.AbstractThrowableAssert; +import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link JsonFormatFactory}. */ -public class JsonFormatFactoryTest extends TestLogger { - @Rule public ExpectedException thrown = ExpectedException.none(); +public class JsonFormatFactoryTest { @Test public void testSeDeSchema() { final Map tableOptions = getAllOptions(); testSchemaSerializationSchema(tableOptions); - testSchemaDeserializationSchema(tableOptions); } @@ -65,12 +61,11 @@ public void testFailOnMissingField() { final Map tableOptions = getModifyOptions(options -> options.put("json.fail-on-missing-field", "true")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "fail-on-missing-field and ignore-parse-errors shouldn't both be true."))); - testSchemaDeserializationSchema(tableOptions); + assertThatCreateRuntimeDecoder(tableOptions) + .satisfies( + anyCauseMatches( + ValidationException.class, + "fail-on-missing-field and ignore-parse-errors shouldn't both be true.")); } @Test @@ -78,12 +73,11 @@ public void testInvalidOptionForIgnoreParseErrors() { final Map tableOptions = getModifyOptions(options -> options.put("json.ignore-parse-errors", "abc")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new IllegalArgumentException( - "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); - testSchemaDeserializationSchema(tableOptions); + assertThatCreateRuntimeDecoder(tableOptions) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)")); } @Test @@ -91,12 +85,11 @@ public void testInvalidOptionForTimestampFormat() { final Map tableOptions = getModifyOptions(options -> options.put("json.timestamp-format.standard", "test")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - testSchemaDeserializationSchema(tableOptions); + assertThatCreateRuntimeDecoder(tableOptions) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601].")); } @Test @@ -105,12 +98,11 @@ public void testLowerCaseOptionForTimestampFormat() { getModifyOptions( options -> options.put("json.timestamp-format.standard", "iso-8601")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'iso-8601' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - testSchemaDeserializationSchema(tableOptions); + assertThatCreateRuntimeDecoder(tableOptions) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'iso-8601' for timestamp-format.standard. Supported values are [SQL, ISO-8601].")); } @Test @@ -118,12 +110,11 @@ public void testInvalidOptionForMapNullKeyMode() { final Map tableOptions = getModifyOptions(options -> options.put("json.map-null-key.mode", "invalid")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - testSchemaSerializationSchema(tableOptions); + assertThatCreateRuntimeEncoder(tableOptions) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP].")); } @Test @@ -138,6 +129,27 @@ public void testLowerCaseOptionForMapNullKeyMode() { // Utilities // ------------------------------------------------------------------------ + private AbstractThrowableAssert assertThatCreateRuntimeDecoder( + Map options) { + return assertThatThrownBy( + () -> + createTableSource(options) + .valueFormat + .createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toPhysicalRowDataType())); + } + + private AbstractThrowableAssert assertThatCreateRuntimeEncoder( + Map options) { + return assertThatThrownBy( + () -> + createTableSink(options) + .valueFormat + .createRuntimeEncoder( + new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE)); + } + private void testSchemaDeserializationSchema(Map options) { final JsonRowDataDeserializationSchema expectedDeser = new JsonRowDataDeserializationSchema( @@ -147,16 +159,14 @@ private void testSchemaDeserializationSchema(Map options) { true, TimestampFormat.ISO_8601); - final DynamicTableSource actualSource = createTableSource(SCHEMA, options); - assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - DeserializationSchema actualDeser = - scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); + createTableSource(options) + .valueFormat + .createRuntimeDecoder( + ScanRuntimeProviderContext.INSTANCE, + SCHEMA.toPhysicalRowDataType()); - assertEquals(expectedDeser, actualDeser); + assertThat(actualDeser).isEqualTo(expectedDeser); } private void testSchemaSerializationSchema(Map options) { @@ -168,16 +178,29 @@ private void testSchemaSerializationSchema(Map options) { "null", true); - final DynamicTableSink actualSink = createTableSink(SCHEMA, options); - assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - SerializationSchema actualSer = - sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); + createTableSink(options) + .valueFormat + .createRuntimeEncoder( + new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); + + assertThat(actualSer).isEqualTo(expectedSer); + } + + private TestDynamicTableFactory.DynamicTableSinkMock createTableSink( + Map options) { + final DynamicTableSink actualSink = FactoryMocks.createTableSink(SCHEMA, options); + assertThat(actualSink).isInstanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class); + + return (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + } + + private TestDynamicTableFactory.DynamicTableSourceMock createTableSource( + Map options) { + final DynamicTableSource actualSource = FactoryMocks.createTableSource(SCHEMA, options); + assertThat(actualSource).isInstanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class); - assertEquals(expectedSer, actualSer); + return (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; } /** From f5ca5ea58184eaa62e2da82850df35d38a815921 Mon Sep 17 00:00:00 2001 From: martijnvisser Date: Thu, 13 Jan 2022 13:27:40 +0100 Subject: [PATCH 174/322] [FLINK-25504][Kafka] Upgrade Confluent to v6.2.2 and Kafka-Client to 2.8.1 --- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 4 ++-- .../src/main/resources/META-INF/NOTICE | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 6dda87354..4b791287a 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -32,8 +32,8 @@ under the License. Flink : Formats : Avro confluent registry - 2.4.1 - 5.5.2 + 2.8.1 + 6.2.2 diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index ddbec159d..bfddc9b0f 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -11,10 +11,10 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-databind:2.13.0 - com.fasterxml.jackson.core:jackson-annotations:2.13.0 - org.apache.commons:commons-compress:1.21 -- io.confluent:kafka-schema-registry-client:5.5.2 -- org.apache.kafka:kafka-clients:5.5.2-ccs -- io.confluent:common-config:5.5.2 -- io.confluent:common-utils:5.5.2 +- io.confluent:kafka-schema-registry-client:6.2.2 +- org.apache.kafka:kafka-clients:6.2.2-ccs +- io.confluent:common-config:6.2.2 +- io.confluent:common-utils:6.2.2 - org.glassfish.jersey.core:jersey-common:2.30 The binary distribution of this product bundles these dependencies under the Eclipse Public License - v 2.0 (https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt) From ca7413a9da0dbfa5796cff2d95b44bc378825289 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Thu, 6 Jan 2022 16:37:17 +0100 Subject: [PATCH 175/322] [FLINK-25391][format-avro] Forward catalog table options --- .../table/formats/avro-confluent.md | 15 ++++++++++++++- .../confluent/RegistryAvroFormatFactory.java | 19 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/docs/content/docs/connectors/table/formats/avro-confluent.md b/docs/content/docs/connectors/table/formats/avro-confluent.md index cf2fe7023..28b33da71 100644 --- a/docs/content/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content/docs/connectors/table/formats/avro-confluent.md @@ -176,15 +176,17 @@ Format Options Option Required + Forwarded Default Type - Description + Description

format
required + no (none) String Specify what format to use, here should be 'avro-confluent'. @@ -192,6 +194,7 @@ Format Options
avro-confluent.basic-auth.credentials-source
optional + yes (none) String Basic auth credentials source for Schema Registry @@ -199,6 +202,7 @@ Format Options
avro-confluent.basic-auth.user-info
optional + yes (none) String Basic auth user info for schema registry @@ -206,6 +210,7 @@ Format Options
avro-confluent.bearer-auth.credentials-source
optional + yes (none) String Bearer auth credentials source for Schema Registry @@ -213,6 +218,7 @@ Format Options
avro-confluent.bearer-auth.token
optional + yes (none) String Bearer auth token for Schema Registry @@ -220,6 +226,7 @@ Format Options
avro-confluent.properties
optional + yes (none) Map Properties map that is forwarded to the underlying Schema Registry. This is useful for options that are not officially exposed via Flink config options. However, note that Flink options have higher precedence. @@ -227,6 +234,7 @@ Format Options
avro-confluent.ssl.keystore.location
optional + yes (none) String Location / File of SSL keystore @@ -234,6 +242,7 @@ Format Options
avro-confluent.ssl.keystore.password
optional + yes (none) String Password for SSL keystore @@ -241,6 +250,7 @@ Format Options
avro-confluent.ssl.truststore.location
optional + yes (none) String Location / File of SSL truststore @@ -248,6 +258,7 @@ Format Options
avro-confluent.ssl.truststore.password
optional + yes (none) String Password for SSL truststore @@ -255,6 +266,7 @@ Format Options
avro-confluent.subject
optional + yes (none) String The Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, 'kafka' and 'upsert-kafka' connectors use '<topic_name>-value' or '<topic_name>-key' as the default subject name if this format is used as the value or key format. But for other connectors (e.g. 'filesystem'), the subject option is required when used as sink. @@ -262,6 +274,7 @@ Format Options
avro-confluent.url
required + yes (none) String The URL of the Confluent Schema Registry to fetch/register schemas. diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index 659160b77..4030168c9 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -52,6 +52,8 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_CREDENTIALS_SOURCE; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO; @@ -175,6 +177,23 @@ public Set> optionalOptions() { return options; } + @Override + public Set> forwardOptions() { + return Stream.of( + URL, + SUBJECT, + PROPERTIES, + SSL_KEYSTORE_LOCATION, + SSL_KEYSTORE_PASSWORD, + SSL_TRUSTSTORE_LOCATION, + SSL_TRUSTSTORE_PASSWORD, + BASIC_AUTH_CREDENTIALS_SOURCE, + BASIC_AUTH_USER_INFO, + BEARER_AUTH_CREDENTIALS_SOURCE, + BEARER_AUTH_TOKEN) + .collect(Collectors.toSet()); + } + public static @Nullable Map buildOptionalPropertiesMap( ReadableConfig formatOptions) { final Map properties = new HashMap<>(); From d8566a8f06c852ce243dd395092d71a8c1f96a38 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Wed, 12 Jan 2022 18:57:48 +0100 Subject: [PATCH 176/322] [FLINK-25391][format-json] Forward catalog table options This closes #18290. --- .../java/org/apache/flink/formats/json/JsonFormatFactory.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java index bf2e2878b..74d8c5310 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java @@ -155,7 +155,6 @@ public Set> optionalOptions() { @Override public Set> forwardOptions() { Set> options = new HashSet<>(); - options.add(IGNORE_PARSE_ERRORS); options.add(TIMESTAMP_FORMAT); options.add(MAP_NULL_KEY_MODE); options.add(MAP_NULL_KEY_LITERAL); From aa6e30c0f613275ddaa588217a84e550982906f4 Mon Sep 17 00:00:00 2001 From: wgzhao Date: Wed, 3 Nov 2021 15:11:35 +0800 Subject: [PATCH 177/322] [FLINK-24745][format][json] Support Oracle OGG json format This closes #17657. --- .../json/ogg/OggJsonDecodingFormat.java | 227 +++++++++++++ .../ogg/OggJsonDeserializationSchema.java | 300 ++++++++++++++++++ .../json/ogg/OggJsonFormatFactory.java | 148 +++++++++ .../json/ogg/OggJsonFormatOptions.java | 41 +++ .../json/ogg/OggJsonSerializationSchema.java | 129 ++++++++ .../org.apache.flink.table.factories.Factory | 1 + .../json/ogg/OggJsonFileSystemITCase.java | 157 +++++++++ .../json/ogg/OggJsonFormatFactoryTest.java | 143 +++++++++ .../json/ogg/OggJsonSerDeSchemaTest.java | 252 +++++++++++++++ .../src/test/resources/ogg-data.txt | 16 + 10 files changed, 1414 insertions(+) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java new file mode 100644 index 000000000..0329129b6 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.ogg; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.ogg.OggJsonDeserializationSchema.MetadataConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.types.RowKind; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** {@link DecodingFormat} for Ogg using JSON encoding. */ +public class OggJsonDecodingFormat implements DecodingFormat> { + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + private static final StringData KEY_SOURCE_TIMESTAMP = StringData.fromString("op_ts"); + + // -------------------------------------------------------------------------------------------- + // Ogg-specific attributes + // -------------------------------------------------------------------------------------------- + private static final StringData KEY_SOURCE_TABLE = StringData.fromString("table"); + private final boolean ignoreParseErrors; + private final TimestampFormat timestampFormat; + private List metadataKeys; + + public OggJsonDecodingFormat(boolean ignoreParseErrors, TimestampFormat timestampFormat) { + this.ignoreParseErrors = ignoreParseErrors; + this.timestampFormat = timestampFormat; + this.metadataKeys = Collections.emptyList(); + } + + private static Object readProperty(GenericRowData row, int pos, StringData key) { + final GenericMapData map = (GenericMapData) row.getMap(pos); + if (map == null) { + return null; + } + return map.get(key); + } + + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType physicalDataType) { + + final List readableMetadata = + metadataKeys.stream() + .map( + k -> + Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .collect(Collectors.toList()); + + final List metadataFields = + readableMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList()); + + final DataType producedDataType = + DataTypeUtils.appendRowFields(physicalDataType, metadataFields); + + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + + return new OggJsonDeserializationSchema( + physicalDataType, + readableMetadata, + producedTypeInfo, + ignoreParseErrors, + timestampFormat); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + + @Override + public Map listReadableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + Stream.of(ReadableMetadata.values()) + .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyReadableMetadata(List metadataKeys) { + this.metadataKeys = metadataKeys; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + /** List of metadata that can be read with this format. */ + enum ReadableMetadata { + INGESTION_TIMESTAMP( + "current_ts", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + true, + DataTypes.FIELD("current_ts", DataTypes.BIGINT()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + return TimestampData.fromEpochMillis(row.getLong(pos)); + } + }), + + SOURCE_TIMESTAMP( + "op_ts", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + true, + DataTypes.FIELD("op_ts", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + final StringData timestamp = + (StringData) readProperty(row, pos, KEY_SOURCE_TIMESTAMP); + if (timestamp == null) { + return null; + } + return TimestampData.fromEpochMillis(Long.parseLong(timestamp.toString())); + } + }), + + SOURCE_TABLE( + "table", + DataTypes.STRING().nullable(), + true, + DataTypes.FIELD("table", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return readProperty(row, pos, KEY_SOURCE_TABLE); + } + }), + + SOURCE_PROPERTIES( + "source.properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) + .nullable(), + true, + DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getMap(pos); + } + }); + + final String key; + + final DataType dataType; + + final boolean isJsonPayload; + + final DataTypes.Field requiredJsonField; + + final MetadataConverter converter; + + ReadableMetadata( + String key, + DataType dataType, + boolean isJsonPayload, + DataTypes.Field requiredJsonField, + MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.isJsonPayload = isJsonPayload; + this.requiredJsonField = requiredJsonField; + this.converter = converter; + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java new file mode 100644 index 000000000..4a018d573 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java @@ -0,0 +1,300 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.ogg; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; +import org.apache.flink.formats.json.ogg.OggJsonDecodingFormat.ReadableMetadata; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +import static java.lang.String.format; + +/** + * Deserialization schema from Ogg JSON to Flink Table/SQL internal data structure {@link RowData}. + * The deserialization schema knows Ogg's schema definition and can extract the database data and + * convert into {@link RowData} with {@link RowKind}. + * + *

Deserializes a byte[] message as a JSON object and reads the specified fields. + * + *

Failures during deserialization are forwarded as wrapped IOExceptions. + * + * @see Ogg + */ +@Internal +public final class OggJsonDeserializationSchema implements DeserializationSchema { + private static final long serialVersionUID = 1L; + + private static final String OP_CREATE = "I"; // insert + private static final String OP_UPDATE = "U"; // update + private static final String OP_DELETE = "D"; // delete + private static final String OP_TRUNCATE = "T"; // truncate + + private static final String REPLICA_IDENTITY_EXCEPTION = + "The \"before\" field of %s message is null, " + + "if you are using Ogg Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; + + /** The deserializer to deserialize Ogg JSON data. */ + private final JsonRowDataDeserializationSchema jsonDeserializer; + + /** Flag that indicates that an additional projection is required for metadata. */ + private final boolean hasMetadata; + + /** Metadata to be extracted for every record. */ + private final MetadataConverter[] metadataConverters; + + /** {@link TypeInformation} of the produced {@link RowData} (physical + metadata). */ + private final TypeInformation producedTypeInfo; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + public OggJsonDeserializationSchema( + DataType physicalDataType, + List requestedMetadata, + TypeInformation producedTypeInfo, + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { + final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata); + this.jsonDeserializer = + new JsonRowDataDeserializationSchema( + jsonRowType, + // the result type is never used, so it's fine to pass in the produced type + // info + producedTypeInfo, + false, // ignoreParseErrors already contains the functionality of + // failOnMissingField + ignoreParseErrors, + timestampFormat); + this.hasMetadata = requestedMetadata.size() > 0; + this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata); + this.producedTypeInfo = producedTypeInfo; + this.ignoreParseErrors = ignoreParseErrors; + } + + private static RowType createJsonRowType( + DataType physicalDataType, List readableMetadata) { + DataType payload = + DataTypes.ROW( + DataTypes.FIELD("before", physicalDataType), + DataTypes.FIELD("after", physicalDataType), + DataTypes.FIELD("op_type", DataTypes.STRING())); + + // append fields that are required for reading metadata in the payload + final List payloadMetadataFields = + readableMetadata.stream() + .filter(m -> m.isJsonPayload) + .map(m -> m.requiredJsonField) + .distinct() + .collect(Collectors.toList()); + payload = DataTypeUtils.appendRowFields(payload, payloadMetadataFields); + + DataType root = payload; + + // append fields that are required for reading metadata in the root + final List rootMetadataFields = + readableMetadata.stream() + .filter(m -> !m.isJsonPayload) + .map(m -> m.requiredJsonField) + .distinct() + .collect(Collectors.toList()); + root = DataTypeUtils.appendRowFields(root, rootMetadataFields); + + return (RowType) root.getLogicalType(); + } + + private static MetadataConverter[] createMetadataConverters( + RowType jsonRowType, List requestedMetadata) { + return requestedMetadata.stream() + .map( + m -> { + if (m.isJsonPayload) { + return convertInPayload(jsonRowType, m); + } else { + return convertInRoot(jsonRowType, m); + } + }) + .toArray(MetadataConverter[]::new); + } + + private static MetadataConverter convertInRoot(RowType jsonRowType, ReadableMetadata metadata) { + final int pos = findFieldPos(metadata, jsonRowType); + return new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData root, int unused) { + return metadata.converter.convert(root, pos); + } + }; + } + + private static MetadataConverter convertInPayload( + RowType jsonRowType, ReadableMetadata metadata) { + return convertInRoot(jsonRowType, metadata); + } + + private static int findFieldPos(ReadableMetadata metadata, RowType jsonRowType) { + return jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); + } + + @Override + public RowData deserialize(byte[] message) { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + + @Override + public void deserialize(byte[] message, Collector out) throws IOException { + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } + try { + GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); + + GenericRowData before = (GenericRowData) row.getField(0); + GenericRowData after = (GenericRowData) row.getField(1); + String op = row.getField(2).toString(); + if (OP_CREATE.equals(op)) { + after.setRowKind(RowKind.INSERT); + emitRow(row, after, out); + } else if (OP_UPDATE.equals(op)) { + if (before == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + before.setRowKind(RowKind.UPDATE_BEFORE); + after.setRowKind(RowKind.UPDATE_AFTER); + emitRow(row, before, out); + emitRow(row, after, out); + } else if (OP_DELETE.equals(op)) { + if (before == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); + } + before.setRowKind(RowKind.DELETE); + emitRow(row, before, out); + } else { + if (!ignoreParseErrors) { + throw new IOException( + format( + "Unknown \"op_type\" value \"%s\". The Ogg JSON message is '%s'", + op, new String(message))); + } + } + } catch (Throwable t) { + // a big try catch to protect the processing. + if (!ignoreParseErrors) { + throw new IOException( + format("Corrupt Ogg JSON message '%s'.", new String(message)), t); + } + } + } + + // -------------------------------------------------------------------------------------------- + + private void emitRow( + GenericRowData rootRow, GenericRowData physicalRow, Collector out) { + // shortcut in case no output projection is required + if (!hasMetadata) { + out.collect(physicalRow); + return; + } + + final int physicalArity = physicalRow.getArity(); + final int metadataArity = metadataConverters.length; + + final GenericRowData producedRow = + new GenericRowData(physicalRow.getRowKind(), physicalArity + metadataArity); + + for (int physicalPos = 0; physicalPos < physicalArity; physicalPos++) { + producedRow.setField(physicalPos, physicalRow.getField(physicalPos)); + } + + for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { + producedRow.setField( + physicalArity + metadataPos, metadataConverters[metadataPos].convert(rootRow)); + } + + out.collect(producedRow); + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return producedTypeInfo; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OggJsonDeserializationSchema that = (OggJsonDeserializationSchema) o; + return Objects.equals(jsonDeserializer, that.jsonDeserializer) + && hasMetadata == that.hasMetadata + && Objects.equals(producedTypeInfo, that.producedTypeInfo) + && ignoreParseErrors == that.ignoreParseErrors; + } + + @Override + public int hashCode() { + return Objects.hash(jsonDeserializer, hasMetadata, producedTypeInfo, ignoreParseErrors); + } + + // -------------------------------------------------------------------------------------------- + + /** + * Converter that extracts a metadata field from the row (root or payload) that comes out of the + * JSON schema and converts it to the desired data type. + */ + interface MetadataConverter extends Serializable { + + // Method for top-level access. + default Object convert(GenericRowData row) { + return convert(row, -1); + } + + Object convert(GenericRowData row, int pos); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java new file mode 100644 index 000000000..f853983d4 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.ogg; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.JsonFormatOptions; +import org.apache.flink.formats.json.JsonFormatOptionsUtil; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; +import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL; +import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.JSON_MAP_NULL_KEY_MODE; +import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.TIMESTAMP_FORMAT; + +/** + * Format factory for providing configured instances of Ogg JSON to RowData {@link + * DeserializationSchema}. + */ +@Internal +public class OggJsonFormatFactory + implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "ogg-json"; + + /** Validator for ogg decoding format. */ + private static void validateDecodingFormatOptions(ReadableConfig tableOptions) { + JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions); + } + + /** Validator for ogg encoding format. */ + private static void validateEncodingFormatOptions(ReadableConfig tableOptions) { + JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions); + } + + @Override + public DecodingFormat> createDecodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateDecodingFormatOptions(formatOptions); + + final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); + + final TimestampFormat timestampFormat = + JsonFormatOptionsUtil.getTimestampFormat(formatOptions); + + return new OggJsonDecodingFormat(ignoreParseErrors, timestampFormat); + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + + FactoryUtil.validateFactoryOptions(this, formatOptions); + validateEncodingFormatOptions(formatOptions); + + TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); + JsonFormatOptions.MapNullKeyMode mapNullKeyMode = + JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); + String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); + + final boolean encodeDecimalAsPlainNumber = + formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); + + return new EncodingFormat>() { + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + final RowType rowType = (RowType) consumedDataType.getLogicalType(); + return new OggJsonSerializationSchema( + rowType, + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); + } + }; + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(IGNORE_PARSE_ERRORS); + options.add(TIMESTAMP_FORMAT); + options.add(JSON_MAP_NULL_KEY_MODE); + options.add(JSON_MAP_NULL_KEY_LITERAL); + options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); + return options; + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java new file mode 100644 index 000000000..feeca7975 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.ogg; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.formats.json.JsonFormatOptions; + +/** Option utils for ogg-json format. */ +@PublicEvolving +public class OggJsonFormatOptions { + + public static final ConfigOption IGNORE_PARSE_ERRORS = + JsonFormatOptions.IGNORE_PARSE_ERRORS; + + public static final ConfigOption TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT; + + public static final ConfigOption JSON_MAP_NULL_KEY_MODE = + JsonFormatOptions.MAP_NULL_KEY_MODE; + + public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = + JsonFormatOptions.MAP_NULL_KEY_LITERAL; + + private OggJsonFormatOptions() {} +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java new file mode 100644 index 000000000..2189f86c3 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.ogg; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.JsonFormatOptions; +import org.apache.flink.formats.json.JsonRowDataSerializationSchema; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +import java.util.Objects; + +import static java.lang.String.format; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Ogg JSON. + * + * @see Ogg + * JSON Message + */ +public class OggJsonSerializationSchema implements SerializationSchema { + private static final long serialVersionUID = 1L; + + private static final StringData OP_INSERT = StringData.fromString("I"); // insert + private static final StringData OP_DELETE = StringData.fromString("D"); // delete + + /** The serializer to serialize Ogg JSON data. * */ + private final JsonRowDataSerializationSchema jsonSerializer; + + private transient GenericRowData genericRowData; + + public OggJsonSerializationSchema( + RowType rowType, + TimestampFormat timestampFormat, + JsonFormatOptions.MapNullKeyMode mapNullKeyMode, + String mapNullKeyLiteral, + boolean encodeDecimalAsPlainNumber) { + jsonSerializer = + new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); + } + + private static RowType createJsonRowType(DataType databaseSchema) { + // Ogg JSON contains some other information, e.g. "source", "ts_ms" + // but we don't need them. + return (RowType) + DataTypes.ROW( + DataTypes.FIELD("before", databaseSchema), + DataTypes.FIELD("after", databaseSchema), + DataTypes.FIELD("op_type", DataTypes.STRING())) + .getLogicalType(); + } + + @Override + public void open(InitializationContext context) { + genericRowData = new GenericRowData(3); + } + + @Override + public byte[] serialize(RowData rowData) { + try { + switch (rowData.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + genericRowData.setField(0, null); + genericRowData.setField(1, rowData); + genericRowData.setField(2, OP_INSERT); + return jsonSerializer.serialize(genericRowData); + case UPDATE_BEFORE: + case DELETE: + genericRowData.setField(0, rowData); + genericRowData.setField(1, null); + genericRowData.setField(2, OP_DELETE); + return jsonSerializer.serialize(genericRowData); + default: + throw new UnsupportedOperationException( + format( + "Unsupported operation '%s' for row kind.", + rowData.getRowKind())); + } + } catch (Throwable t) { + throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OggJsonSerializationSchema that = (OggJsonSerializationSchema) o; + return Objects.equals(jsonSerializer, that.jsonSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(jsonSerializer); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 08f4657d4..35e647de6 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -17,3 +17,4 @@ org.apache.flink.formats.json.JsonFormatFactory org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory org.apache.flink.formats.json.canal.CanalJsonFormatFactory org.apache.flink.formats.json.maxwell.MaxwellJsonFormatFactory +org.apache.flink.formats.json.ogg.OggJsonFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java new file mode 100644 index 000000000..eaac0d333 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.ogg; + +import org.apache.flink.table.planner.runtime.utils.StreamingTestBase; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.CollectionUtil; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static java.lang.String.format; + +/** Test Filesystem connector with DebeziumJson. */ +public class OggJsonFileSystemITCase extends StreamingTestBase { + + private static final List EXPECTED = + Arrays.asList( + "+I[101, SCOOTER, Small 2-wheel scooter, 3.14]", + "+I[102, CAR BATTERY, 12V car battery, 8.1]", + "+I[103, 12-PACK DRILL BITS, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+I[104, HAMMER, 12oz carpenter's hammer, 0.75]", + "+I[105, HAMMER, 14oz carpenter's hammer, 0.875]", + "+I[106, HAMMER, 16oz carpenter's hammer, 1.0]", + "+I[107, ROCKS, box of assorted rocks, 5.3]", + "+I[108, JACKET, water resistent black wind breaker, 0.1]", + "+I[109, SPARE TIRE, 24 inch spare tire, 22.2]", + "-D[106, HAMMER, 16oz carpenter's hammer, 1.0]", // -U + "+I[106, HAMMER, 18oz carpenter hammer, 1.0]", // +U + "-D[107, ROCKS, box of assorted rocks, 5.3]", // -U + "+I[107, ROCKS, box of assorted rocks, 5.1]", // +U + "+I[110, JACKET, water resistent white wind breaker, 0.2]", + "+I[111, SCOOTER, Big 2-wheel scooter , 5.18]", + "-D[110, JACKET, water resistent white wind breaker, 0.2]", // -U + "+I[110, JACKET, new water resistent white wind breaker, 0.5]", // +U + "-D[111, SCOOTER, Big 2-wheel scooter , 5.18]", // -U + "+I[111, SCOOTER, Big 2-wheel scooter , 5.17]", // +U + "-D[111, SCOOTER, Big 2-wheel scooter , 5.17]"); + + private File source; + private File sink; + + private static byte[] readBytes(String resource) throws IOException { + final URL url = OggJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllBytes(path); + } + + private void prepareTables(boolean isPartition) throws IOException { + byte[] bytes = readBytes("ogg-data.txt"); + source = TEMPORARY_FOLDER.newFolder(); + File file; + if (isPartition) { + File partition = new File(source, "p=1"); + partition.mkdirs(); + file = new File(partition, "my_file"); + } else { + file = new File(source, "my_file"); + } + file.createNewFile(); + Files.write(file.toPath(), bytes); + + sink = TEMPORARY_FOLDER.newFolder(); + + env().setParallelism(1); + } + + private void createTable(boolean isSink, String path, boolean isPartition) { + tEnv().executeSql( + format("create table %s (", isSink ? "sink" : "source") + + "id int, name string," + + (isSink ? "upper_name string," : "") + + " description string, weight float" + + (isPartition ? ", p int) partitioned by (p) " : ")") + + " with (" + + "'connector'='filesystem'," + + "'format'='debezium-json'," + + format("'path'='%s'", path) + + ")"); + } + + @Test + public void testNonPartition() throws Exception { + prepareTables(false); + createTable(false, source.toURI().toString(), false); + createTable(true, sink.toURI().toString(), false); + + tEnv().executeSql( + "insert into sink select id,name,UPPER(name),description,weight from source") + .await(); + CloseableIterator iter = + tEnv().executeSql("select id,upper_name,description,weight from sink").collect(); + + List results = + CollectionUtil.iteratorToList(iter).stream() + .map(Row::toString) + .collect(Collectors.toList()); + iter.close(); + + Assert.assertEquals(EXPECTED, results); + } + + @Test + public void testPartition() throws Exception { + prepareTables(true); + createTable(false, source.toURI().toString(), true); + createTable(true, sink.toURI().toString(), true); + + tEnv().executeSql( + "insert into sink select id,name,UPPER(name),description,weight,p from source") + .await(); + CloseableIterator iter = + tEnv().executeSql("select id,upper_name,description,weight,p from sink").collect(); + List list = CollectionUtil.iteratorToList(iter); + iter.close(); + + List results = + list.stream() + .map(row -> Row.project(row, new int[] {0, 1, 2, 3})) + .map(Row::toString) + .collect(Collectors.toList()); + + Assert.assertEquals(EXPECTED, results); + + // check partition value + for (Row row : list) { + Assert.assertEquals(1, row.getField(4)); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java new file mode 100644 index 000000000..fd669a1ac --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.ogg; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.JsonFormatOptions; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.TestDynamicTableFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.Consumer; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; +import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; +import static org.junit.Assert.assertEquals; + +/** Tests for {@link OggJsonFormatFactory}. */ +public class OggJsonFormatFactoryTest extends TestLogger { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testSeDeSchema() { + final Map options = getAllOptions(); + + final OggJsonSerializationSchema expectedSer = + new OggJsonSerializationSchema( + (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), + TimestampFormat.ISO_8601, + JsonFormatOptions.MapNullKeyMode.LITERAL, + "null", + true); + + final DynamicTableSink actualSink = createTableSink(SCHEMA, options); + assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + SerializationSchema actualSer = + sinkMock.valueFormat.createRuntimeEncoder( + new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); + + assertEquals(expectedSer, actualSer); + } + + @Test + public void testInvalidIgnoreParseError() { + thrown.expect( + containsCause( + new IllegalArgumentException( + "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); + + final Map options = + getModifiedOptions(opts -> opts.put("ogg-json.ignore-parse-errors", "abc")); + + createTableSource(SCHEMA, options); + } + + @Test + public void testInvalidOptionForTimestampFormat() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("ogg-json.timestamp-format.standard", "test")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); + createTableSource(SCHEMA, tableOptions); + } + + @Test + public void testInvalidOptionForMapNullKeyMode() { + final Map tableOptions = + getModifiedOptions(opts -> opts.put("ogg-json.map-null-key.mode", "invalid")); + + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); + createTableSink(SCHEMA, tableOptions); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Returns the full options modified by the given consumer {@code optionModifier}. + * + * @param optionModifier Consumer to modify the options + */ + private Map getModifiedOptions(Consumer> optionModifier) { + Map options = getAllOptions(); + optionModifier.accept(options); + return options; + } + + private Map getAllOptions() { + final Map options = new HashMap<>(); + options.put("connector", TestDynamicTableFactory.IDENTIFIER); + options.put("target", "MyTarget"); + options.put("buffer-size", "1000"); + + options.put("format", "ogg-json"); + options.put("ogg-json.ignore-parse-errors", "true"); + options.put("ogg-json.timestamp-format.standard", "ISO-8601"); + options.put("ogg-json.map-null-key.mode", "LITERAL"); + options.put("ogg-json.map-null-key.literal", "null"); + options.put("ogg-json.encode.decimal-as-plain-number", "true"); + return options; + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java new file mode 100644 index 000000000..dc99f5879 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json.ogg; + +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.JsonFormatOptions; +import org.apache.flink.formats.json.ogg.OggJsonDecodingFormat.ReadableMetadata; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.util.Collector; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.FLOAT; +import static org.apache.flink.table.api.DataTypes.INT; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** Tests for {@link OggJsonSerializationSchema} and {@link OggJsonDeserializationSchema}. */ +public class OggJsonSerDeSchemaTest { + + private static final DataType PHYSICAL_DATA_TYPE = + ROW( + FIELD("id", INT().notNull()), + FIELD("name", STRING()), + FIELD("description", STRING()), + FIELD("weight", FLOAT())); + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static List readLines(String resource) throws IOException { + final URL url = OggJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllLines(path); + } + + @Test + public void testSerializationAndDeserialization() throws Exception { + testSerializationDeserialization("ogg-data.txt"); + } + + @Test + public void testTombstoneMessages() throws Exception { + OggJsonDeserializationSchema deserializationSchema = + new OggJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), + false, + TimestampFormat.ISO_8601); + SimpleCollector collector = new SimpleCollector(); + deserializationSchema.deserialize(null, collector); + deserializationSchema.deserialize(new byte[] {}, collector); + assertTrue(collector.list.isEmpty()); + } + + private void testSerializationDeserialization(String resourceFile) throws Exception { + List lines = readLines(resourceFile); + OggJsonDeserializationSchema deserializationSchema = + new OggJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + Collections.emptyList(), + InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), + false, + TimestampFormat.ISO_8601); + + SimpleCollector collector = new SimpleCollector(); + for (String line : lines) { + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + } + + // Ogg captures change data (`ogg-data.txt`) on the `product` + // table: + // + // CREATE TABLE product ( + // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + // name VARCHAR(255), + // description VARCHAR(512), + // weight FLOAT + // ); + // ALTER TABLE product AUTO_INCREMENT = 101; + // + // INSERT INTO product + // VALUES (default,"scooter","Small 2-wheel scooter",3.14), + // (default,"car battery","12V car battery",8.1), + // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 + // to #3",0.8), + // (default,"hammer","12oz carpenter's hammer",0.75), + // (default,"hammer","14oz carpenter's hammer",0.875), + // (default,"hammer","16oz carpenter's hammer",1.0), + // (default,"rocks","box of assorted rocks",5.3), + // (default,"jacket","water resistent black wind breaker",0.1), + // (default,"spare tire","24 inch spare tire",22.2); + // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; + // UPDATE product SET weight='5.1' WHERE id=107; + // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); + // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); + // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' + // WHERE id=110; + // UPDATE product SET weight='5.17' WHERE id=111; + // DELETE FROM product WHERE id=111; + List expected = + Arrays.asList( + "+I(101,scooter,Small 2-wheel scooter,3.14)", + "+I(102,car battery,12V car battery,8.1)", + "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", + "+I(104,hammer,12oz carpenter's hammer,0.75)", + "+I(105,hammer,14oz carpenter's hammer,0.875)", + "+I(106,hammer,16oz carpenter's hammer,1.0)", + "+I(107,rocks,box of assorted rocks,5.3)", + "+I(108,jacket,water resistent black wind breaker,0.1)", + "+I(109,spare tire,24 inch spare tire,22.2)", + "-U(106,hammer,16oz carpenter's hammer,1.0)", + "+U(106,hammer,18oz carpenter hammer,1.0)", + "-U(107,rocks,box of assorted rocks,5.3)", + "+U(107,rocks,box of assorted rocks,5.1)", + "+I(110,jacket,water resistent white wind breaker,0.2)", + "+I(111,scooter,Big 2-wheel scooter ,5.18)", + "-U(110,jacket,water resistent white wind breaker,0.2)", + "+U(110,jacket,new water resistent white wind breaker,0.5)", + "-U(111,scooter,Big 2-wheel scooter ,5.18)", + "+U(111,scooter,Big 2-wheel scooter ,5.17)", + "-D(111,scooter,Big 2-wheel scooter ,5.17)"); + List actual = + collector.list.stream().map(Object::toString).collect(Collectors.toList()); + assertEquals(expected, actual); + + OggJsonSerializationSchema serializationSchema = + new OggJsonSerializationSchema( + (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), + TimestampFormat.SQL, + JsonFormatOptions.MapNullKeyMode.LITERAL, + "null", + true); + + serializationSchema.open(null); + actual = new ArrayList<>(); + for (RowData rowData : collector.list) { + actual.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + + expected = + Arrays.asList( + "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op_type\":\"I\"}", + "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op_type\":\"I\"}", + "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op_type\":\"I\"}", + "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op_type\":\"I\"}", + "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op_type\":\"I\"}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op_type\":\"I\"}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op_type\":\"I\"}", + "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op_type\":\"I\"}", + "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op_type\":\"I\"}", + "{\"before\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"after\":null,\"op_type\":\"D\"}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op_type\":\"I\"}", + "{\"before\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"after\":null,\"op_type\":\"D\"}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op_type\":\"I\"}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op_type\":\"I\"}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op_type\":\"I\"}", + "{\"before\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"after\":null,\"op_type\":\"D\"}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op_type\":\"I\"}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":null,\"op_type\":\"D\"}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op_type\":\"I\"}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op_type\":\"D\"}"); + assertEquals(expected, actual); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private void testDeserializationWithMetadata( + String resourceFile, Consumer testConsumer) throws Exception { + // we only read the first line for keeping the test simple + final String firstLine = readLines(resourceFile).get(0); + + final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); + + final DataType producedDataType = + DataTypeUtils.appendRowFields( + PHYSICAL_DATA_TYPE, + requestedMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList())); + + final OggJsonDeserializationSchema deserializationSchema = + new OggJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + requestedMetadata, + InternalTypeInfo.of(producedDataType.getLogicalType()), + false, + TimestampFormat.ISO_8601); + + final SimpleCollector collector = new SimpleCollector(); + deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); + + assertEquals(1, collector.list.size()); + testConsumer.accept(collector.list.get(0)); + } + + private static class SimpleCollector implements Collector { + + private final List list = new ArrayList<>(); + + @Override + public void collect(RowData record) { + list.add(record); + } + + @Override + public void close() { + // do nothing + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt new file mode 100644 index 000000000..04b4953c7 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt @@ -0,0 +1,16 @@ +{"table":"OGG.TBL_TEST","pos":"00000000000000000000143","priamry_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I","op_ts":"2020-05-13 15:40:06.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000144","priamry_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000145","priamry_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000146","priamry_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000147","priamry_keys":["id"],"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000148","priamry_keys":["id"],"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000149","priamry_keys":["id"],"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000150","priamry_keys":["id"],"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000151","priamry_keys":["id"],"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000152","priamry_keys":["id"],"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1},"op_type":"U","op_ts":"2020-05-13 17:26:27.936000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000153","priamry_keys":["id"],"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"op_type":"U","op_ts":"2020-05-13 17:28:19.505000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000154","priamry_keys":["id"],"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","op_ts":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op_type":"I","op_ts":"2020-05-13 17:30:10.230000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000155","priamry_keys":["id"],"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"op_type":"I","op_ts":"2020-05-13 17:30:43.428000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000156","priamry_keys":["id"],"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"op_type":"U","op_ts":"2020-05-13 17:32:20.327000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000157","priamry_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"op_type":"U","op_ts":"2020-05-13 17:32:10.904000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000158","priamry_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"op_type":"D","op_ts":"2020-05-13 17:32:24.455000"} From d2b7bb3642c9e9f67a6d06b1c2ad79530a6f82a3 Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Sat, 12 Feb 2022 16:14:17 +0800 Subject: [PATCH 178/322] [FLINK-24745][format][json] Improve the metadata tests and documentation of OGG json format This closes #18738. --- .../json/ogg/OggJsonDecodingFormat.java | 84 +++++++------------ .../ogg/OggJsonDeserializationSchema.java | 33 +------- .../json/ogg/OggJsonFileSystemITCase.java | 4 +- .../json/ogg/OggJsonSerDeSchemaTest.java | 76 +++++++++-------- .../src/test/resources/ogg-data.txt | 32 +++---- 5 files changed, 96 insertions(+), 133 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java index 0329129b6..6730fdca7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java @@ -26,13 +26,11 @@ import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.GenericMapData; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.table.utils.DateTimeUtils; import org.apache.flink.types.RowKind; import java.util.Collections; @@ -49,15 +47,14 @@ public class OggJsonDecodingFormat implements DecodingFormat metadataKeys; // -------------------------------------------------------------------------------------------- // Ogg-specific attributes // -------------------------------------------------------------------------------------------- - private static final StringData KEY_SOURCE_TABLE = StringData.fromString("table"); + private final boolean ignoreParseErrors; private final TimestampFormat timestampFormat; - private List metadataKeys; public OggJsonDecodingFormat(boolean ignoreParseErrors, TimestampFormat timestampFormat) { this.ignoreParseErrors = ignoreParseErrors; @@ -65,14 +62,6 @@ public OggJsonDecodingFormat(boolean ignoreParseErrors, TimestampFormat timestam this.metadataKeys = Collections.emptyList(); } - private static Object readProperty(GenericRowData row, int pos, StringData key) { - final GenericMapData map = (GenericMapData) row.getMap(pos); - if (map == null) { - return null; - } - return map.get(key); - } - @Override public DeserializationSchema createRuntimeDecoder( DynamicTableSource.Context context, DataType physicalDataType) { @@ -135,69 +124,64 @@ public ChangelogMode getChangelogMode() { /** List of metadata that can be read with this format. */ enum ReadableMetadata { - INGESTION_TIMESTAMP( - "current_ts", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), - true, - DataTypes.FIELD("current_ts", DataTypes.BIGINT()), + TABLE( + "table", + DataTypes.STRING().nullable(), + DataTypes.FIELD("table", DataTypes.STRING()), new MetadataConverter() { private static final long serialVersionUID = 1L; @Override public Object convert(GenericRowData row, int pos) { - if (row.isNullAt(pos)) { - return null; - } - return TimestampData.fromEpochMillis(row.getLong(pos)); + return row.getString(pos); } }), - SOURCE_TIMESTAMP( - "op_ts", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), - true, - DataTypes.FIELD("op_ts", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + PRIMARY_KEYS( + "primary-keys", + DataTypes.ARRAY(DataTypes.STRING()).nullable(), + DataTypes.FIELD("primary_keys", DataTypes.ARRAY(DataTypes.STRING())), new MetadataConverter() { private static final long serialVersionUID = 1L; @Override public Object convert(GenericRowData row, int pos) { - final StringData timestamp = - (StringData) readProperty(row, pos, KEY_SOURCE_TIMESTAMP); - if (timestamp == null) { - return null; - } - return TimestampData.fromEpochMillis(Long.parseLong(timestamp.toString())); + return row.getArray(pos); } }), - SOURCE_TABLE( - "table", - DataTypes.STRING().nullable(), - true, - DataTypes.FIELD("table", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + INGESTION_TIMESTAMP( + "ingestion-timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(6).nullable(), + DataTypes.FIELD("current_ts", DataTypes.STRING()), new MetadataConverter() { private static final long serialVersionUID = 1L; @Override public Object convert(GenericRowData row, int pos) { - return readProperty(row, pos, KEY_SOURCE_TABLE); + if (row.isNullAt(pos)) { + return null; + } + // the timestamp follows the ISO-8601 format + return DateTimeUtils.parseTimestampData( + row.getString(pos).toString(), "yyyy-MM-dd'T'HH:mm:ss.SSSSSS"); } }), - SOURCE_PROPERTIES( - "source.properties", - // key and value of the map are nullable to make handling easier in queries - DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) - .nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + EVENT_TIMESTAMP( + "event-timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(6).nullable(), + DataTypes.FIELD("op_ts", DataTypes.STRING()), new MetadataConverter() { private static final long serialVersionUID = 1L; @Override public Object convert(GenericRowData row, int pos) { - return row.getMap(pos); + if (row.isNullAt(pos)) { + return null; + } + // the timestamp format is like "yyyy-MM-dd HH:mm:ss.SSSSSS" + return DateTimeUtils.parseTimestampData(row.getString(pos).toString()); } }); @@ -205,8 +189,6 @@ public Object convert(GenericRowData row, int pos) { final DataType dataType; - final boolean isJsonPayload; - final DataTypes.Field requiredJsonField; final MetadataConverter converter; @@ -214,12 +196,10 @@ public Object convert(GenericRowData row, int pos) { ReadableMetadata( String key, DataType dataType, - boolean isJsonPayload, DataTypes.Field requiredJsonField, MetadataConverter converter) { this.key = key; this.dataType = dataType; - this.isJsonPayload = isJsonPayload; this.requiredJsonField = requiredJsonField; this.converter = converter; } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java index 4a018d573..4337bedda 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java @@ -106,46 +106,24 @@ public OggJsonDeserializationSchema( private static RowType createJsonRowType( DataType physicalDataType, List readableMetadata) { - DataType payload = + DataType root = DataTypes.ROW( DataTypes.FIELD("before", physicalDataType), DataTypes.FIELD("after", physicalDataType), DataTypes.FIELD("op_type", DataTypes.STRING())); - - // append fields that are required for reading metadata in the payload - final List payloadMetadataFields = - readableMetadata.stream() - .filter(m -> m.isJsonPayload) - .map(m -> m.requiredJsonField) - .distinct() - .collect(Collectors.toList()); - payload = DataTypeUtils.appendRowFields(payload, payloadMetadataFields); - - DataType root = payload; - // append fields that are required for reading metadata in the root final List rootMetadataFields = readableMetadata.stream() - .filter(m -> !m.isJsonPayload) .map(m -> m.requiredJsonField) .distinct() .collect(Collectors.toList()); - root = DataTypeUtils.appendRowFields(root, rootMetadataFields); - - return (RowType) root.getLogicalType(); + return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType(); } private static MetadataConverter[] createMetadataConverters( RowType jsonRowType, List requestedMetadata) { return requestedMetadata.stream() - .map( - m -> { - if (m.isJsonPayload) { - return convertInPayload(jsonRowType, m); - } else { - return convertInRoot(jsonRowType, m); - } - }) + .map(m -> convertInRoot(jsonRowType, m)) .toArray(MetadataConverter[]::new); } @@ -161,11 +139,6 @@ public Object convert(GenericRowData root, int unused) { }; } - private static MetadataConverter convertInPayload( - RowType jsonRowType, ReadableMetadata metadata) { - return convertInRoot(jsonRowType, metadata); - } - private static int findFieldPos(ReadableMetadata metadata, RowType jsonRowType) { return jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java index eaac0d333..07cded6d2 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java @@ -37,7 +37,7 @@ import static java.lang.String.format; -/** Test Filesystem connector with DebeziumJson. */ +/** Test Filesystem connector with OGG Json. */ public class OggJsonFileSystemITCase extends StreamingTestBase { private static final List EXPECTED = @@ -101,7 +101,7 @@ private void createTable(boolean isSink, String path, boolean isPartition) { + (isPartition ? ", p int) partitioned by (p) " : ")") + " with (" + "'connector'='filesystem'," - + "'format'='debezium-json'," + + "'format'='ogg-json'," + format("'path'='%s'", path) + ")"); } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java index dc99f5879..8ad6735fd 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java @@ -77,6 +77,11 @@ public void testSerializationAndDeserialization() throws Exception { testSerializationDeserialization("ogg-data.txt"); } + @Test + public void testDeserializationWithMetadata() throws Exception { + testDeserializationWithMetadata("ogg-data.txt"); + } + @Test public void testTombstoneMessages() throws Exception { OggJsonDeserializationSchema deserializationSchema = @@ -92,6 +97,44 @@ public void testTombstoneMessages() throws Exception { assertTrue(collector.list.isEmpty()); } + public void testDeserializationWithMetadata(String resourceFile) throws Exception { + // we only read the first line for keeping the test simple + final String firstLine = readLines(resourceFile).get(0); + + final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); + + final DataType producedDataTypes = + DataTypeUtils.appendRowFields( + PHYSICAL_DATA_TYPE, + requestedMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList())); + final OggJsonDeserializationSchema deserializationSchema = + new OggJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, + requestedMetadata, + InternalTypeInfo.of(producedDataTypes.getLogicalType()), + false, + TimestampFormat.ISO_8601); + + final SimpleCollector collector = new SimpleCollector(); + deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); + assertEquals(1, collector.list.size()); + + Consumer consumer = + row -> { + assertEquals(101, row.getInt(0)); + assertEquals("scooter", row.getString(1).toString()); + assertEquals("Small 2-wheel scooter", row.getString(2).toString()); + assertEquals(3.140000104904175, row.getFloat(3), 1e-15); + assertEquals("OGG.TBL_TEST", row.getString(4).toString()); + assertEquals("id", row.getArray(5).getString(0).toString()); + assertEquals(1589377175766L, row.getTimestamp(6, 6).getMillisecond()); + assertEquals(1589384406000L, row.getTimestamp(7, 6).getMillisecond()); + }; + consumer.accept(collector.list.get(0)); + } + private void testSerializationDeserialization(String resourceFile) throws Exception { List lines = readLines(resourceFile); OggJsonDeserializationSchema deserializationSchema = @@ -202,39 +245,6 @@ private void testSerializationDeserialization(String resourceFile) throws Except assertEquals(expected, actual); } - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - private void testDeserializationWithMetadata( - String resourceFile, Consumer testConsumer) throws Exception { - // we only read the first line for keeping the test simple - final String firstLine = readLines(resourceFile).get(0); - - final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); - - final DataType producedDataType = - DataTypeUtils.appendRowFields( - PHYSICAL_DATA_TYPE, - requestedMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList())); - - final OggJsonDeserializationSchema deserializationSchema = - new OggJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - requestedMetadata, - InternalTypeInfo.of(producedDataType.getLogicalType()), - false, - TimestampFormat.ISO_8601); - - final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - - assertEquals(1, collector.list.size()); - testConsumer.accept(collector.list.get(0)); - } - private static class SimpleCollector implements Collector { private final List list = new ArrayList<>(); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt index 04b4953c7..d29592532 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt @@ -1,16 +1,16 @@ -{"table":"OGG.TBL_TEST","pos":"00000000000000000000143","priamry_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I","op_ts":"2020-05-13 15:40:06.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000144","priamry_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000145","priamry_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000146","priamry_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000147","priamry_keys":["id"],"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000148","priamry_keys":["id"],"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000149","priamry_keys":["id"],"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000150","priamry_keys":["id"],"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000151","priamry_keys":["id"],"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000152","priamry_keys":["id"],"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1},"op_type":"U","op_ts":"2020-05-13 17:26:27.936000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000153","priamry_keys":["id"],"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"op_type":"U","op_ts":"2020-05-13 17:28:19.505000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000154","priamry_keys":["id"],"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","op_ts":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op_type":"I","op_ts":"2020-05-13 17:30:10.230000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000155","priamry_keys":["id"],"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"op_type":"I","op_ts":"2020-05-13 17:30:43.428000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000156","priamry_keys":["id"],"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"op_type":"U","op_ts":"2020-05-13 17:32:20.327000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000157","priamry_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"op_type":"U","op_ts":"2020-05-13 17:32:10.904000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000158","priamry_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"op_type":"D","op_ts":"2020-05-13 17:32:24.455000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000147","primary_keys":["id"],"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000148","primary_keys":["id"],"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000149","primary_keys":["id"],"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000150","primary_keys":["id"],"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000151","primary_keys":["id"],"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000152","primary_keys":["id"],"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1},"op_type":"U","op_ts":"2020-05-13 17:26:27.936000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000153","primary_keys":["id"],"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"op_type":"U","op_ts":"2020-05-13 17:28:19.505000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000154","primary_keys":["id"],"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","op_ts":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op_type":"I","op_ts":"2020-05-13 17:30:10.230000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000155","primary_keys":["id"],"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"op_type":"I","op_ts":"2020-05-13 17:30:43.428000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000156","primary_keys":["id"],"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"op_type":"U","op_ts":"2020-05-13 17:32:20.327000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000157","primary_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"op_type":"U","op_ts":"2020-05-13 17:32:10.904000"} +{"table":"OGG.TBL_TEST","pos":"00000000000000000000158","primary_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"op_type":"D","op_ts":"2020-05-13 17:32:24.455000"} From 4b1ca3e4d5183072f3cae7f0feada399f74f7f3d Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Tue, 15 Mar 2022 01:27:55 +0800 Subject: [PATCH 179/322] Update version to 1.16-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 55fbbf2bb..648065d66 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.16-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 4b791287a..2e925688e 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.15-SNAPSHOT + 1.16-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index a04294917..38603dbd3 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.16-SNAPSHOT .. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index b00aa61a8..1827f7597 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.16-SNAPSHOT .. From 9beeb21f67e11c3a536d05a5d22246a11a501f22 Mon Sep 17 00:00:00 2001 From: Ryan Skraba Date: Fri, 25 Mar 2022 10:31:02 +0100 Subject: [PATCH 180/322] [FLINK-26736][tests] Migrate flink-avro-confluent-registry to JUnit5 --- .../CachedSchemaCoderProviderTest.java | 36 +++++----- .../ConfluentSchemaRegistryCoderTest.java | 26 +++---- .../RegistryAvroFormatFactoryTest.java | 54 +++++++-------- .../RegistryAvroRowDataSeDeSchemaTest.java | 67 ++++++++----------- .../DebeziumAvroFormatFactoryTest.java | 23 +++---- .../debezium/DebeziumAvroSerDeSchemaTest.java | 53 ++++++++++----- .../org.junit.jupiter.api.extension.Extension | 16 +++++ 7 files changed, 143 insertions(+), 132 deletions(-) create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java index db877d8ed..cddbf1349 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java @@ -22,7 +22,7 @@ import io.confluent.kafka.schemaregistry.client.rest.RestService; import io.confluent.kafka.schemaregistry.client.security.basicauth.BasicAuthCredentialProvider; import io.confluent.kafka.schemaregistry.client.security.bearerauth.BearerAuthCredentialProvider; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.powermock.reflect.Whitebox; import javax.net.ssl.SSLSocketFactory; @@ -31,26 +31,24 @@ import java.util.HashMap; import java.util.Map; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; +import static org.assertj.core.api.Assertions.assertThat; /** * Tests for properties set by {@link RegistryAvroFormatFactory} in {@link * CachedSchemaCoderProvider}. */ -public class CachedSchemaCoderProviderTest { +class CachedSchemaCoderProviderTest { @Test - public void testThatSslIsNotInitializedForNoSslProperties() { + void testThatSslIsNotInitializedForNoSslProperties() { CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(new HashMap<>()); SSLSocketFactory sslSocketFactory = getSslSocketFactoryFromProvider(provider); - assertNull(sslSocketFactory); + assertThat(sslSocketFactory).isNull(); } @Test - public void testThatSslIsInitializedForSslProperties() throws URISyntaxException { + void testThatSslIsInitializedForSslProperties() throws URISyntaxException { String keystoreFile = getAbsolutePath("/test-keystore.jks"); String keystorePassword = "123456"; Map configs = new HashMap<>(); @@ -62,20 +60,20 @@ public void testThatSslIsInitializedForSslProperties() throws URISyntaxException CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(configs); SSLSocketFactory sslSocketFactory = getSslSocketFactoryFromProvider(provider); - assertNotNull(sslSocketFactory); + assertThat(sslSocketFactory).isNotNull(); } @Test - public void testThatBasicAuthIsNotInitializedForNoBasicAuthProperties() { + void testThatBasicAuthIsNotInitializedForNoBasicAuthProperties() { CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(new HashMap<>()); BasicAuthCredentialProvider basicAuthCredentialProvider = getBasicAuthFromProvider(provider); - assertNull(basicAuthCredentialProvider); + assertThat(basicAuthCredentialProvider).isNull(); } @Test - public void testThatBasicAuthIsInitializedForBasicAuthProperties() { + void testThatBasicAuthIsInitializedForBasicAuthProperties() { String userPassword = "user:pwd"; Map configs = new HashMap<>(); configs.put("basic.auth.credentials.source", "USER_INFO"); @@ -85,21 +83,21 @@ public void testThatBasicAuthIsInitializedForBasicAuthProperties() { BasicAuthCredentialProvider basicAuthCredentialProvider = getBasicAuthFromProvider(provider); - assertNotNull(basicAuthCredentialProvider); - assertEquals(basicAuthCredentialProvider.getUserInfo(null), userPassword); + assertThat(basicAuthCredentialProvider).isNotNull(); + assertThat(basicAuthCredentialProvider.getUserInfo(null)).isEqualTo(userPassword); } @Test - public void testThatBearerAuthIsNotInitializedForNoBearerAuthProperties() { + void testThatBearerAuthIsNotInitializedForNoBearerAuthProperties() { CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(new HashMap<>()); BearerAuthCredentialProvider bearerAuthCredentialProvider = getBearerAuthFromProvider(provider); - assertNull(bearerAuthCredentialProvider); + assertThat(bearerAuthCredentialProvider).isNull(); } @Test - public void testThatBearerAuthIsInitializedForBearerAuthProperties() { + void testThatBearerAuthIsInitializedForBearerAuthProperties() { String token = "123456"; Map configs = new HashMap<>(); configs.put("bearer.auth.credentials.source", "STATIC_TOKEN"); @@ -109,8 +107,8 @@ public void testThatBearerAuthIsInitializedForBearerAuthProperties() { BearerAuthCredentialProvider bearerAuthCredentialProvider = getBearerAuthFromProvider(provider); - assertNotNull(bearerAuthCredentialProvider); - assertEquals(bearerAuthCredentialProvider.getBearerToken(null), token); + assertThat(bearerAuthCredentialProvider).isNotNull(); + assertThat(bearerAuthCredentialProvider.getBearerToken(null)).isEqualTo(token); } private String getAbsolutePath(String path) throws URISyntaxException { diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java index 009ef029b..2c93348b0 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java @@ -21,20 +21,21 @@ import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.io.IOException; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link ConfluentSchemaRegistryCoder}. */ -public class ConfluentSchemaRegistryCoderTest { +class ConfluentSchemaRegistryCoderTest { @Test - public void testSpecificRecordWithConfluentSchemaRegistry() throws Exception { + void testSpecificRecordWithConfluentSchemaRegistry() throws Exception { MockSchemaRegistryClient client = new MockSchemaRegistryClient(); Schema schema = @@ -51,12 +52,12 @@ public void testSpecificRecordWithConfluentSchemaRegistry() throws Exception { ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray()); Schema readSchema = registryCoder.readSchema(byteInStream); - assertEquals(schema, readSchema); - assertEquals(0, byteInStream.available()); + assertThat(readSchema).isEqualTo(schema); + assertThat(byteInStream).isEmpty(); } - @Test(expected = IOException.class) - public void testMagicByteVerification() throws Exception { + @Test + void testMagicByteVerification() throws Exception { MockSchemaRegistryClient client = new MockSchemaRegistryClient(); int schemaId = client.register("testTopic", Schema.create(Schema.Type.BOOLEAN)); @@ -67,9 +68,10 @@ public void testMagicByteVerification() throws Exception { dataOutputStream.writeInt(schemaId); dataOutputStream.flush(); - ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray()); - coder.readSchema(byteInStream); - - // exception is thrown + try (ByteArrayInputStream byteInStream = + new ByteArrayInputStream(byteOutStream.toByteArray())) { + assertThatThrownBy(() -> coder.readSchema(byteInStream)) + .isInstanceOf(IOException.class); + } } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java index 7c5b42666..cf80adf3e 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java @@ -37,23 +37,20 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.hamcrest.CoreMatchers.instanceOf; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link RegistryAvroFormatFactory}. */ -public class RegistryAvroFormatFactoryTest { +class RegistryAvroFormatFactoryTest { private static final ResolvedSchema SCHEMA = ResolvedSchema.of( @@ -81,10 +78,8 @@ public class RegistryAvroFormatFactoryTest { EXPECTED_OPTIONAL_PROPERTIES.put("bearer.auth.token", "CUSTOM"); } - @Rule public ExpectedException thrown = ExpectedException.none(); - @Test - public void testDeserializationSchema() { + void testDeserializationSchema() { final AvroRowDataDeserializationSchema expectedDeser = new AvroRowDataDeserializationSchema( ConfluentRegistryAvroDeserializationSchema.forGeneric( @@ -93,7 +88,7 @@ public void testDeserializationSchema() { InternalTypeInfo.of(ROW_TYPE)); final DynamicTableSource actualSource = createTableSource(SCHEMA, getDefaultOptions()); - assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); + assertThat(actualSource).isInstanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class); TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; @@ -101,11 +96,11 @@ public void testDeserializationSchema() { scanSourceMock.valueFormat.createRuntimeDecoder( ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); - assertEquals(expectedDeser, actualDeser); + assertThat(actualDeser).isEqualTo(expectedDeser); } @Test - public void testSerializationSchema() { + void testSerializationSchema() { final AvroRowDataSerializationSchema expectedSer = new AvroRowDataSerializationSchema( ROW_TYPE, @@ -116,32 +111,31 @@ public void testSerializationSchema() { RowDataToAvroConverters.createConverter(ROW_TYPE)); final DynamicTableSink actualSink = createTableSink(SCHEMA, getDefaultOptions()); - assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); + assertThat(actualSink).isInstanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class); TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; SerializationSchema actualSer = sinkMock.valueFormat.createRuntimeEncoder(null, SCHEMA.toPhysicalRowDataType()); - assertEquals(expectedSer, actualSer); + assertThat(actualSer).isEqualTo(expectedSer); } @Test - public void testMissingSubjectForSink() { - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Option avro-confluent.subject is required for serialization"))); - + void testMissingSubjectForSink() { final Map options = getModifiedOptions(opts -> opts.remove("avro-confluent.subject")); - createTableSink(SCHEMA, options); + assertThatThrownBy(() -> createTableSink(SCHEMA, options)) + .isInstanceOf(ValidationException.class) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Option avro-confluent.subject is required for serialization")); } @Test - public void testDeserializationSchemaWithOptionalProperties() { + void testDeserializationSchemaWithOptionalProperties() { final AvroRowDataDeserializationSchema expectedDeser = new AvroRowDataDeserializationSchema( ConfluentRegistryAvroDeserializationSchema.forGeneric( @@ -152,7 +146,7 @@ public void testDeserializationSchemaWithOptionalProperties() { InternalTypeInfo.of(ROW_TYPE)); final DynamicTableSource actualSource = createTableSource(SCHEMA, getOptionalProperties()); - assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); + assertThat(actualSource).isInstanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class); TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; @@ -160,11 +154,11 @@ public void testDeserializationSchemaWithOptionalProperties() { scanSourceMock.valueFormat.createRuntimeDecoder( ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); - assertEquals(expectedDeser, actualDeser); + assertThat(actualDeser).isEqualTo(expectedDeser); } @Test - public void testSerializationSchemaWithOptionalProperties() { + void testSerializationSchemaWithOptionalProperties() { final AvroRowDataSerializationSchema expectedSer = new AvroRowDataSerializationSchema( ROW_TYPE, @@ -176,14 +170,14 @@ public void testSerializationSchemaWithOptionalProperties() { RowDataToAvroConverters.createConverter(ROW_TYPE)); final DynamicTableSink actualSink = createTableSink(SCHEMA, getOptionalProperties()); - assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); + assertThat(actualSink).isInstanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class); TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; SerializationSchema actualSer = sinkMock.valueFormat.createRuntimeEncoder(null, SCHEMA.toPhysicalRowDataType()); - assertEquals(expectedSer, actualSer); + assertThat(actualSer).isEqualTo(expectedSer); } // ------------------------------------------------------------------------ diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java index 0061e8b63..a5046ac23 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java @@ -39,29 +39,23 @@ import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.junit.After; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Random; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.apache.flink.formats.avro.utils.AvroTestUtils.writeRecord; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * Tests for {@link AvroRowDataDeserializationSchema} and {@link AvroRowDataSerializationSchema} for * schema registry avro. */ -public class RegistryAvroRowDataSeDeSchemaTest { +class RegistryAvroRowDataSeDeSchemaTest { private static final Schema ADDRESS_SCHEMA = Address.getClassSchema(); private static final Schema ADDRESS_SCHEMA_COMPATIBLE = @@ -83,45 +77,43 @@ public class RegistryAvroRowDataSeDeSchemaTest { private Address address; - @Rule public ExpectedException expectedEx = ExpectedException.none(); - - @BeforeClass - public static void beforeClass() { + @BeforeAll + static void beforeClass() { client = new MockSchemaRegistryClient(); } - @Before - public void before() { + @BeforeEach + void before() { this.address = TestDataGenerator.generateRandomAddress(new Random()); } - @After - public void after() throws IOException, RestClientException { + @AfterEach + void after() throws IOException, RestClientException { client.deleteSubject(SUBJECT); } @Test - public void testRowDataWriteReadWithFullSchema() throws Exception { + void testRowDataWriteReadWithFullSchema() throws Exception { testRowDataWriteReadWithSchema(ADDRESS_SCHEMA); } @Test - public void testRowDataWriteReadWithCompatibleSchema() throws Exception { + void testRowDataWriteReadWithCompatibleSchema() throws Exception { testRowDataWriteReadWithSchema(ADDRESS_SCHEMA_COMPATIBLE); // Validates new schema has been registered. - assertThat(client.getAllVersions(SUBJECT).size(), is(1)); + assertThat(client.getAllVersions(SUBJECT)).hasSize(1); } @Test - public void testRowDataWriteReadWithPreRegisteredSchema() throws Exception { + void testRowDataWriteReadWithPreRegisteredSchema() throws Exception { client.register(SUBJECT, ADDRESS_SCHEMA); testRowDataWriteReadWithSchema(ADDRESS_SCHEMA); // Validates it does not produce new schema. - assertThat(client.getAllVersions(SUBJECT).size(), is(1)); + assertThat(client.getAllVersions(SUBJECT)).hasSize(1); } @Test - public void testRowDataReadWithNonRegistryAvro() throws Exception { + void testRowDataReadWithNonRegistryAvro() throws Exception { DataType dataType = AvroSchemaConverter.convertToDataType(ADDRESS_SCHEMA.toString()); RowType rowType = (RowType) dataType.getLogicalType(); @@ -132,10 +124,9 @@ public void testRowDataReadWithNonRegistryAvro() throws Exception { client.register(SUBJECT, ADDRESS_SCHEMA); byte[] oriBytes = writeRecord(address, ADDRESS_SCHEMA); - expectedEx.expect(IOException.class); - expectedEx.expect( - containsCause(new IOException("Unknown data format. Magic number does not match"))); - deserializer.deserialize(oriBytes); + assertThatThrownBy(() -> deserializer.deserialize(oriBytes)) + .isInstanceOf(IOException.class) + .hasCause(new IOException("Unknown data format. Magic number does not match")); } private void testRowDataWriteReadWithSchema(Schema schema) throws Exception { @@ -150,18 +141,18 @@ private void testRowDataWriteReadWithSchema(Schema schema) throws Exception { serializer.open(null); deserializer.open(null); - assertNull(deserializer.deserialize(null)); + assertThat(deserializer.deserialize(null)).isNull(); RowData oriData = address2RowData(address); byte[] serialized = serializer.serialize(oriData); RowData rowData = deserializer.deserialize(serialized); - assertThat(rowData.getArity(), equalTo(schema.getFields().size())); - assertEquals(address.getNum(), rowData.getInt(0)); - assertEquals(address.getStreet(), rowData.getString(1).toString()); + assertThat(rowData.getArity()).isEqualTo(schema.getFields().size()); + assertThat(rowData.getInt(0)).isEqualTo(address.getNum()); + assertThat(rowData.getString(1).toString()).isEqualTo(address.getStreet()); if (schema != ADDRESS_SCHEMA_COMPATIBLE) { - assertEquals(address.getCity(), rowData.getString(2).toString()); - assertEquals(address.getState(), rowData.getString(3).toString()); - assertEquals(address.getZip(), rowData.getString(4).toString()); + assertThat(rowData.getString(2).toString()).isEqualTo(address.getCity()); + assertThat(rowData.getString(3).toString()).isEqualTo(address.getState()); + assertThat(rowData.getString(4).toString()).isEqualTo(address.getZip()); } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java index eb91ccbed..50d270baf 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java @@ -31,25 +31,18 @@ import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.util.TestLogger; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; -import static junit.framework.TestCase.assertEquals; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.hamcrest.CoreMatchers.instanceOf; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link DebeziumAvroFormatFactory}. */ -public class DebeziumAvroFormatFactoryTest extends TestLogger { - @Rule public ExpectedException thrown = ExpectedException.none(); +class DebeziumAvroFormatFactoryTest { private static final ResolvedSchema SCHEMA = ResolvedSchema.of( @@ -64,7 +57,7 @@ public class DebeziumAvroFormatFactoryTest extends TestLogger { private static final String REGISTRY_URL = "http://localhost:8081"; @Test - public void testSeDeSchema() { + void testSeDeSchema() { final Map options = getAllOptions(); final Map registryConfigs = new HashMap<>(); @@ -75,13 +68,13 @@ public void testSeDeSchema() { new DebeziumAvroDeserializationSchema( ROW_TYPE, InternalTypeInfo.of(ROW_TYPE), REGISTRY_URL, registryConfigs); DeserializationSchema actualDeser = createDeserializationSchema(options); - assertEquals(expectedDeser, actualDeser); + assertThat(actualDeser).isEqualTo(expectedDeser); DebeziumAvroSerializationSchema expectedSer = new DebeziumAvroSerializationSchema( ROW_TYPE, REGISTRY_URL, SUBJECT, registryConfigs); SerializationSchema actualSer = createSerializationSchema(options); - Assert.assertEquals(expectedSer, actualSer); + assertThat(actualSer).isEqualTo(expectedSer); } private Map getAllOptions() { @@ -101,7 +94,7 @@ private Map getAllOptions() { private static DeserializationSchema createDeserializationSchema( Map options) { final DynamicTableSource actualSource = createTableSource(SCHEMA, options); - assertThat(actualSource, instanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class)); + assertThat(actualSource).isInstanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class); TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; @@ -112,7 +105,7 @@ private static DeserializationSchema createDeserializationSchema( private static SerializationSchema createSerializationSchema( Map options) { final DynamicTableSink actualSink = createTableSink(SCHEMA, options); - assertThat(actualSink, instanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class)); + assertThat(actualSink).isInstanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class); TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java index a1130386c..2640b50e4 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java @@ -28,6 +28,8 @@ import org.apache.flink.formats.avro.RowDataToAvroConverters; import org.apache.flink.formats.avro.registry.confluent.ConfluentSchemaRegistryCoder; import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; @@ -35,12 +37,14 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; import org.apache.flink.util.FileUtils; +import org.apache.flink.util.SimpleUserCodeClassLoader; +import org.apache.flink.util.UserCodeClassLoader; import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -58,11 +62,10 @@ import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link DebeziumAvroDeserializationSchema}. */ -public class DebeziumAvroSerDeSchemaTest { +class DebeziumAvroSerDeSchemaTest { private static final String SUBJECT = "testDebeziumAvro"; @@ -78,10 +81,10 @@ public class DebeziumAvroSerDeSchemaTest { private static final Schema DEBEZIUM_SCHEMA_COMPATIBLE_TEST = new Schema.Parser().parse(new String(readBytesFromFile("debezium-test-schema.json"))); - private SchemaRegistryClient client = new MockSchemaRegistryClient(); + private final SchemaRegistryClient client = new MockSchemaRegistryClient(); @Test - public void testSerializationDeserialization() throws Exception { + void testSerializationDeserialization() throws Exception { RowType rowTypeDe = DebeziumAvroDeserializationSchema.createDebeziumAvroRowType( @@ -92,7 +95,7 @@ public void testSerializationDeserialization() throws Exception { DebeziumAvroSerializationSchema dbzSerializer = new DebeziumAvroSerializationSchema(getSerializationSchema(rowTypeSe)); - dbzSerializer.open(mock(SerializationSchema.InitializationContext.class)); + dbzSerializer.open(new MockInitializationContext()); byte[] serialize = dbzSerializer.serialize(debeziumRow2RowData()); @@ -100,7 +103,7 @@ public void testSerializationDeserialization() throws Exception { DebeziumAvroDeserializationSchema dbzDeserializer = new DebeziumAvroDeserializationSchema( InternalTypeInfo.of(rowType), getDeserializationSchema(rowTypeDe)); - dbzDeserializer.open(mock(DeserializationSchema.InitializationContext.class)); + dbzDeserializer.open(new MockInitializationContext()); SimpleCollector collector = new SimpleCollector(); dbzDeserializer.deserialize(serialize, collector); @@ -110,37 +113,37 @@ public void testSerializationDeserialization() throws Exception { List expected = Collections.singletonList("+I(107,rocks,box of assorted rocks,5.3)"); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test - public void testInsertDataDeserialization() throws Exception { + void testInsertDataDeserialization() throws Exception { List actual = testDeserialization("debezium-avro-insert.avro"); List expected = Collections.singletonList("+I(1,lisi,test debezium avro data,21.799999237060547)"); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test - public void testUpdateDataDeserialization() throws Exception { + void testUpdateDataDeserialization() throws Exception { List actual = testDeserialization("debezium-avro-update.avro"); List expected = Arrays.asList( "-U(1,lisi,test debezium avro data,21.799999237060547)", "+U(1,zhangsan,test debezium avro data,21.799999237060547)"); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test - public void testDeleteDataDeserialization() throws Exception { + void testDeleteDataDeserialization() throws Exception { List actual = testDeserialization("debezium-avro-delete.avro"); List expected = Collections.singletonList( "-D(1,zhangsan,test debezium avro data,21.799999237060547)"); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } public List testDeserialization(String dataPath) throws Exception { @@ -153,7 +156,7 @@ public List testDeserialization(String dataPath) throws Exception { DebeziumAvroDeserializationSchema dbzDeserializer = new DebeziumAvroDeserializationSchema( InternalTypeInfo.of(rowType), getDeserializationSchema(rowTypeDe)); - dbzDeserializer.open(mock(DeserializationSchema.InitializationContext.class)); + dbzDeserializer.open(new MockInitializationContext()); SimpleCollector collector = new SimpleCollector(); dbzDeserializer.deserialize(readBytesFromFile(dataPath), collector); @@ -200,7 +203,7 @@ private static RowData debeziumRow2RowData() { private static byte[] readBytesFromFile(String filePath) { try { URL url = DebeziumAvroSerDeSchemaTest.class.getClassLoader().getResource(filePath); - assert url != null; + assertThat(url).isNotNull(); Path path = new File(url.getFile()).toPath(); return FileUtils.readAllBytes(path); } catch (IOException e) { @@ -210,7 +213,7 @@ private static byte[] readBytesFromFile(String filePath) { private static class SimpleCollector implements Collector { - private List list = new ArrayList<>(); + private final List list = new ArrayList<>(); @Override public void collect(RowData record) { @@ -222,4 +225,18 @@ public void close() { // do nothing } } + + private static class MockInitializationContext + implements DeserializationSchema.InitializationContext, + SerializationSchema.InitializationContext { + @Override + public MetricGroup getMetricGroup() { + return new UnregisteredMetricsGroup(); + } + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + return SimpleUserCodeClassLoader.create(getClass().getClassLoader()); + } + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension new file mode 100644 index 000000000..28999133c --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.util.TestLoggerExtension \ No newline at end of file From 239b3613f129a6b51c6610672769debf34959c2c Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Thu, 31 Mar 2022 15:42:00 +0200 Subject: [PATCH 181/322] [FLINK-26961][connectors][filesystems][formats] Update Jackson Databind and Annotations to 2.13.2.2, Jackson Dataformat to 2.13.2, Jackson Core to 2.13.2 and Jackson-BOM to 2.13.2.20220328. This closes #19303 --- .../src/main/resources/META-INF/NOTICE | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index bfddc9b0f..48394cfce 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -7,9 +7,9 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - org.apache.avro:avro:1.10.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 - org.apache.commons:commons-compress:1.21 - io.confluent:kafka-schema-registry-client:6.2.2 - org.apache.kafka:kafka-clients:6.2.2-ccs From 1b0ab39c34f357502e0e188b735dd28ae7a7b83c Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Mon, 4 Apr 2022 16:33:02 +0200 Subject: [PATCH 182/322] [FLINK-26011][avro-confluent-registry][test] add ArchUnit tests for the test code --- .../1483f895-db24-4888-a2fa-991d602eaafc | 0 .../bfcdf96e-7dcd-4715-9f39-73483823ec16 | 0 .../archunit-violations/stored.rules | 4 ++ .../flink-avro-confluent-registry/pom.xml | 14 +++++++ .../TestCodeArchitectureTest.java | 40 +++++++++++++++++++ .../src/test/resources/archunit.properties | 31 ++++++++++++++ 6 files changed, 89 insertions(+) create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/1483f895-db24-4888-a2fa-991d602eaafc create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/bfcdf96e-7dcd-4715-9f39-73483823ec16 create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/stored.rules create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java create mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/archunit.properties diff --git a/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/1483f895-db24-4888-a2fa-991d602eaafc b/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/1483f895-db24-4888-a2fa-991d602eaafc new file mode 100644 index 000000000..e69de29bb diff --git a/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/bfcdf96e-7dcd-4715-9f39-73483823ec16 b/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/bfcdf96e-7dcd-4715-9f39-73483823ec16 new file mode 100644 index 000000000..e69de29bb diff --git a/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/stored.rules b/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/stored.rules new file mode 100644 index 000000000..8f05e97f7 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Mon Apr 04 16:32:09 CEST 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=bfcdf96e-7dcd-4715-9f39-73483823ec16 +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=1483f895-db24-4888-a2fa-991d602eaafc diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 2e925688e..cd29ae1c8 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -131,6 +131,20 @@ under the License. test test-jar + + + + + org.apache.flink + flink-architecture-tests-test + + + com.google.guava + guava + + + test + diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java new file mode 100644 index 000000000..837f39e68 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.architecture; + +import org.apache.flink.architecture.common.ImportOptions; + +import com.tngtech.archunit.core.importer.ImportOption; +import com.tngtech.archunit.junit.AnalyzeClasses; +import com.tngtech.archunit.junit.ArchTest; +import com.tngtech.archunit.junit.ArchTests; + +/** Architecture tests for test code. */ +@AnalyzeClasses( + packages = {"org.apache.flink.formats.avro.registry.confluent"}, + importOptions = { + ImportOption.OnlyIncludeTests.class, + ImportOptions.ExcludeScalaImportOption.class, + ImportOptions.ExcludeShadedImportOption.class + }) +public class TestCodeArchitectureTest { + + @ArchTest + public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); +} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/archunit.properties b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/archunit.properties new file mode 100644 index 000000000..15be88c95 --- /dev/null +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/archunit.properties @@ -0,0 +1,31 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# By default we allow removing existing violations, but fail when new violations are added. +freeze.store.default.allowStoreUpdate=true + +# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. +#freeze.store.default.allowStoreCreation=true + +# Enable this to add allow new violations to be recorded. +# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new +# violation, please try to avoid creating the violation. If the violation was created due to a +# shortcoming of the rule, file a JIRA issue so the rule can be improved. +#freeze.refreeze=true + +freeze.store.default.path=archunit-violations From a56a1329284057e10d46e3a36f8002dc6a4f0b55 Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Mon, 4 Apr 2022 17:12:02 +0200 Subject: [PATCH 183/322] [FLINK-26011][json][test] add ArchUnit tests for the test code --- .../24119078-1071-4906-b2ac-ed57c8154eaa | 0 .../62c5e4e5-2b0e-41ed-a268-ee33d5edd162 | 6 +++ .../archunit-violations/stored.rules | 4 ++ .../flink-json-debezium/pom.xml | 8 ++++ .../TestCodeArchitectureTest.java | 40 +++++++++++++++++++ .../src/test/resources/archunit.properties | 31 ++++++++++++++ 6 files changed, 89 insertions(+) create mode 100644 flink-formats-kafka/flink-json-debezium/archunit-violations/24119078-1071-4906-b2ac-ed57c8154eaa create mode 100644 flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 create mode 100644 flink-formats-kafka/flink-json-debezium/archunit-violations/stored.rules create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/archunit.properties diff --git a/flink-formats-kafka/flink-json-debezium/archunit-violations/24119078-1071-4906-b2ac-ed57c8154eaa b/flink-formats-kafka/flink-json-debezium/archunit-violations/24119078-1071-4906-b2ac-ed57c8154eaa new file mode 100644 index 000000000..e69de29bb diff --git a/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 b/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 new file mode 100644 index 000000000..a13a3ce8f --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 @@ -0,0 +1,6 @@ +org.apache.flink.formats.json.JsonBatchFileSystemITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/archunit-violations/stored.rules b/flink-formats-kafka/flink-json-debezium/archunit-violations/stored.rules new file mode 100644 index 000000000..d43a1445d --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Mon Apr 04 17:11:32 CEST 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=24119078-1071-4906-b2ac-ed57c8154eaa +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=62c5e4e5-2b0e-41ed-a268-ee33d5edd162 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 38603dbd3..967652a46 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -123,6 +123,14 @@ under the License. scala-compiler test + + + + + org.apache.flink + flink-architecture-tests-test + test + diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java new file mode 100644 index 000000000..ad61f218b --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.architecture; + +import org.apache.flink.architecture.common.ImportOptions; + +import com.tngtech.archunit.core.importer.ImportOption; +import com.tngtech.archunit.junit.AnalyzeClasses; +import com.tngtech.archunit.junit.ArchTest; +import com.tngtech.archunit.junit.ArchTests; + +/** Architecture tests for test code. */ +@AnalyzeClasses( + packages = {"org.apache.flink.formats.json"}, + importOptions = { + ImportOption.OnlyIncludeTests.class, + ImportOptions.ExcludeScalaImportOption.class, + ImportOptions.ExcludeShadedImportOption.class + }) +public class TestCodeArchitectureTest { + + @ArchTest + public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/archunit.properties b/flink-formats-kafka/flink-json-debezium/src/test/resources/archunit.properties new file mode 100644 index 000000000..15be88c95 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/archunit.properties @@ -0,0 +1,31 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# By default we allow removing existing violations, but fail when new violations are added. +freeze.store.default.allowStoreUpdate=true + +# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. +#freeze.store.default.allowStoreCreation=true + +# Enable this to add allow new violations to be recorded. +# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new +# violation, please try to avoid creating the violation. If the violation was created due to a +# shortcoming of the rule, file a JIRA issue so the rule can be improved. +#freeze.refreeze=true + +freeze.store.default.path=archunit-violations From 3507b288e4cd8815e104dc413d4f0e1c0a72710c Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 20 Apr 2022 13:55:43 +0200 Subject: [PATCH 184/322] [FLINK-24721][build] Remove unnecessary relativePath to parent --- flink-formats-kafka/flink-json-debezium/pom.xml | 1 - flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml | 1 - 2 files changed, 2 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 967652a46..ebef86486 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -26,7 +26,6 @@ under the License. org.apache.flink flink-formats 1.16-SNAPSHOT - .. flink-json diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index 1827f7597..2ce0a31d0 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -26,7 +26,6 @@ under the License. org.apache.flink flink-formats 1.16-SNAPSHOT - .. flink-sql-avro-confluent-registry From 2371b929eb29c6c1ec907c112af0987bf260853b Mon Sep 17 00:00:00 2001 From: BiGsuw Date: Tue, 3 May 2022 16:10:17 +0800 Subject: [PATCH 185/322] [FLINK-27352][json][tests] Migrate flink-json to JUnit5 --- .../json/JsonBatchFileSystemITCase.java | 7 +- .../formats/json/JsonFormatFactoryTest.java | 16 +- .../JsonNodeDeserializationSchemaTest.java | 12 +- .../json/JsonRowDataSerDeSchemaTest.java | 138 ++++++++---------- .../json/JsonRowSchemaConverterTest.java | 56 +++---- .../canal/CanalJsonFormatFactoryTest.java | 69 +++++---- .../json/canal/CanalJsonSerDeSchemaTest.java | 72 +++++---- .../DebeziumJsonFileSystemITCase.java | 28 ++-- .../DebeziumJsonFormatFactoryTest.java | 97 ++++++------ .../debezium/DebeziumJsonSerDeSchemaTest.java | 129 ++++++++-------- .../maxwell/MaxwellJsonFormatFactoryTest.java | 63 ++++---- .../json/maxwell/MaxwellJsonSerDerTest.java | 45 +++--- .../json/ogg/OggJsonFileSystemITCase.java | 28 ++-- .../json/ogg/OggJsonFormatFactoryTest.java | 61 ++++---- .../json/ogg/OggJsonSerDeSchemaTest.java | 56 +++---- .../org.junit.jupiter.api.extension.Extension | 16 ++ 16 files changed, 440 insertions(+), 453 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/services/org.junit.jupiter.api.extension.Extension diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java index 6cb7e080b..b2efe8c86 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java @@ -24,7 +24,6 @@ import org.apache.flink.types.Row; import org.apache.flink.util.FileUtils; -import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -36,6 +35,8 @@ import java.util.Arrays; import java.util.List; +import static org.assertj.core.api.Assertions.assertThat; + /** ITCase to test json format for {@link JsonFormatFactory}. */ public class JsonBatchFileSystemITCase extends BatchFileSystemITCaseBase { @@ -88,7 +89,7 @@ public void bigDataTest() throws IOException { TableResult result = tEnv().executeSql("select * from bigdata_source"); List elements = new ArrayList<>(); result.collect().forEachRemaining(r -> elements.add((String) r.getField(1))); - Assert.assertEquals(numRecords, elements.size()); + assertThat(elements).hasSize(numRecords); elements.sort(String::compareTo); List expected = new ArrayList<>(); @@ -97,7 +98,7 @@ public void bigDataTest() throws IOException { } expected.sort(String::compareTo); - Assert.assertEquals(expected, elements); + assertThat(elements).isEqualTo(expected); } private static File generateTestData(int numRecords) throws IOException { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java index e22a67aeb..7c1c553a6 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java @@ -46,10 +46,10 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link JsonFormatFactory}. */ -public class JsonFormatFactoryTest { +class JsonFormatFactoryTest { @Test - public void testSeDeSchema() { + void testSeDeSchema() { final Map tableOptions = getAllOptions(); testSchemaSerializationSchema(tableOptions); @@ -57,7 +57,7 @@ public void testSeDeSchema() { } @Test - public void testFailOnMissingField() { + void testFailOnMissingField() { final Map tableOptions = getModifyOptions(options -> options.put("json.fail-on-missing-field", "true")); @@ -69,7 +69,7 @@ public void testFailOnMissingField() { } @Test - public void testInvalidOptionForIgnoreParseErrors() { + void testInvalidOptionForIgnoreParseErrors() { final Map tableOptions = getModifyOptions(options -> options.put("json.ignore-parse-errors", "abc")); @@ -81,7 +81,7 @@ public void testInvalidOptionForIgnoreParseErrors() { } @Test - public void testInvalidOptionForTimestampFormat() { + void testInvalidOptionForTimestampFormat() { final Map tableOptions = getModifyOptions(options -> options.put("json.timestamp-format.standard", "test")); @@ -93,7 +93,7 @@ public void testInvalidOptionForTimestampFormat() { } @Test - public void testLowerCaseOptionForTimestampFormat() { + void testLowerCaseOptionForTimestampFormat() { final Map tableOptions = getModifyOptions( options -> options.put("json.timestamp-format.standard", "iso-8601")); @@ -106,7 +106,7 @@ public void testLowerCaseOptionForTimestampFormat() { } @Test - public void testInvalidOptionForMapNullKeyMode() { + void testInvalidOptionForMapNullKeyMode() { final Map tableOptions = getModifyOptions(options -> options.put("json.map-null-key.mode", "invalid")); @@ -118,7 +118,7 @@ public void testInvalidOptionForMapNullKeyMode() { } @Test - public void testLowerCaseOptionForMapNullKeyMode() { + void testLowerCaseOptionForMapNullKeyMode() { final Map tableOptions = getModifyOptions(options -> options.put("json.map-null-key.mode", "fail")); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java index 43ca2033c..741b492de 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java @@ -20,17 +20,17 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link JsonNodeDeserializationSchema}. */ -public class JsonNodeDeserializationSchemaTest { +class JsonNodeDeserializationSchemaTest { @Test - public void testDeserialize() throws IOException { + void testDeserialize() throws IOException { ObjectMapper mapper = new ObjectMapper(); ObjectNode initialValue = mapper.createObjectNode(); initialValue.put("key", 4).put("value", "world"); @@ -39,7 +39,7 @@ public void testDeserialize() throws IOException { JsonNodeDeserializationSchema schema = new JsonNodeDeserializationSchema(); ObjectNode deserializedValue = schema.deserialize(serializedValue); - assertEquals(4, deserializedValue.get("key").asInt()); - assertEquals("world", deserializedValue.get("value").asText()); + assertThat(deserializedValue.get("key").asInt()).isEqualTo(4); + assertThat(deserializedValue.get("value").asText()).isEqualTo("world"); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index e14849b74..fb30702bf 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -19,7 +19,7 @@ package org.apache.flink.formats.json; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.core.testutils.FlinkMatchers; +import org.apache.flink.core.testutils.FlinkAssertions; import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.data.GenericMapData; import org.apache.flink.table.data.GenericRowData; @@ -35,8 +35,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.math.BigDecimal; import java.sql.Timestamp; @@ -53,6 +52,7 @@ import java.util.Random; import java.util.concurrent.ThreadLocalRandom; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.api.DataTypes.ARRAY; import static org.apache.flink.table.api.DataTypes.BIGINT; import static org.apache.flink.table.api.DataTypes.BOOLEAN; @@ -73,18 +73,16 @@ import static org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE; import static org.apache.flink.table.api.DataTypes.TINYINT; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * Tests for {@link JsonRowDataDeserializationSchema} and {@link JsonRowDataSerializationSchema}. */ -public class JsonRowDataSerDeSchemaTest { +class JsonRowDataSerDeSchemaTest { @Test - public void testSerDe() throws Exception { + void testSerDe() throws Exception { byte tinyint = 'c'; short smallint = 128; int intValue = 45536; @@ -190,7 +188,7 @@ public void testSerDe() throws Exception { RowData rowData = deserializationSchema.deserialize(serializedJson); Row actual = convertToExternal(rowData, dataType); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); // test serialization JsonRowDataSerializationSchema serializationSchema = @@ -202,7 +200,7 @@ public void testSerDe() throws Exception { true); byte[] actualBytes = serializationSchema.serialize(rowData); - assertEquals(new String(serializedJson), new String(actualBytes)); + assertThat(serializedJson).containsExactly(actualBytes); } /** @@ -210,7 +208,7 @@ public void testSerDe() throws Exception { * Double#parseDouble(String)}. */ @Test - public void testSlowDeserialization() throws Exception { + void testSlowDeserialization() throws Exception { Random random = new Random(); boolean bool = random.nextBoolean(); int integer = random.nextInt(); @@ -260,11 +258,11 @@ public void testSlowDeserialization() throws Exception { RowData rowData = deserializationSchema.deserialize(serializedJson); Row actual = convertToExternal(rowData, dataType); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test - public void testSerDeMultiRows() throws Exception { + void testSerDeMultiRows() throws Exception { RowType rowType = (RowType) ROW( @@ -310,7 +308,7 @@ public void testSerDeMultiRows() throws Exception { byte[] serializedJson = objectMapper.writeValueAsBytes(root); RowData rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); - assertEquals(new String(serializedJson), new String(actual)); + assertThat(serializedJson).containsExactly(actual); } // the second row @@ -330,12 +328,12 @@ public void testSerDeMultiRows() throws Exception { byte[] serializedJson = objectMapper.writeValueAsBytes(root); RowData rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); - assertEquals(new String(serializedJson), new String(actual)); + assertThat(serializedJson).containsExactly(actual); } } @Test - public void testSerDeMultiRowsWithNullValues() throws Exception { + void testSerDeMultiRowsWithNullValues() throws Exception { String[] jsons = new String[] { "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{\"k1\":10.01,\"k2\":\"invalid\"}}", @@ -380,12 +378,12 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { String json = jsons[i]; RowData row = deserializationSchema.deserialize(json.getBytes()); String result = new String(serializationSchema.serialize(row)); - assertEquals(expected[i], result); + assertThat(result).isEqualTo(expected[i]); } } @Test - public void testDeserializationNullRow() throws Exception { + void testDeserializationNullRow() throws Exception { DataType dataType = ROW(FIELD("name", STRING())); RowType schema = (RowType) dataType.getLogicalType(); @@ -393,11 +391,11 @@ public void testDeserializationNullRow() throws Exception { new JsonRowDataDeserializationSchema( schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); - assertNull(deserializationSchema.deserialize(null)); + assertThat(deserializationSchema.deserialize(null)).isNull(); } @Test - public void testDeserializationMissingNode() throws Exception { + void testDeserializationMissingNode() throws Exception { DataType dataType = ROW(FIELD("name", STRING())); RowType schema = (RowType) dataType.getLogicalType(); @@ -405,11 +403,11 @@ public void testDeserializationMissingNode() throws Exception { new JsonRowDataDeserializationSchema( schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); RowData rowData = deserializationSchema.deserialize("".getBytes()); - assertEquals(null, rowData); + assertThat(rowData).isNull(); } @Test - public void testDeserializationMissingField() throws Exception { + void testDeserializationMissingField() throws Exception { ObjectMapper objectMapper = new ObjectMapper(); // Root @@ -431,7 +429,7 @@ public void testDeserializationMissingField() throws Exception { Row expected = new Row(1); Row actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); // fail on missing field deserializationSchema = @@ -439,34 +437,33 @@ public void testDeserializationMissingField() throws Exception { schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); String errorMessage = "Failed to deserialize JSON '{\"id\":123123123}'."; - try { - deserializationSchema.deserialize(serializedJson); - fail("expecting exception message: " + errorMessage); - } catch (Throwable t) { - assertEquals(errorMessage, t.getMessage()); - } + + JsonRowDataDeserializationSchema finalDeserializationSchema = deserializationSchema; + assertThatThrownBy(() -> finalDeserializationSchema.deserialize(serializedJson)) + .hasMessage(errorMessage); // ignore on parse error deserializationSchema = new JsonRowDataDeserializationSchema( schema, InternalTypeInfo.of(schema), false, true, TimestampFormat.ISO_8601); actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); errorMessage = "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."; - try { - // failOnMissingField and ignoreParseErrors both enabled - new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), true, true, TimestampFormat.ISO_8601); - Assert.fail("expecting exception message: " + errorMessage); - } catch (Throwable t) { - assertEquals(errorMessage, t.getMessage()); - } + assertThatThrownBy( + () -> + new JsonRowDataDeserializationSchema( + schema, + InternalTypeInfo.of(schema), + true, + true, + TimestampFormat.ISO_8601)) + .hasMessage(errorMessage); } @Test - public void testSerDeSQLTimestampFormat() throws Exception { + void testSerDeSQLTimestampFormat() throws Exception { RowType rowType = (RowType) ROW( @@ -501,11 +498,11 @@ public void testSerDeSQLTimestampFormat() throws Exception { byte[] serializedJson = objectMapper.writeValueAsBytes(root); RowData rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); - assertEquals(new String(serializedJson), new String(actual)); + assertThat(serializedJson).containsExactly(actual); } @Test - public void testSerializationMapNullKey() throws Exception { + void testSerializationMapNullKey() { RowType rowType = (RowType) ROW(FIELD("nestedMap", MAP(STRING(), MAP(STRING(), INT())))) @@ -559,25 +556,20 @@ public void testSerializationMapNullKey() throws Exception { String expectResult3 = "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1,\"nullKey\":2},\"nullKey\":{\"no-null key\":1,\"nullKey\":2}}}"; - try { - // throw exception when mapNullKey Mode is fail - serializationSchema1.serialize(rowData); - Assert.fail("expecting exception message: " + errorMessage1); - } catch (Throwable t) { - assertThat(t, FlinkMatchers.containsMessage(errorMessage1)); - } + assertThatThrownBy(() -> serializationSchema1.serialize(rowData)) + .satisfies(FlinkAssertions.anyCauseMatches(errorMessage1)); // mapNullKey Mode is drop byte[] actual2 = serializationSchema2.serialize(rowData); - assertEquals(expectResult2, new String(actual2)); + assertThat(new String(actual2)).isEqualTo(expectResult2); // mapNullKey Mode is literal byte[] actual3 = serializationSchema3.serialize(rowData); - assertEquals(expectResult3, new String(actual3)); + assertThat(new String(actual3)).isEqualTo(expectResult3); } @Test - public void testSerializationDecimalEncode() throws Exception { + void testSerializationDecimalEncode() throws Exception { RowType schema = (RowType) ROW( @@ -612,17 +604,17 @@ public void testSerializationDecimalEncode() throws Exception { RowData rowData = deserializer.deserialize(plainDecimalJson.getBytes()); String plainDecimalResult = new String(plainDecimalSerializer.serialize(rowData)); - assertEquals(plainDecimalJson, plainDecimalResult); + assertThat(plainDecimalResult).isEqualTo(plainDecimalJson); String scientificDecimalJson = "{\"decimal1\":123.456789,\"decimal2\":4.5462186404924617E+17,\"decimal3\":2.7E-8}"; String scientificDecimalResult = new String(scientificDecimalSerializer.serialize(rowData)); - assertEquals(scientificDecimalJson, scientificDecimalResult); + assertThat(scientificDecimalResult).isEqualTo(scientificDecimalJson); } @Test - public void testJsonParse() throws Exception { + void testJsonParse() throws Exception { for (TestSpec spec : testData) { testIgnoreParseErrors(spec); if (spec.errorMessage != null) { @@ -632,7 +624,7 @@ public void testJsonParse() throws Exception { } @Test - public void testSerializationWithTypesMismatch() { + void testSerializationWithTypesMismatch() { RowType rowType = (RowType) ROW(FIELD("f0", INT()), FIELD("f1", STRING())).getLogicalType(); GenericRowData genericRowData = new GenericRowData(2); genericRowData.setField(0, 1); @@ -645,28 +637,22 @@ public void testSerializationWithTypesMismatch() { "null", true); String errorMessage = "Fail to serialize at field: f1."; - try { - serializationSchema.serialize(genericRowData); - fail("expecting exception message: " + errorMessage); - } catch (Throwable t) { - assertThat(t, FlinkMatchers.containsMessage(errorMessage)); - } + + assertThatThrownBy(() -> serializationSchema.serialize(genericRowData)) + .satisfies(anyCauseMatches(RuntimeException.class, errorMessage)); } @Test - public void testDeserializationWithTypesMismatch() { + void testDeserializationWithTypesMismatch() { RowType rowType = (RowType) ROW(FIELD("f0", STRING()), FIELD("f1", INT())).getLogicalType(); String json = "{\"f0\":\"abc\", \"f1\": \"abc\"}"; JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); String errorMessage = "Fail to deserialize at field: f1."; - try { - deserializationSchema.deserialize(json.getBytes()); - fail("expecting exception message: " + errorMessage); - } catch (Throwable t) { - assertThat(t, FlinkMatchers.containsMessage(errorMessage)); - } + + assertThatThrownBy(() -> deserializationSchema.deserialize(json.getBytes())) + .satisfies(anyCauseMatches(errorMessage)); } private void testIgnoreParseErrors(TestSpec spec) throws Exception { @@ -686,10 +672,12 @@ private void testIgnoreParseErrors(TestSpec spec) throws Exception { } RowData rowData = ignoreErrorsSchema.deserialize(spec.json.getBytes()); Row actual = convertToExternal(rowData, fromLogicalToDataType(spec.rowType)); - assertEquals("Test Ignore Parse Error: " + spec.json, expected, actual); + assertThat(actual) + .isEqualTo(expected) + .withFailMessage("Test Ignore Parse Error: " + spec.json); } - private void testParseErrors(TestSpec spec) throws Exception { + private void testParseErrors(TestSpec spec) { // expect exception if parse error is not ignored JsonRowDataDeserializationSchema failingSchema = new JsonRowDataDeserializationSchema( @@ -699,12 +687,8 @@ private void testParseErrors(TestSpec spec) throws Exception { false, spec.timestampFormat); - try { - failingSchema.deserialize(spec.json.getBytes()); - fail("expecting exception " + spec.errorMessage); - } catch (Throwable t) { - assertEquals(t.getMessage(), spec.errorMessage); - } + assertThatThrownBy(() -> failingSchema.deserialize(spec.json.getBytes())) + .hasMessageContaining(spec.errorMessage); } private static List testData = diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java index 74d4d03da..b15444dc5 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java @@ -22,19 +22,20 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.util.FileUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.File; import java.net.URL; import java.util.Objects; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link JsonRowSchemaConverter}. */ -public class JsonRowSchemaConverterTest { +class JsonRowSchemaConverterTest { @Test - public void testComplexSchema() throws Exception { + void testComplexSchema() throws Exception { final URL url = getClass().getClassLoader().getResource("complex-schema.json"); Objects.requireNonNull(url); final String schema = FileUtils.readFileUtf8(new File(url.getFile())); @@ -66,11 +67,11 @@ public void testComplexSchema() throws Exception { Types.VOID, Types.ROW_NAMED(new String[] {"organizationUnit"}, Types.ROW())); - assertEquals(expected, result); + assertThat(result).isEqualTo(expected); } @Test - public void testReferenceSchema() throws Exception { + void testReferenceSchema() throws Exception { final URL url = getClass().getClassLoader().getResource("reference-schema.json"); Objects.requireNonNull(url); final String schema = FileUtils.readFileUtf8(new File(url.getFile())); @@ -95,52 +96,57 @@ public void testReferenceSchema() throws Exception { Types.STRING, Types.STRING)); - assertEquals(expected, result); + assertThat(result).isEqualTo(expected); } @Test - public void testAtomicType() { + void testAtomicType() { final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'number' }"); - assertEquals(Types.BIG_DEC, result); + assertThat(result).isEqualTo(Types.BIG_DEC); } - @Test(expected = IllegalArgumentException.class) - public void testMissingType() { - JsonRowSchemaConverter.convert("{ }"); + @Test + void testMissingType() { + assertThatThrownBy(() -> JsonRowSchemaConverter.convert("{ }")) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testWrongType() { - JsonRowSchemaConverter.convert("{ type: 'whatever' }"); + @Test + void testWrongType() { + assertThatThrownBy(() -> JsonRowSchemaConverter.convert("{ type: 'whatever' }")) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testArrayWithAdditionalItems() { - JsonRowSchemaConverter.convert( - "{ type: 'array', items: [{type: 'integer'}], additionalItems: true }"); + @Test + void testArrayWithAdditionalItems() { + assertThatThrownBy( + () -> + JsonRowSchemaConverter.convert( + "{ type: 'array', items: [{type: 'integer'}], additionalItems: true }")) + .isInstanceOf(IllegalArgumentException.class); } @Test - public void testMissingProperties() { + void testMissingProperties() { final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'object' }"); - assertEquals(Types.ROW(), result); + assertThat(result).isEqualTo(Types.ROW()); } @Test - public void testNullUnionTypes() { + void testNullUnionTypes() { final TypeInformation result = JsonRowSchemaConverter.convert("{ type: ['string', 'null'] }"); - assertEquals(Types.STRING, result); + assertThat(result).isEqualTo(Types.STRING); } @Test - public void testTimestamp() { + void testTimestamp() { final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'string', format: 'date-time' }"); - assertEquals(Types.SQL_TIMESTAMP, result); + assertThat(result).isEqualTo(Types.SQL_TIMESTAMP); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java index 8c8f8443d..00bd5a062 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java @@ -30,34 +30,31 @@ import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.util.TestLogger; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link CanalJsonFormatFactory}. */ -public class CanalJsonFormatFactoryTest extends TestLogger { - @Rule public ExpectedException thrown = ExpectedException.none(); +class CanalJsonFormatFactoryTest { private static final InternalTypeInfo ROW_TYPE_INFO = InternalTypeInfo.of(PHYSICAL_TYPE); @Test - public void testDefaultOptions() { + void testDefaultOptions() { Map options = getAllOptions(); // test Deser @@ -68,7 +65,7 @@ public void testDefaultOptions() { .setTimestampFormat(TimestampFormat.SQL) .build(); DeserializationSchema actualDeser = createDeserializationSchema(options); - assertEquals(expectedDeser, actualDeser); + assertThat(actualDeser).isEqualTo(expectedDeser); // test Ser CanalJsonSerializationSchema expectedSer = @@ -79,11 +76,11 @@ public void testDefaultOptions() { "null", false); SerializationSchema actualSer = createSerializationSchema(options); - assertEquals(expectedSer, actualSer); + assertThat(actualSer).isEqualTo(expectedSer); } @Test - public void testUserDefinedOptions() { + void testUserDefinedOptions() { Map options = getAllOptions(); options.put("canal-json.ignore-parse-errors", "true"); options.put("canal-json.timestamp-format.standard", "ISO-8601"); @@ -103,7 +100,7 @@ public void testUserDefinedOptions() { .setTable("mytable") .build(); DeserializationSchema actualDeser = createDeserializationSchema(options); - assertEquals(expectedDeser, actualDeser); + assertThat(actualDeser).isEqualTo(expectedDeser); // test Ser CanalJsonSerializationSchema expectedSer = @@ -114,47 +111,49 @@ public void testUserDefinedOptions() { "nullKey", true); SerializationSchema actualSer = createSerializationSchema(options); - assertEquals(expectedSer, actualSer); + assertThat(actualSer).isEqualTo(expectedSer); } @Test - public void testInvalidIgnoreParseError() { - thrown.expect( - containsCause( - new IllegalArgumentException( - "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); - + void testInvalidIgnoreParseError() { final Map options = getModifiedOptions(opts -> opts.put("canal-json.ignore-parse-errors", "abc")); - createDeserializationSchema(options); + assertThatThrownBy(() -> createDeserializationSchema(options)) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Unrecognized option for boolean: abc. Expected either true" + + " or false(case insensitive)")); } @Test - public void testInvalidOptionForTimestampFormat() { + void testInvalidOptionForTimestampFormat() { final Map tableOptions = getModifiedOptions( opts -> opts.put("canal-json.timestamp-format.standard", "test")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - createDeserializationSchema(tableOptions); + assertThatThrownBy(() -> createDeserializationSchema(tableOptions)) + .isInstanceOf(ValidationException.class) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'test' for timestamp-format.standard. " + + "Supported values are [SQL, ISO-8601].")); } @Test - public void testInvalidOptionForMapNullKeyMode() { + void testInvalidOptionForMapNullKeyMode() { final Map tableOptions = getModifiedOptions(opts -> opts.put("canal-json.map-null-key.mode", "invalid")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - createSerializationSchema(tableOptions); + assertThatThrownBy(() -> createSerializationSchema(tableOptions)) + .isInstanceOf(ValidationException.class) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'invalid' for option map-null-key.mode. " + + "Supported values are [LITERAL, FAIL, DROP].")); } // ------------------------------------------------------------------------ diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index d7f70ecc0..1083ae656 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -29,9 +29,7 @@ import org.apache.flink.table.types.utils.DataTypeUtils; import org.apache.flink.util.Collector; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -51,14 +49,10 @@ import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link CanalJsonSerializationSchema} and {@link CanalJsonDeserializationSchema}. */ -public class CanalJsonSerDeSchemaTest { - - @Rule public ExpectedException thrown = ExpectedException.none(); +class CanalJsonSerDeSchemaTest { private static final DataType PHYSICAL_DATA_TYPE = ROW( @@ -68,7 +62,7 @@ public class CanalJsonSerDeSchemaTest { FIELD("weight", FLOAT())); @Test - public void testFilteringTables() throws Exception { + void testFilteringTables() throws Exception { List lines = readLines("canal-data-filter-table.txt"); CanalJsonDeserializationSchema deserializationSchema = CanalJsonDeserializationSchema.builder( @@ -82,7 +76,7 @@ public void testFilteringTables() throws Exception { } @Test - public void testDeserializeNullRow() throws Exception { + void testDeserializeNullRow() throws Exception { final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); final CanalJsonDeserializationSchema deserializationSchema = createCanalJsonDeserializationSchema(null, null, requestedMetadata); @@ -90,47 +84,47 @@ public void testDeserializeNullRow() throws Exception { deserializationSchema.deserialize(null, collector); deserializationSchema.deserialize(new byte[0], collector); - assertEquals(0, collector.list.size()); + assertThat(collector.list).hasSize(0); } @Test - public void testDeserializationWithMetadata() throws Exception { + void testDeserializationWithMetadata() throws Exception { testDeserializationWithMetadata( "canal-data.txt", null, null, row -> { - assertThat(row.getInt(0), equalTo(101)); - assertThat(row.getString(1).toString(), equalTo("scooter")); - assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); - assertThat(row.getFloat(3), equalTo(3.14f)); - assertThat(row.getString(4).toString(), equalTo("inventory")); - assertThat(row.getString(5).toString(), equalTo("products2")); - assertThat(row.getMap(6).size(), equalTo(4)); - assertThat(row.getArray(7).getString(0).toString(), equalTo("id")); - assertThat(row.getTimestamp(8, 3).getMillisecond(), equalTo(1589373515477L)); - assertThat(row.getTimestamp(9, 3).getMillisecond(), equalTo(1589373515000L)); + assertThat(row.getInt(0)).isEqualTo(101); + assertThat(row.getString(1).toString()).isEqualTo("scooter"); + assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter"); + assertThat(row.getFloat(3)).isEqualTo(3.14f); + assertThat(row.getString(4).toString()).isEqualTo("inventory"); + assertThat(row.getString(5).toString()).isEqualTo("products2"); + assertThat(row.getMap(6).size()).isEqualTo(4); + assertThat(row.getArray(7).getString(0).toString()).isEqualTo("id"); + assertThat(row.getTimestamp(8, 3).getMillisecond()).isEqualTo(1589373515477L); + assertThat(row.getTimestamp(9, 3).getMillisecond()).isEqualTo(1589373515000L); }); testDeserializationWithMetadata( "canal-data-filter-table.txt", "mydb", "product", row -> { - assertThat(row.getInt(0), equalTo(101)); - assertThat(row.getString(1).toString(), equalTo("scooter")); - assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); - assertThat(row.getFloat(3), equalTo(3.14f)); - assertThat(row.getString(4).toString(), equalTo("mydb")); - assertThat(row.getString(5).toString(), equalTo("product")); - assertThat(row.getMap(6).size(), equalTo(4)); - assertThat(row.getArray(7).getString(0).toString(), equalTo("id")); - assertThat(row.getTimestamp(8, 3).getMillisecond(), equalTo(1598944146308L)); - assertThat(row.getTimestamp(9, 3).getMillisecond(), equalTo(1598944132000L)); + assertThat(row.getInt(0)).isEqualTo(101); + assertThat(row.getString(1).toString()).isEqualTo("scooter"); + assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter"); + assertThat(row.getFloat(3)).isEqualTo(3.14f); + assertThat(row.getString(4).toString()).isEqualTo("mydb"); + assertThat(row.getString(5).toString()).isEqualTo("product"); + assertThat(row.getMap(6).size()).isEqualTo(4); + assertThat(row.getArray(7).getString(0).toString()).isEqualTo("id"); + assertThat(row.getTimestamp(8, 3).getMillisecond()).isEqualTo(1598944146308L); + assertThat(row.getTimestamp(9, 3).getMillisecond()).isEqualTo(1598944132000L); }); } @Test - public void testSerializationDeserialization() throws Exception { + void testSerializationDeserialization() throws Exception { List lines = readLines("canal-data.txt"); CanalJsonDeserializationSchema deserializationSchema = CanalJsonDeserializationSchema.builder( @@ -211,7 +205,7 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); // test Serialization CanalJsonSerializationSchema serializationSchema = @@ -257,7 +251,7 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"DELETE\"}", "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"DELETE\"}"); - assertEquals(expectedResult, result); + assertThat(result).isEqualTo(expectedResult); } private void testDeserializationWithMetadata( @@ -271,8 +265,8 @@ private void testDeserializationWithMetadata( final SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - assertEquals(9, collector.list.size()); - testConsumer.accept(collector.list.get(0)); + assertThat(collector.list).hasSize(9); + assertThat(collector.list.get(0)).satisfies(testConsumer); } private CanalJsonDeserializationSchema createCanalJsonDeserializationSchema( @@ -307,7 +301,7 @@ private static List readLines(String resource) throws IOException { private static class SimpleCollector implements Collector { - private List list = new ArrayList<>(); + private final List list = new ArrayList<>(); @Override public void collect(RowData record) { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java index 7ab0e90af..0c84351f0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java @@ -23,8 +23,8 @@ import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.CollectionUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import java.io.File; import java.io.IOException; @@ -36,9 +36,10 @@ import java.util.stream.Collectors; import static java.lang.String.format; +import static org.assertj.core.api.Assertions.assertThat; /** Test Filesystem connector with DebeziumJson. */ -public class DebeziumJsonFileSystemITCase extends StreamingTestBase { +class DebeziumJsonFileSystemITCase extends StreamingTestBase { private static final List EXPECTED = Arrays.asList( @@ -66,9 +67,10 @@ public class DebeziumJsonFileSystemITCase extends StreamingTestBase { private File source; private File sink; - private void prepareTables(boolean isPartition) throws IOException { + private void prepareTables(boolean isPartition, Path tempSourceDir, Path tempSinkDir) + throws IOException { byte[] bytes = readBytes("debezium-data-schema-exclude.txt"); - source = TEMPORARY_FOLDER.newFolder(); + source = tempSourceDir.toFile(); File file; if (isPartition) { File partition = new File(source, "p=1"); @@ -80,7 +82,7 @@ private void prepareTables(boolean isPartition) throws IOException { file.createNewFile(); Files.write(file.toPath(), bytes); - sink = TEMPORARY_FOLDER.newFolder(); + sink = tempSinkDir.toFile(); env().setParallelism(1); } @@ -100,8 +102,8 @@ private void createTable(boolean isSink, String path, boolean isPartition) { } @Test - public void testNonPartition() throws Exception { - prepareTables(false); + void testNonPartition(@TempDir Path tempSourceDir, @TempDir Path tempSinkDir) throws Exception { + prepareTables(false, tempSourceDir, tempSinkDir); createTable(false, source.toURI().toString(), false); createTable(true, sink.toURI().toString(), false); @@ -117,12 +119,12 @@ public void testNonPartition() throws Exception { .collect(Collectors.toList()); iter.close(); - Assert.assertEquals(EXPECTED, results); + assertThat(results).isEqualTo(EXPECTED); } @Test - public void testPartition() throws Exception { - prepareTables(true); + void testPartition(@TempDir Path tempSourceDir, @TempDir Path tempSinkDir) throws Exception { + prepareTables(true, tempSourceDir, tempSinkDir); createTable(false, source.toURI().toString(), true); createTable(true, sink.toURI().toString(), true); @@ -140,11 +142,11 @@ public void testPartition() throws Exception { .map(Row::toString) .collect(Collectors.toList()); - Assert.assertEquals(EXPECTED, results); + assertThat(results).isEqualTo(EXPECTED); // check partition value for (Row row : list) { - Assert.assertEquals(1, row.getField(4)); + assertThat(row.getField(4)).isEqualTo(1); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java index 1eadf9c59..d000877b2 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java @@ -31,32 +31,28 @@ import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.util.TestLogger; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link DebeziumJsonFormatFactory}. */ -public class DebeziumJsonFormatFactoryTest extends TestLogger { - @Rule public ExpectedException thrown = ExpectedException.none(); +class DebeziumJsonFormatFactoryTest { @Test - public void testSeDeSchema() { + void testSeDeSchema() { final DebeziumJsonDeserializationSchema expectedDeser = new DebeziumJsonDeserializationSchema( PHYSICAL_DATA_TYPE, @@ -77,7 +73,7 @@ public void testSeDeSchema() { scanSourceMock.valueFormat.createRuntimeDecoder( ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); - assertEquals(expectedDeser, actualDeser); + assertThat(actualDeser).isEqualTo(expectedDeser); final DebeziumJsonSerializationSchema expectedSer = new DebeziumJsonSerializationSchema( @@ -96,24 +92,24 @@ public void testSeDeSchema() { sinkMock.valueFormat.createRuntimeEncoder( new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); - assertEquals(expectedSer, actualSer); + assertThat(actualSer).isEqualTo(expectedSer); } @Test - public void testInvalidIgnoreParseError() { - thrown.expect( - containsCause( - new IllegalArgumentException( - "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); - + void testInvalidIgnoreParseError() { final Map options = getModifiedOptions(opts -> opts.put("debezium-json.ignore-parse-errors", "abc")); - createTableSource(SCHEMA, options); + assertThatThrownBy(() -> createTableSource(SCHEMA, options)) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Unrecognized option for boolean: abc. " + + "Expected either true or false(case insensitive)")); } @Test - public void testSchemaIncludeOption() { + void testSchemaIncludeOption() { Map options = getAllOptions(); options.put("debezium-json.schema-include", "true"); @@ -131,49 +127,50 @@ public void testSchemaIncludeOption() { DeserializationSchema actualDeser = scanSourceMock.valueFormat.createRuntimeDecoder( ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); - assertEquals(expectedDeser, actualDeser); - - try { - final DynamicTableSink actualSink = createTableSink(SCHEMA, options); - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - // should fail - sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); - fail(); - } catch (Exception e) { - assertEquals( - e.getCause().getCause().getMessage(), - "Debezium JSON serialization doesn't support " - + "'debezium-json.schema-include' option been set to true."); - } + assertThat(actualDeser).isEqualTo(expectedDeser); + + assertThatThrownBy( + () -> { + final DynamicTableSink actualSink = createTableSink(SCHEMA, options); + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + sinkMock.valueFormat.createRuntimeEncoder( + new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); + }) + .satisfies( + anyCauseMatches( + RuntimeException.class, + "Debezium JSON serialization doesn't support " + + "'debezium-json.schema-include' option been set to true.")); } @Test - public void testInvalidOptionForTimestampFormat() { + void testInvalidOptionForTimestampFormat() { final Map tableOptions = getModifiedOptions( opts -> opts.put("debezium-json.timestamp-format.standard", "test")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - createTableSource(SCHEMA, tableOptions); + assertThatThrownBy(() -> createTableSource(SCHEMA, tableOptions)) + .isInstanceOf(ValidationException.class) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'test' for timestamp-format.standard. " + + "Supported values are [SQL, ISO-8601].")); } @Test - public void testInvalidOptionForMapNullKeyMode() { + void testInvalidOptionForMapNullKeyMode() { final Map tableOptions = getModifiedOptions(opts -> opts.put("debezium-json.map-null-key.mode", "invalid")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - createTableSink(SCHEMA, tableOptions); + assertThatThrownBy(() -> createTableSink(SCHEMA, tableOptions)) + .isInstanceOf(ValidationException.class) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'invalid' for option map-null-key.mode. " + + "Supported values are [LITERAL, FAIL, DROP].")); } // ------------------------------------------------------------------------ diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java index 2422dbd3a..ef3b1d8ab 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -28,11 +28,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.utils.DataTypeUtils; import org.apache.flink.util.Collector; -import org.apache.flink.util.ExceptionUtils; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -52,18 +49,13 @@ import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.startsWith; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * Tests for {@link DebeziumJsonSerializationSchema} and {@link DebeziumJsonDeserializationSchema}. */ -public class DebeziumJsonSerDeSchemaTest { - - @Rule public ExpectedException thrown = ExpectedException.none(); +class DebeziumJsonSerDeSchemaTest { private static final DataType PHYSICAL_DATA_TYPE = ROW( @@ -73,41 +65,39 @@ public class DebeziumJsonSerDeSchemaTest { FIELD("weight", FLOAT())); @Test - public void testSerializationAndSchemaIncludeDeserialization() throws Exception { + void testSerializationAndSchemaIncludeDeserialization() throws Exception { testSerializationDeserialization("debezium-data-schema-include.txt", true); } @Test - public void testSerializationAndSchemaExcludeDeserialization() throws Exception { + void testSerializationAndSchemaExcludeDeserialization() throws Exception { testSerializationDeserialization("debezium-data-schema-exclude.txt", false); } @Test - public void testSerializationAndPostgresSchemaIncludeDeserialization() throws Exception { + void testSerializationAndPostgresSchemaIncludeDeserialization() throws Exception { testSerializationDeserialization("debezium-postgres-data-schema-include.txt", true); } @Test - public void testSerializationAndPostgresSchemaExcludeDeserialization() throws Exception { + void testSerializationAndPostgresSchemaExcludeDeserialization() throws Exception { testSerializationDeserialization("debezium-postgres-data-schema-exclude.txt", false); } @Test - public void testPostgresDefaultReplicaIdentify() { - try { - testSerializationDeserialization("debezium-postgres-data-replica-identity.txt", false); - } catch (Exception e) { - assertTrue( - ExceptionUtils.findThrowableWithMessage( - e, - "The \"before\" field of UPDATE message is null, if you are using Debezium Postgres Connector, " - + "please check the Postgres table has been set REPLICA IDENTITY to FULL level.") - .isPresent()); - } + void testPostgresDefaultReplicaIdentify() { + assertThatThrownBy( + () -> + testSerializationDeserialization( + "debezium-postgres-data-replica-identity.txt", false)) + .as( + "The \"before\" field of UPDATE message is null, if you are using Debezium Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level.") + .isInstanceOf(Exception.class); } @Test - public void testTombstoneMessages() throws Exception { + void testTombstoneMessages() throws Exception { DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema( PHYSICAL_DATA_TYPE, @@ -119,63 +109,62 @@ public void testTombstoneMessages() throws Exception { SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(null, collector); deserializationSchema.deserialize(new byte[] {}, collector); - assertTrue(collector.list.isEmpty()); + assertThat(collector.list).isEmpty(); } @Test - public void testDeserializationWithMetadata() throws Exception { + void testDeserializationWithMetadata() throws Exception { testDeserializationWithMetadata( "debezium-data-schema-include.txt", true, row -> { - assertThat(row.getInt(0), equalTo(101)); - assertThat(row.getString(1).toString(), equalTo("scooter")); - assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); - assertThat(row.getFloat(3), equalTo(3.14f)); - assertThat( - row.getString(4).toString(), - startsWith( - "{\"type\":\"struct\",\"fields\":[{\"type\":\"struct\",\"fields\":[{\"type\":\"int32\",\"optional\":false,\"field\":\"id\"},")); - assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1589355606100L)); - assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(0L)); - assertThat(row.getString(7).toString(), equalTo("inventory")); - assertThat(row.isNullAt(8), equalTo(true)); - assertThat(row.getString(9).toString(), equalTo("products")); - assertThat(row.getMap(10).size(), equalTo(14)); + assertThat(row.getInt(0)).isEqualTo(101); + assertThat(row.getString(1).toString()).isEqualTo("scooter"); + assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter"); + assertThat(row.getFloat(3)).isEqualTo(3.14f); + assertThat(row.getString(4).toString()) + .startsWith( + "{\"type\":\"struct\",\"fields\":[{\"type\":\"struct\",\"fields\":[{\"type\":\"int32\",\"optional\":false,\"field\":\"id\"},"); + assertThat(row.getTimestamp(5, 3).getMillisecond()).isEqualTo(1589355606100L); + assertThat(row.getTimestamp(6, 3).getMillisecond()).isEqualTo(0L); + assertThat(row.getString(7).toString()).isEqualTo("inventory"); + assertThat(row.isNullAt(8)).isEqualTo(true); + assertThat(row.getString(9).toString()).isEqualTo("products"); + assertThat(row.getMap(10).size()).isEqualTo(14); }); testDeserializationWithMetadata( "debezium-data-schema-exclude.txt", false, row -> { - assertThat(row.getInt(0), equalTo(101)); - assertThat(row.getString(1).toString(), equalTo("scooter")); - assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); - assertThat(row.getFloat(3), equalTo(3.14f)); - assertThat(row.isNullAt(4), equalTo(true)); - assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1589355606100L)); - assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(0L)); - assertThat(row.getString(7).toString(), equalTo("inventory")); - assertThat(row.isNullAt(8), equalTo(true)); - assertThat(row.getString(9).toString(), equalTo("products")); - assertThat(row.getMap(10).size(), equalTo(14)); + assertThat(row.getInt(0)).isEqualTo(101); + assertThat(row.getString(1).toString()).isEqualTo("scooter"); + assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter"); + assertThat(row.getFloat(3)).isEqualTo(3.14f); + assertThat(row.isNullAt(4)).isEqualTo(true); + assertThat(row.getTimestamp(5, 3).getMillisecond()).isEqualTo(1589355606100L); + assertThat(row.getTimestamp(6, 3).getMillisecond()).isEqualTo(0L); + assertThat(row.getString(7).toString()).isEqualTo("inventory"); + assertThat(row.isNullAt(8)).isEqualTo(true); + assertThat(row.getString(9).toString()).isEqualTo("products"); + assertThat(row.getMap(10).size()).isEqualTo(14); }); testDeserializationWithMetadata( "debezium-postgres-data-schema-exclude.txt", false, row -> { - assertThat(row.getInt(0), equalTo(101)); - assertThat(row.getString(1).toString(), equalTo("scooter")); - assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); - assertThat(row.getFloat(3), equalTo(3.14f)); - assertThat(row.isNullAt(4), equalTo(true)); - assertThat(row.getTimestamp(5, 3).getMillisecond(), equalTo(1596001099434L)); - assertThat(row.getTimestamp(6, 3).getMillisecond(), equalTo(1596001099434L)); - assertThat(row.getString(7).toString(), equalTo("postgres")); - assertThat(row.getString(8).toString(), equalTo("inventory")); - assertThat(row.getString(9).toString(), equalTo("products")); - assertThat(row.getMap(10).size(), equalTo(11)); + assertThat(row.getInt(0)).isEqualTo(101); + assertThat(row.getString(1).toString()).isEqualTo("scooter"); + assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter"); + assertThat(row.getFloat(3)).isEqualTo(3.14f); + assertThat(row.isNullAt(4)).isEqualTo(true); + assertThat(row.getTimestamp(5, 3).getMillisecond()).isEqualTo(1596001099434L); + assertThat(row.getTimestamp(6, 3).getMillisecond()).isEqualTo(1596001099434L); + assertThat(row.getString(7).toString()).isEqualTo("postgres"); + assertThat(row.getString(8).toString()).isEqualTo("inventory"); + assertThat(row.getString(9).toString()).isEqualTo("products"); + assertThat(row.getMap(10).size()).isEqualTo(11); }); } @@ -250,7 +239,7 @@ private void testSerializationDeserialization(String resourceFile, boolean schem "-D(111,scooter,Big 2-wheel scooter ,5.17)"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); DebeziumJsonSerializationSchema serializationSchema = new DebeziumJsonSerializationSchema( @@ -288,7 +277,7 @@ private void testSerializationDeserialization(String resourceFile, boolean schem "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":null,\"op\":\"d\"}", "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op\":\"c\"}", "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op\":\"d\"}"); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } private void testDeserializationWithMetadata( @@ -318,8 +307,8 @@ private void testDeserializationWithMetadata( final SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - assertEquals(1, collector.list.size()); - testConsumer.accept(collector.list.get(0)); + assertThat(collector.list).hasSize(1); + assertThat(collector.list.get(0)).satisfies(testConsumer); } // -------------------------------------------------------------------------------------------- diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java index 3c517bac0..bc47d1e68 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java @@ -30,34 +30,31 @@ import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.util.TestLogger; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link MaxwellJsonFormatFactory}. */ -public class MaxwellJsonFormatFactoryTest extends TestLogger { - @Rule public ExpectedException thrown = ExpectedException.none(); +class MaxwellJsonFormatFactoryTest { private static final InternalTypeInfo ROW_TYPE_INFO = InternalTypeInfo.of(PHYSICAL_TYPE); @Test - public void testSeDeSchema() { + void testSeDeSchema() { final MaxwellJsonDeserializationSchema expectedDeser = new MaxwellJsonDeserializationSchema( PHYSICAL_DATA_TYPE, @@ -85,7 +82,7 @@ public void testSeDeSchema() { scanSourceMock.valueFormat.createRuntimeDecoder( ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); - assertEquals(expectedDeser, actualDeser); + assertThat(actualDeser).isEqualTo(expectedDeser); final DynamicTableSink actualSink = createTableSink(SCHEMA, options); assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; @@ -96,47 +93,49 @@ public void testSeDeSchema() { sinkMock.valueFormat.createRuntimeEncoder( new SinkRuntimeProviderContext(false), SCHEMA.toPhysicalRowDataType()); - assertEquals(expectedSer, actualSer); + assertThat(actualSer).isEqualTo(expectedSer); } @Test - public void testInvalidIgnoreParseError() { - thrown.expect( - containsCause( - new IllegalArgumentException( - "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); - + void testInvalidIgnoreParseError() { final Map options = getModifiedOptions(opts -> opts.put("maxwell-json.ignore-parse-errors", "abc")); - createTableSource(SCHEMA, options); + assertThatThrownBy(() -> createTableSource(SCHEMA, options)) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Unrecognized option for boolean: abc. " + + "Expected either true or false(case insensitive)")); } @Test - public void testInvalidOptionForTimestampFormat() { + void testInvalidOptionForTimestampFormat() { final Map tableOptions = getModifiedOptions( opts -> opts.put("maxwell-json.timestamp-format.standard", "test")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - createTableSource(SCHEMA, tableOptions); + assertThatThrownBy(() -> createTableSource(SCHEMA, tableOptions)) + .isInstanceOf(ValidationException.class) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'test' for timestamp-format.standard. " + + "Supported values are [SQL, ISO-8601].")); } @Test - public void testInvalidOptionForMapNullKeyMode() { + void testInvalidOptionForMapNullKeyMode() { final Map tableOptions = getModifiedOptions(opts -> opts.put("maxwell-json.map-null-key.mode", "invalid")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - createTableSink(SCHEMA, tableOptions); + assertThatThrownBy(() -> createTableSink(SCHEMA, tableOptions)) + .isInstanceOf(ValidationException.class) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'invalid' for option map-null-key.mode. " + + "Supported values are [LITERAL, FAIL, DROP].")); } // ------------------------------------------------------------------------ diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java index 9fb55dfce..ffd68b20b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java @@ -29,7 +29,7 @@ import org.apache.flink.table.types.utils.DataTypeUtils; import org.apache.flink.util.Collector; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -41,7 +41,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.function.Consumer; import java.util.stream.Collectors; import static org.apache.flink.table.api.DataTypes.FIELD; @@ -49,14 +48,12 @@ import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; /** * Tests for {@link MaxwellJsonSerializationSchema} and {@link MaxwellJsonDeserializationSchema}. */ -public class MaxwellJsonSerDerTest { +class MaxwellJsonSerDerTest { private static final DataType PHYSICAL_DATA_TYPE = ROW( @@ -66,7 +63,7 @@ public class MaxwellJsonSerDerTest { FIELD("weight", FLOAT())); @Test - public void testDeserializationWithMetadata() throws Exception { + void testDeserializationWithMetadata() throws Exception { // we only read the first line for keeping the test simple final String firstLine = readLines("maxwell-data.txt").get(0); final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); @@ -85,23 +82,25 @@ public void testDeserializationWithMetadata() throws Exception { TimestampFormat.ISO_8601); final SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - assertEquals(1, collector.list.size()); - Consumer consumer = - row -> { - assertThat(row.getInt(0), equalTo(101)); - assertThat(row.getString(1).toString(), equalTo("scooter")); - assertThat(row.getString(2).toString(), equalTo("Small 2-wheel scooter")); - assertThat(row.getFloat(3), equalTo(3.14f)); - assertThat(row.getString(4).toString(), equalTo("test")); - assertThat(row.getString(5).toString(), equalTo("product")); - assertThat(row.getArray(6).getString(0).toString(), equalTo("id")); - assertThat(row.getTimestamp(7, 3).getMillisecond(), equalTo(1596684883000L)); - }; - consumer.accept(collector.list.get(0)); + assertThat(collector.list).hasSize(1); + assertThat(collector.list.get(0)) + .satisfies( + row -> { + assertThat(row.getInt(0)).isEqualTo(101); + assertThat(row.getString(1).toString()).isEqualTo("scooter"); + assertThat(row.getString(2).toString()) + .isEqualTo("Small 2-wheel scooter"); + assertThat(row.getFloat(3)).isEqualTo(3.14f); + assertThat(row.getString(4).toString()).isEqualTo("test"); + assertThat(row.getString(5).toString()).isEqualTo("product"); + assertThat(row.getArray(6).getString(0).toString()).isEqualTo("id"); + assertThat(row.getTimestamp(7, 3).getMillisecond()) + .isEqualTo(1596684883000L); + }); } @Test - public void testSerializationDeserialization() throws Exception { + void testSerializationDeserialization() throws Exception { List lines = readLines("maxwell-data.txt"); MaxwellJsonDeserializationSchema deserializationSchema = new MaxwellJsonDeserializationSchema( @@ -177,7 +176,7 @@ public void testSerializationDeserialization() throws Exception { "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); MaxwellJsonSerializationSchema serializationSchema = new MaxwellJsonSerializationSchema( @@ -219,7 +218,7 @@ public void testSerializationDeserialization() throws Exception { "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"insert\"}", "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"delete\"}", "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"delete\"}"); - assertEquals(expectedResult, result); + assertThat(result).isEqualTo(expectedResult); } // -------------------------------------------------------------------------------------------- diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java index 07cded6d2..84b293cd9 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java @@ -23,8 +23,8 @@ import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.CollectionUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import java.io.File; import java.io.IOException; @@ -36,9 +36,10 @@ import java.util.stream.Collectors; import static java.lang.String.format; +import static org.assertj.core.api.Assertions.assertThat; /** Test Filesystem connector with OGG Json. */ -public class OggJsonFileSystemITCase extends StreamingTestBase { +class OggJsonFileSystemITCase extends StreamingTestBase { private static final List EXPECTED = Arrays.asList( @@ -73,9 +74,10 @@ private static byte[] readBytes(String resource) throws IOException { return Files.readAllBytes(path); } - private void prepareTables(boolean isPartition) throws IOException { + private void prepareTables(boolean isPartition, Path tempSourceDir, Path tempSinkDir) + throws IOException { byte[] bytes = readBytes("ogg-data.txt"); - source = TEMPORARY_FOLDER.newFolder(); + source = tempSourceDir.toFile(); File file; if (isPartition) { File partition = new File(source, "p=1"); @@ -87,7 +89,7 @@ private void prepareTables(boolean isPartition) throws IOException { file.createNewFile(); Files.write(file.toPath(), bytes); - sink = TEMPORARY_FOLDER.newFolder(); + sink = tempSinkDir.toFile(); env().setParallelism(1); } @@ -107,8 +109,8 @@ private void createTable(boolean isSink, String path, boolean isPartition) { } @Test - public void testNonPartition() throws Exception { - prepareTables(false); + void testNonPartition(@TempDir Path tempSourceDir, @TempDir Path tempSinkDir) throws Exception { + prepareTables(true, tempSourceDir, tempSinkDir); createTable(false, source.toURI().toString(), false); createTable(true, sink.toURI().toString(), false); @@ -124,12 +126,12 @@ public void testNonPartition() throws Exception { .collect(Collectors.toList()); iter.close(); - Assert.assertEquals(EXPECTED, results); + assertThat(results).isEqualTo(EXPECTED); } @Test - public void testPartition() throws Exception { - prepareTables(true); + void testPartition(@TempDir Path tempSourceDir, @TempDir Path tempSinkDir) throws Exception { + prepareTables(true, tempSourceDir, tempSinkDir); createTable(false, source.toURI().toString(), true); createTable(true, sink.toURI().toString(), true); @@ -147,11 +149,11 @@ public void testPartition() throws Exception { .map(Row::toString) .collect(Collectors.toList()); - Assert.assertEquals(EXPECTED, results); + assertThat(results).isEqualTo(EXPECTED); // check partition value for (Row row : list) { - Assert.assertEquals(1, row.getField(4)); + assertThat(row.getField(4)).isEqualTo(1); } } } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java index fd669a1ac..c04e991a2 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java @@ -27,29 +27,26 @@ import org.apache.flink.table.factories.TestDynamicTableFactory; import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.util.TestLogger; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link OggJsonFormatFactory}. */ -public class OggJsonFormatFactoryTest extends TestLogger { - @Rule public ExpectedException thrown = ExpectedException.none(); +class OggJsonFormatFactoryTest { @Test - public void testSeDeSchema() { + void testSeDeSchema() { final Map options = getAllOptions(); final OggJsonSerializationSchema expectedSer = @@ -69,46 +66,48 @@ public void testSeDeSchema() { sinkMock.valueFormat.createRuntimeEncoder( new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); - assertEquals(expectedSer, actualSer); + assertThat(actualSer).isEqualTo(expectedSer); } @Test - public void testInvalidIgnoreParseError() { - thrown.expect( - containsCause( - new IllegalArgumentException( - "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)"))); - + void testInvalidIgnoreParseError() { final Map options = getModifiedOptions(opts -> opts.put("ogg-json.ignore-parse-errors", "abc")); - createTableSource(SCHEMA, options); + assertThatThrownBy(() -> createTableSource(SCHEMA, options)) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Unrecognized option for boolean: abc. " + + "Expected either true or false(case insensitive)")); } @Test - public void testInvalidOptionForTimestampFormat() { + void testInvalidOptionForTimestampFormat() { final Map tableOptions = getModifiedOptions(opts -> opts.put("ogg-json.timestamp-format.standard", "test")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601]."))); - createTableSource(SCHEMA, tableOptions); + assertThatThrownBy(() -> createTableSource(SCHEMA, tableOptions)) + .isInstanceOf(ValidationException.class) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'test' for timestamp-format.standard. " + + "Supported values are [SQL, ISO-8601].")); } @Test - public void testInvalidOptionForMapNullKeyMode() { + void testInvalidOptionForMapNullKeyMode() { final Map tableOptions = getModifiedOptions(opts -> opts.put("ogg-json.map-null-key.mode", "invalid")); - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( - new ValidationException( - "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP]."))); - createTableSink(SCHEMA, tableOptions); + assertThatThrownBy(() -> createTableSink(SCHEMA, tableOptions)) + .isInstanceOf(ValidationException.class) + .satisfies( + anyCauseMatches( + ValidationException.class, + "Unsupported value 'invalid' for option map-null-key.mode. " + + "Supported values are [LITERAL, FAIL, DROP].")); } // ------------------------------------------------------------------------ diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java index 8ad6735fd..9b694609d 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java @@ -29,9 +29,8 @@ import org.apache.flink.table.types.utils.DataTypeUtils; import org.apache.flink.util.Collector; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.assertj.core.data.Percentage; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -43,7 +42,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.function.Consumer; import java.util.stream.Collectors; import static org.apache.flink.table.api.DataTypes.FIELD; @@ -51,11 +49,10 @@ import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link OggJsonSerializationSchema} and {@link OggJsonDeserializationSchema}. */ -public class OggJsonSerDeSchemaTest { +class OggJsonSerDeSchemaTest { private static final DataType PHYSICAL_DATA_TYPE = ROW( @@ -63,7 +60,6 @@ public class OggJsonSerDeSchemaTest { FIELD("name", STRING()), FIELD("description", STRING()), FIELD("weight", FLOAT())); - @Rule public ExpectedException thrown = ExpectedException.none(); private static List readLines(String resource) throws IOException { final URL url = OggJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); @@ -73,17 +69,17 @@ private static List readLines(String resource) throws IOException { } @Test - public void testSerializationAndDeserialization() throws Exception { + void testSerializationAndDeserialization() throws Exception { testSerializationDeserialization("ogg-data.txt"); } @Test - public void testDeserializationWithMetadata() throws Exception { + void testDeserializationWithMetadata() throws Exception { testDeserializationWithMetadata("ogg-data.txt"); } @Test - public void testTombstoneMessages() throws Exception { + void testTombstoneMessages() throws Exception { OggJsonDeserializationSchema deserializationSchema = new OggJsonDeserializationSchema( PHYSICAL_DATA_TYPE, @@ -94,7 +90,7 @@ public void testTombstoneMessages() throws Exception { SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(null, collector); deserializationSchema.deserialize(new byte[] {}, collector); - assertTrue(collector.list.isEmpty()); + assertThat(collector.list).isEmpty(); } public void testDeserializationWithMetadata(String resourceFile) throws Exception { @@ -119,20 +115,24 @@ public void testDeserializationWithMetadata(String resourceFile) throws Exceptio final SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - assertEquals(1, collector.list.size()); - - Consumer consumer = - row -> { - assertEquals(101, row.getInt(0)); - assertEquals("scooter", row.getString(1).toString()); - assertEquals("Small 2-wheel scooter", row.getString(2).toString()); - assertEquals(3.140000104904175, row.getFloat(3), 1e-15); - assertEquals("OGG.TBL_TEST", row.getString(4).toString()); - assertEquals("id", row.getArray(5).getString(0).toString()); - assertEquals(1589377175766L, row.getTimestamp(6, 6).getMillisecond()); - assertEquals(1589384406000L, row.getTimestamp(7, 6).getMillisecond()); - }; - consumer.accept(collector.list.get(0)); + assertThat(collector.list).hasSize(1); + assertThat(collector.list.get(0)) + .satisfies( + row -> { + assertThat(row.getInt(0)).isEqualTo(101); + assertThat(row.getString(1).toString()).isEqualTo("scooter"); + assertThat(row.getString(2).toString()) + .isEqualTo("Small 2-wheel scooter"); + assertThat(row.getFloat(3)) + .isCloseTo( + 3.140000104904175f, Percentage.withPercentage(1e-15)); + assertThat(row.getString(4).toString()).isEqualTo("OGG.TBL_TEST"); + assertThat(row.getArray(5).getString(0).toString()).isEqualTo("id"); + assertThat(row.getTimestamp(6, 6).getMillisecond()) + .isEqualTo(1589377175766L); + assertThat(row.getTimestamp(7, 6).getMillisecond()) + .isEqualTo(1589384406000L); + }); } private void testSerializationDeserialization(String resourceFile) throws Exception { @@ -204,7 +204,7 @@ private void testSerializationDeserialization(String resourceFile) throws Except "-D(111,scooter,Big 2-wheel scooter ,5.17)"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); - assertEquals(expected, actual); + assertThat(expected).containsExactlyElementsOf(actual); OggJsonSerializationSchema serializationSchema = new OggJsonSerializationSchema( @@ -242,7 +242,7 @@ private void testSerializationDeserialization(String resourceFile) throws Except "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":null,\"op_type\":\"D\"}", "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op_type\":\"I\"}", "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op_type\":\"D\"}"); - assertEquals(expected, actual); + assertThat(expected).containsExactlyElementsOf(actual); } private static class SimpleCollector implements Collector { diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/services/org.junit.jupiter.api.extension.Extension b/flink-formats-kafka/flink-json-debezium/src/test/resources/services/org.junit.jupiter.api.extension.Extension new file mode 100644 index 000000000..28999133c --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/resources/services/org.junit.jupiter.api.extension.Extension @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.util.TestLoggerExtension \ No newline at end of file From 0ff1cd0adc11163fa13fb61102896f018db18744 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 3 May 2022 10:12:32 +0200 Subject: [PATCH 186/322] [hotfix][json][tests] Fix location of services directory --- .../services/org.junit.jupiter.api.extension.Extension | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename flink-formats-kafka/flink-json-debezium/src/test/resources/{ => META-INF}/services/org.junit.jupiter.api.extension.Extension (100%) diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/services/org.junit.jupiter.api.extension.Extension b/flink-formats-kafka/flink-json-debezium/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension similarity index 100% rename from flink-formats-kafka/flink-json-debezium/src/test/resources/services/org.junit.jupiter.api.extension.Extension rename to flink-formats-kafka/flink-json-debezium/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension From dd6c27541a3b7ab68105c49f551382abeb150280 Mon Sep 17 00:00:00 2001 From: MartijnVisser Date: Wed, 4 May 2022 15:24:51 +0200 Subject: [PATCH 187/322] [FLINK-27442][Formats][Avro Confluent] Add Confluent repo to module flink-sql-avro-confluent-registry --- .../flink-sql-avro-confluent-registry/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index 2ce0a31d0..123311900 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -33,6 +33,13 @@ under the License. jar + + + confluent + https://packages.confluent.io/maven/ + + + org.apache.flink From 43854abcb4bd650d181c9185f5a6ae24ffa89c5f Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Fri, 17 Jun 2022 18:56:14 +0200 Subject: [PATCH 188/322] [FLINK-28060][Connector/Kafka] Updated Kafka Clients to 3.1.1 to resolve the issue where Flink is unable to commit its offset back to Kafka in case of Kafka Broker becoming unavailable. This should be resolved when the broker comes back up, but due to KAFKA-13563 that doesn't work. Since that fix has only become available with Kafka Clients 3.1.1, this commit updates the Kafka Clients dependency from 2.8.4 to 3.1.1. No interfaces needed to be adjusted. It was necessary to change some of our Bash e2e tests since they still relied on the Zookeeper parameter which has been removed in this version. The other necessary change was adjusting the KafkaConsumerTestBase class since the level of exception is changed in the new Kafka Clients so exception.getCause().getMessage() throws an NPE in the test case. --- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index cd29ae1c8..7d160a51f 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -32,7 +32,7 @@ under the License. Flink : Formats : Avro confluent registry - 2.8.1 + 3.1.1 6.2.2 From 95f315eab7805b2390ae10c1c35445a7cc1de2ea Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 23 Jun 2022 14:47:18 +0200 Subject: [PATCH 189/322] [FLINK-28222][json] Add flink-sql-json module --- .../flink-json-debezium/pom.xml | 37 ------------------- 1 file changed, 37 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index ebef86486..28c7b4629 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -157,41 +157,4 @@ under the License. - - - - sql-jars - - - !skipSqlJars - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - package - - shade - - - - - org.apache.flink:flink-format-common - - - true - sql-jar - - - - - - - - - From bbc81ba063c2dc5019d2473c2edda3c8179e506f Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 25 Jul 2022 15:50:07 +0200 Subject: [PATCH 190/322] [FLINK-28634][json] Add simple JsonSerDeSchema --- .../flink-json-debezium/pom.xml | 6 + .../json/JsonDeserializationSchema.java | 70 +++++++++++ .../json/JsonNodeDeserializationSchema.java | 17 +-- .../formats/json/JsonSerializationSchema.java | 59 ++++++++++ .../JsonNodeDeserializationSchemaTest.java | 3 + .../formats/json/JsonSerDeSchemaTest.java | 111 ++++++++++++++++++ 6 files changed, 253 insertions(+), 13 deletions(-) create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java create mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 28c7b4629..ab697cf7e 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -109,6 +109,12 @@ under the License. + + org.apache.flink + flink-connector-test-utils + ${project.version} + test + org.apache.flink flink-test-utils diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java new file mode 100644 index 000000000..fd28712d5 --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.util.function.SerializableSupplier; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; + +/** DeserializationSchema that deserializes a JSON String. */ +@PublicEvolving +public class JsonDeserializationSchema extends AbstractDeserializationSchema { + + private static final long serialVersionUID = 1L; + + private final Class clazz; + private final SerializableSupplier mapperFactory; + protected transient ObjectMapper mapper; + + public JsonDeserializationSchema(Class clazz) { + this(clazz, () -> new ObjectMapper()); + } + + public JsonDeserializationSchema(TypeInformation typeInformation) { + this(typeInformation, () -> new ObjectMapper()); + } + + public JsonDeserializationSchema( + Class clazz, SerializableSupplier mapperFactory) { + super(clazz); + this.clazz = clazz; + this.mapperFactory = mapperFactory; + } + + public JsonDeserializationSchema( + TypeInformation typeInformation, SerializableSupplier mapperFactory) { + super(typeInformation); + this.clazz = typeInformation.getTypeClass(); + this.mapperFactory = mapperFactory; + } + + @Override + public void open(InitializationContext context) { + mapper = mapperFactory.get(); + } + + @Override + public T deserialize(byte[] message) throws IOException { + return mapper.readValue(message, clazz); + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java index 55c61e1a6..36aa4843f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java @@ -17,28 +17,19 @@ package org.apache.flink.formats.json; -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; -import java.io.IOException; - /** * DeserializationSchema that deserializes a JSON String into an ObjectNode. * *

Fields can be accessed by calling objectNode.get(<name>).as(<type>) */ @PublicEvolving -public class JsonNodeDeserializationSchema extends AbstractDeserializationSchema { - - private static final long serialVersionUID = -1699854177598621044L; +public class JsonNodeDeserializationSchema extends JsonDeserializationSchema { - private final ObjectMapper mapper = new ObjectMapper(); + private static final long serialVersionUID = 2L; - @Override - public ObjectNode deserialize(byte[] message) throws IOException { - return mapper.readValue(message, ObjectNode.class); + public JsonNodeDeserializationSchema() { + super(ObjectNode.class); } } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java new file mode 100644 index 000000000..c029fa1dc --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.util.function.SerializableSupplier; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +/** SerializationSchema that serializes an object to a JSON String. */ +@PublicEvolving +public class JsonSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = 1L; + + private final SerializableSupplier mapperFactory; + + protected transient ObjectMapper mapper; + + public JsonSerializationSchema() { + this(() -> new ObjectMapper()); + } + + public JsonSerializationSchema(SerializableSupplier mapperFactory) { + this.mapperFactory = mapperFactory; + } + + @Override + public void open(InitializationContext context) { + mapper = mapperFactory.get(); + } + + @Override + public byte[] serialize(T element) { + try { + return mapper.writeValueAsBytes(element); + } catch (JsonProcessingException e) { + throw new RuntimeException( + String.format("Could not serialize value '%s'.", element), e); + } + } +} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java index 741b492de..2bd8dc5d6 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java @@ -17,6 +17,8 @@ package org.apache.flink.formats.json; +import org.apache.flink.connector.testutils.formats.DummyInitializationContext; + import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; @@ -37,6 +39,7 @@ void testDeserialize() throws IOException { byte[] serializedValue = mapper.writeValueAsBytes(initialValue); JsonNodeDeserializationSchema schema = new JsonNodeDeserializationSchema(); + schema.open(new DummyInitializationContext()); ObjectNode deserializedValue = schema.deserialize(serializedValue); assertThat(deserializedValue.get("key").asInt()).isEqualTo(4); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java new file mode 100644 index 000000000..5ed992c1e --- /dev/null +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.connector.testutils.formats.DummyInitializationContext; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Objects; + +import static org.assertj.core.api.Assertions.assertThat; + +class JsonSerDeSchemaTest { + private static final JsonSerializationSchema SERIALIZATION_SCHEMA; + private static final JsonDeserializationSchema DESERIALIZATION_SCHEMA; + private static final String JSON = "{\"x\":34,\"y\":\"hello\"}"; + + static { + SERIALIZATION_SCHEMA = new JsonSerializationSchema<>(); + SERIALIZATION_SCHEMA.open(new DummyInitializationContext()); + DESERIALIZATION_SCHEMA = new JsonDeserializationSchema<>(Event.class); + DESERIALIZATION_SCHEMA.open(new DummyInitializationContext()); + } + + @Test + void testSrialization() throws IOException { + final byte[] serialized = SERIALIZATION_SCHEMA.serialize(new Event(34, "hello")); + assertThat(serialized).isEqualTo(JSON.getBytes(StandardCharsets.UTF_8)); + } + + @Test + void testDeserialization() throws IOException { + final Event deserialized = + DESERIALIZATION_SCHEMA.deserialize(JSON.getBytes(StandardCharsets.UTF_8)); + assertThat(deserialized).isEqualTo(new Event(34, "hello")); + } + + @Test + void testRoundTrip() throws IOException { + final Event original = new Event(34, "hello"); + + final byte[] serialized = SERIALIZATION_SCHEMA.serialize(original); + + final Event deserialized = DESERIALIZATION_SCHEMA.deserialize(serialized); + + assertThat(deserialized).isEqualTo(original); + } + + private static class Event { + + private int x; + private String y = null; + + public Event() {} + + public Event(int x, String y) { + this.x = x; + this.y = y; + } + + public int getX() { + return x; + } + + public void setX(int x) { + this.x = x; + } + + public String getY() { + return y; + } + + public void setY(String y) { + this.y = y; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Event event = (Event) o; + return x == event.x && Objects.equals(y, event.y); + } + + @Override + public int hashCode() { + return Objects.hash(x, y); + } + } +} From 4a141f071ea3d7a6ea230c280dde7d0fe4e9171c Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 25 Jul 2022 15:50:44 +0200 Subject: [PATCH 191/322] [FLINK-28634][json] Deprecate JsonNodeDeserializationSchema Subsumed by more general 'JsonDeserializationSchema'. --- .../flink/formats/json/JsonNodeDeserializationSchema.java | 4 +++- .../flink/formats/json/JsonNodeDeserializationSchemaTest.java | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java index 36aa4843f..928a6f1e5 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java @@ -23,8 +23,10 @@ * DeserializationSchema that deserializes a JSON String into an ObjectNode. * *

Fields can be accessed by calling objectNode.get(<name>).as(<type>) + * + * @deprecated Use {@code new JsonDeserializationSchema(ObjectNode.class)} instead */ -@PublicEvolving +@Deprecated public class JsonNodeDeserializationSchema extends JsonDeserializationSchema { private static final long serialVersionUID = 2L; diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java index 2bd8dc5d6..90751525f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java @@ -29,6 +29,7 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link JsonNodeDeserializationSchema}. */ +@SuppressWarnings("deprecation") class JsonNodeDeserializationSchemaTest { @Test From a31ac02e0c1b6c17e20b8d421acd0b92687f3e3e Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 26 Jul 2022 16:24:43 +0200 Subject: [PATCH 192/322] [FLINK-28807] Honor schema lifecycle --- .../canal/CanalJsonDeserializationSchema.java | 5 ++++ .../canal/CanalJsonSerializationSchema.java | 3 ++- .../DebeziumJsonDeserializationSchema.java | 5 ++++ .../DebeziumJsonSerializationSchema.java | 3 ++- .../MaxwellJsonDeserializationSchema.java | 5 ++++ .../MaxwellJsonSerializationSchema.java | 1 + .../ogg/OggJsonDeserializationSchema.java | 5 ++++ .../json/ogg/OggJsonSerializationSchema.java | 3 ++- .../json/JsonRowDataSerDeSchemaTest.java | 27 +++++++++++++++++++ .../json/JsonRowSerializationSchemaTest.java | 6 +++++ .../json/canal/CanalJsonSerDeSchemaTest.java | 7 ++++- .../debezium/DebeziumJsonSerDeSchemaTest.java | 5 +++- .../json/maxwell/MaxwellJsonSerDerTest.java | 5 +++- .../json/ogg/OggJsonSerDeSchemaTest.java | 6 ++++- .../utils/DeserializationSchemaMatcher.java | 2 ++ .../utils/SerializationSchemaMatcher.java | 3 +++ 16 files changed, 84 insertions(+), 7 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java index d1bd3a37f..bb9777833 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java @@ -198,6 +198,11 @@ public CanalJsonDeserializationSchema build() { // ------------------------------------------------------------------------------------------ + @Override + public void open(InitializationContext context) throws Exception { + jsonDeserializer.open(context); + } + @Override public RowData deserialize(byte[] message) throws IOException { throw new RuntimeException( diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java index 90302d174..362b9df6e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java @@ -70,7 +70,8 @@ public CanalJsonSerializationSchema( } @Override - public void open(InitializationContext context) { + public void open(InitializationContext context) throws Exception { + jsonSerializer.open(context); reuse = new GenericRowData(2); } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java index d12b7cc34..cba336d4f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java @@ -115,6 +115,11 @@ public DebeziumJsonDeserializationSchema( this.ignoreParseErrors = ignoreParseErrors; } + @Override + public void open(InitializationContext context) throws Exception { + jsonDeserializer.open(context); + } + @Override public RowData deserialize(byte[] message) { throw new RuntimeException( diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java index f6e37073d..0dc9a96b0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java @@ -67,7 +67,8 @@ public DebeziumJsonSerializationSchema( } @Override - public void open(InitializationContext context) { + public void open(InitializationContext context) throws Exception { + jsonSerializer.open(context); genericRowData = new GenericRowData(3); } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java index 863620fd9..f58bfd4a1 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java @@ -109,6 +109,11 @@ public MaxwellJsonDeserializationSchema( this.fieldCount = physicalRowType.getFieldCount(); } + @Override + public void open(InitializationContext context) throws Exception { + jsonDeserializer.open(context); + } + @Override public RowData deserialize(byte[] message) throws IOException { throw new RuntimeException( diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java index 8dfac669c..1fe567b08 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java @@ -69,6 +69,7 @@ public MaxwellJsonSerializationSchema( @Override public void open(InitializationContext context) throws Exception { + jsonSerializer.open(context); this.reuse = new GenericRowData(2); } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java index 4337bedda..85bd52dc0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java @@ -143,6 +143,11 @@ private static int findFieldPos(ReadableMetadata metadata, RowType jsonRowType) return jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); } + @Override + public void open(InitializationContext context) throws Exception { + jsonDeserializer.open(context); + } + @Override public RowData deserialize(byte[] message) { throw new RuntimeException( diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java index 2189f86c3..635ff3dc7 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java @@ -79,7 +79,8 @@ private static RowType createJsonRowType(DataType databaseSchema) { } @Override - public void open(InitializationContext context) { + public void open(InitializationContext context) throws Exception { + jsonSerializer.open(context); genericRowData = new GenericRowData(3); } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index fb30702bf..888be3b9f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.json; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.testutils.formats.DummyInitializationContext; import org.apache.flink.core.testutils.FlinkAssertions; import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.data.GenericMapData; @@ -52,6 +53,7 @@ import java.util.Random; import java.util.concurrent.ThreadLocalRandom; +import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.api.DataTypes.ARRAY; import static org.apache.flink.table.api.DataTypes.BIGINT; @@ -165,6 +167,7 @@ void testSerDe() throws Exception { JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); + open(deserializationSchema); Row expected = new Row(18); expected.setField(0, true); @@ -198,6 +201,7 @@ void testSerDe() throws Exception { JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); + open(serializationSchema); byte[] actualBytes = serializationSchema.serialize(rowData); assertThat(serializedJson).containsExactly(actualBytes); @@ -246,6 +250,7 @@ void testSlowDeserialization() throws Exception { false, false, TimestampFormat.ISO_8601); + open(deserializationSchema); Row expected = new Row(7); expected.setField(0, bool); @@ -281,6 +286,7 @@ void testSerDeMultiRows() throws Exception { false, false, TimestampFormat.ISO_8601); + open(deserializationSchema); JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema( rowType, @@ -288,6 +294,7 @@ void testSerDeMultiRows() throws Exception { JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); + open(serializationSchema); ObjectMapper objectMapper = new ObjectMapper(); @@ -366,6 +373,7 @@ void testSerDeMultiRowsWithNullValues() throws Exception { false, true, TimestampFormat.ISO_8601); + open(deserializationSchema); JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema( rowType, @@ -373,6 +381,7 @@ void testSerDeMultiRowsWithNullValues() throws Exception { JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); + open(serializationSchema); for (int i = 0; i < jsons.length; i++) { String json = jsons[i]; @@ -390,6 +399,7 @@ void testDeserializationNullRow() throws Exception { JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); + open(deserializationSchema); assertThat(deserializationSchema.deserialize(null)).isNull(); } @@ -402,6 +412,7 @@ void testDeserializationMissingNode() throws Exception { JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); + open(deserializationSchema); RowData rowData = deserializationSchema.deserialize("".getBytes()); assertThat(rowData).isNull(); } @@ -426,6 +437,7 @@ void testDeserializationMissingField() throws Exception { false, false, TimestampFormat.ISO_8601); + open(deserializationSchema); Row expected = new Row(1); Row actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); @@ -435,6 +447,7 @@ void testDeserializationMissingField() throws Exception { deserializationSchema = new JsonRowDataDeserializationSchema( schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); + open(deserializationSchema); String errorMessage = "Failed to deserialize JSON '{\"id\":123123123}'."; @@ -446,6 +459,7 @@ void testDeserializationMissingField() throws Exception { deserializationSchema = new JsonRowDataDeserializationSchema( schema, InternalTypeInfo.of(schema), false, true, TimestampFormat.ISO_8601); + open(deserializationSchema); actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); assertThat(actual).isEqualTo(expected); @@ -480,6 +494,7 @@ void testSerDeSQLTimestampFormat() throws Exception { JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); + open(deserializationSchema); JsonRowDataSerializationSchema serializationSchema = new JsonRowDataSerializationSchema( rowType, @@ -487,6 +502,7 @@ void testSerDeSQLTimestampFormat() throws Exception { JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); + open(serializationSchema); ObjectMapper objectMapper = new ObjectMapper(); @@ -530,6 +546,7 @@ void testSerializationMapNullKey() { JsonFormatOptions.MapNullKeyMode.FAIL, "null", true); + open(serializationSchema1); // expect message for serializationSchema1 String errorMessage1 = "JSON format doesn't support to serialize map data with null keys." @@ -542,6 +559,7 @@ void testSerializationMapNullKey() { JsonFormatOptions.MapNullKeyMode.DROP, "null", true); + open(serializationSchema2); // expect result for serializationSchema2 String expectResult2 = "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1}}}"; @@ -552,6 +570,7 @@ void testSerializationMapNullKey() { JsonFormatOptions.MapNullKeyMode.LITERAL, "nullKey", true); + open(serializationSchema3); // expect result for serializationSchema3 String expectResult3 = "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1,\"nullKey\":2},\"nullKey\":{\"no-null key\":1,\"nullKey\":2}}}"; @@ -583,6 +602,7 @@ void testSerializationDecimalEncode() throws Exception { JsonRowDataDeserializationSchema deserializer = new JsonRowDataDeserializationSchema( schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); + deserializer.open(new DummyInitializationContext()); JsonRowDataSerializationSchema plainDecimalSerializer = new JsonRowDataSerializationSchema( @@ -591,6 +611,7 @@ void testSerializationDecimalEncode() throws Exception { JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); + plainDecimalSerializer.open(new DummyInitializationContext()); JsonRowDataSerializationSchema scientificDecimalSerializer = new JsonRowDataSerializationSchema( schema, @@ -598,6 +619,7 @@ void testSerializationDecimalEncode() throws Exception { JsonFormatOptions.MapNullKeyMode.LITERAL, "null", false); + scientificDecimalSerializer.open(new DummyInitializationContext()); String plainDecimalJson = "{\"decimal1\":123.456789,\"decimal2\":454621864049246170,\"decimal3\":0.000000027}"; @@ -636,6 +658,7 @@ void testSerializationWithTypesMismatch() { JsonFormatOptions.MapNullKeyMode.FAIL, "null", true); + open(serializationSchema); String errorMessage = "Fail to serialize at field: f1."; assertThatThrownBy(() -> serializationSchema.serialize(genericRowData)) @@ -649,6 +672,7 @@ void testDeserializationWithTypesMismatch() { JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); + open(deserializationSchema); String errorMessage = "Fail to deserialize at field: f1."; assertThatThrownBy(() -> deserializationSchema.deserialize(json.getBytes())) @@ -664,6 +688,8 @@ private void testIgnoreParseErrors(TestSpec spec) throws Exception { false, true, spec.timestampFormat); + ignoreErrorsSchema.open(new DummyInitializationContext()); + Row expected; if (spec.expected != null) { expected = spec.expected; @@ -686,6 +712,7 @@ private void testParseErrors(TestSpec spec) { false, false, spec.timestampFormat); + open(failingSchema); assertThatThrownBy(() -> failingSchema.deserialize(spec.json.getBytes())) .hasMessageContaining(spec.errorMessage); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java index e9c46be1e..ac1fe1818 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java @@ -30,6 +30,7 @@ import java.sql.Timestamp; import java.util.concurrent.ThreadLocalRandom; +import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; import static org.apache.flink.formats.utils.SerializationSchemaMatcher.whenSerializedWith; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; @@ -81,8 +82,10 @@ public void testSerializationOfTwoRows() throws IOException { final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema).build(); + open(serializationSchema); final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(rowSchema).build(); + open(deserializationSchema); byte[] bytes = serializationSchema.serialize(row1); assertEquals(row1, deserializationSchema.deserialize(bytes)); @@ -122,8 +125,10 @@ public void testMultiRowsWithNullValues() throws IOException { Types.PRIMITIVE_ARRAY(Types.INT)); JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(schema).build(); + open(deserializationSchema); JsonRowSerializationSchema serializationSchema = JsonRowSerializationSchema.builder().withTypeInfo(schema).build(); + open(serializationSchema); for (int i = 0; i < jsons.length; i++) { String json = jsons[i]; @@ -173,6 +178,7 @@ public void testSerializeRowWithInvalidNumberOfFields() { final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema.Builder(rowSchema).build(); + open(serializationSchema); assertThat( row, whenSerializedWith(serializationSchema) diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java index 1083ae656..e45bfcc5e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json.canal; +import org.apache.flink.connector.testutils.formats.DummyInitializationContext; import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonFormatOptions; import org.apache.flink.formats.json.canal.CanalJsonDecodingFormat.ReadableMetadata; @@ -44,6 +45,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; +import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; import static org.apache.flink.table.api.DataTypes.FIELD; import static org.apache.flink.table.api.DataTypes.FLOAT; import static org.apache.flink.table.api.DataTypes.INT; @@ -80,6 +82,7 @@ void testDeserializeNullRow() throws Exception { final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); final CanalJsonDeserializationSchema deserializationSchema = createCanalJsonDeserializationSchema(null, null, requestedMetadata); + open(deserializationSchema); final SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(null, collector); @@ -139,6 +142,7 @@ void testSerializationDeserialization() throws Exception { public void runTest(List lines, CanalJsonDeserializationSchema deserializationSchema) throws Exception { + open(deserializationSchema); SimpleCollector collector = new SimpleCollector(); for (String line : lines) { deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); @@ -215,7 +219,7 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); - serializationSchema.open(null); + serializationSchema.open(new DummyInitializationContext()); List result = new ArrayList<>(); for (RowData rowData : collector.list) { @@ -262,6 +266,7 @@ private void testDeserializationWithMetadata( final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); final CanalJsonDeserializationSchema deserializationSchema = createCanalJsonDeserializationSchema(database, table, requestedMetadata); + open(deserializationSchema); final SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java index ef3b1d8ab..3b9151f33 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -44,6 +44,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; +import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; import static org.apache.flink.table.api.DataTypes.FIELD; import static org.apache.flink.table.api.DataTypes.FLOAT; import static org.apache.flink.table.api.DataTypes.INT; @@ -179,6 +180,7 @@ private void testSerializationDeserialization(String resourceFile, boolean schem schemaInclude, false, TimestampFormat.ISO_8601); + open(deserializationSchema); SimpleCollector collector = new SimpleCollector(); for (String line : lines) { @@ -249,7 +251,7 @@ private void testSerializationDeserialization(String resourceFile, boolean schem "null", true); - serializationSchema.open(null); + open(serializationSchema); actual = new ArrayList<>(); for (RowData rowData : collector.list) { actual.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); @@ -303,6 +305,7 @@ private void testDeserializationWithMetadata( schemaInclude, false, TimestampFormat.ISO_8601); + open(deserializationSchema); final SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java index ffd68b20b..12d64fd99 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java @@ -43,6 +43,7 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; import static org.apache.flink.table.api.DataTypes.FIELD; import static org.apache.flink.table.api.DataTypes.FLOAT; import static org.apache.flink.table.api.DataTypes.INT; @@ -80,6 +81,7 @@ void testDeserializationWithMetadata() throws Exception { InternalTypeInfo.of(producedDataType.getLogicalType()), false, TimestampFormat.ISO_8601); + open(deserializationSchema); final SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); assertThat(collector.list).hasSize(1); @@ -109,6 +111,7 @@ void testSerializationDeserialization() throws Exception { InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), false, TimestampFormat.ISO_8601); + open(deserializationSchema); SimpleCollector collector = new SimpleCollector(); for (String line : lines) { @@ -185,7 +188,7 @@ void testSerializationDeserialization() throws Exception { JsonFormatOptions.MapNullKeyMode.LITERAL, "null", true); - serializationSchema.open(null); + open(serializationSchema); List result = new ArrayList<>(); for (RowData rowData : collector.list) { result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java index 9b694609d..2fa78c894 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java @@ -44,6 +44,7 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; import static org.apache.flink.table.api.DataTypes.FIELD; import static org.apache.flink.table.api.DataTypes.FLOAT; import static org.apache.flink.table.api.DataTypes.INT; @@ -87,6 +88,7 @@ void testTombstoneMessages() throws Exception { InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), false, TimestampFormat.ISO_8601); + open(deserializationSchema); SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(null, collector); deserializationSchema.deserialize(new byte[] {}, collector); @@ -112,6 +114,7 @@ public void testDeserializationWithMetadata(String resourceFile) throws Exceptio InternalTypeInfo.of(producedDataTypes.getLogicalType()), false, TimestampFormat.ISO_8601); + open(deserializationSchema); final SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); @@ -144,6 +147,7 @@ private void testSerializationDeserialization(String resourceFile) throws Except InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), false, TimestampFormat.ISO_8601); + open(deserializationSchema); SimpleCollector collector = new SimpleCollector(); for (String line : lines) { @@ -214,7 +218,7 @@ private void testSerializationDeserialization(String resourceFile) throws Except "null", true); - serializationSchema.open(null); + open(serializationSchema); actual = new ArrayList<>(); for (RowData rowData : collector.list) { actual.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java index 59ee17820..c6d9e6229 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.Objects; +import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; import static org.apache.flink.util.InstantiationUtil.deserializeObject; import static org.apache.flink.util.InstantiationUtil.serializeObject; @@ -141,6 +142,7 @@ private DeserializationSchemaMatcherBuilder( deserializeObject( serializeObject(deserializationSchema), this.getClass().getClassLoader()); + open(this.deserializationSchema); } catch (IOException | ClassNotFoundException e) { throw new RuntimeException(e); } diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java index be0951551..1cf6f6d20 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.util.Objects; +import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; import static org.apache.flink.util.InstantiationUtil.deserializeObject; import static org.apache.flink.util.InstantiationUtil.serializeObject; @@ -150,10 +151,12 @@ private SerializationWithDeserializationSchemaMatcherBuilder( deserializeObject( serializeObject(serializationSchema), this.getClass().getClassLoader()); + open(this.serializationSchema); this.deserializationSchema = deserializeObject( serializeObject(deserializationSchema), this.getClass().getClassLoader()); + open(this.deserializationSchema); } catch (IOException | ClassNotFoundException e) { throw new RuntimeException(e); } From a6b822a6c45382ba5edbd5615427baf0dce09028 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 28 Jul 2022 16:09:43 +0200 Subject: [PATCH 193/322] [FLINK-28621][formats] Initialize mappers in open() --- .../json/JsonRowDataDeserializationSchema.java | 17 +++++++++++++---- .../json/JsonRowDataSerializationSchema.java | 12 +++++++++--- .../json/JsonRowDeserializationSchema.java | 12 +++++++++--- .../json/JsonRowSerializationSchema.java | 7 ++++++- 4 files changed, 37 insertions(+), 11 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index 83d1b2dc3..805b299c1 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -67,11 +67,13 @@ public class JsonRowDataDeserializationSchema implements DeserializationSchema resultTypeInfo, @@ -89,12 +91,19 @@ public JsonRowDataDeserializationSchema( new JsonToRowDataConverters(failOnMissingField, ignoreParseErrors, timestampFormat) .createConverter(checkNotNull(rowType)); this.timestampFormat = timestampFormat; - boolean hasDecimalType = - LogicalTypeChecks.hasNested(rowType, t -> t instanceof DecimalType); + this.hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t instanceof DecimalType); + } + + @Override + public void open(InitializationContext context) throws Exception { + objectMapper = + new ObjectMapper() + .configure( + JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(), + true); if (hasDecimalType) { objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); } - objectMapper.configure(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(), true); } @Override diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index 1b77aab1c..6a8c619ee 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -50,7 +50,7 @@ public class JsonRowDataSerializationSchema implements SerializationSchema private boolean failOnMissingField; + private final boolean hasDecimalType; + /** Object mapper for parsing the JSON. */ - private final ObjectMapper objectMapper = new ObjectMapper(); + private transient ObjectMapper objectMapper; private DeserializationRuntimeConverter runtimeConverter; @@ -114,8 +116,12 @@ private JsonRowDeserializationSchema( this.runtimeConverter = createConverter(this.typeInfo); this.ignoreParseErrors = ignoreParseErrors; RowType rowType = (RowType) fromLegacyInfoToDataType(this.typeInfo).getLogicalType(); - boolean hasDecimalType = - LogicalTypeChecks.hasNested(rowType, t -> t.getTypeRoot().equals(DECIMAL)); + hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t.getTypeRoot().equals(DECIMAL)); + } + + @Override + public void open(InitializationContext context) throws Exception { + objectMapper = new ObjectMapper(); if (hasDecimalType) { objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index 3dd8be1fd..e78930796 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -79,7 +79,7 @@ public class JsonRowSerializationSchema implements SerializationSchema { private final RowTypeInfo typeInfo; /** Object mapper that is used to create output JSON objects. */ - private final ObjectMapper mapper = new ObjectMapper(); + private transient ObjectMapper mapper; private final SerializationRuntimeConverter runtimeConverter; @@ -94,6 +94,11 @@ private JsonRowSerializationSchema(TypeInformation typeInfo) { this.runtimeConverter = createConverter(typeInfo); } + @Override + public void open(InitializationContext context) throws Exception { + mapper = new ObjectMapper(); + } + /** Builder for {@link JsonRowSerializationSchema}. */ @PublicEvolving public static class Builder { From f34906d5f72247b22c85575f4e7d60772e63bcbe Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 28 Jul 2022 16:02:26 +0200 Subject: [PATCH 194/322] [FLINK-28621][core] Add central Jackson mapper factory methods --- .../json/JsonDeserializationSchema.java | 5 ++- .../JsonRowDataDeserializationSchema.java | 3 +- .../json/JsonRowDataSerializationSchema.java | 3 +- .../json/JsonRowDeserializationSchema.java | 3 +- .../formats/json/JsonRowSchemaConverter.java | 3 +- .../json/JsonRowSerializationSchema.java | 3 +- .../JsonNodeDeserializationSchemaTest.java | 3 +- .../json/JsonRowDataSerDeSchemaTest.java | 37 ++++++++----------- .../JsonRowDeserializationSchemaTest.java | 21 +++++------ 9 files changed, 40 insertions(+), 41 deletions(-) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java index fd28712d5..cc244b0a0 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.util.function.SerializableSupplier; +import org.apache.flink.util.jackson.JacksonMapperFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -37,11 +38,11 @@ public class JsonDeserializationSchema extends AbstractDeserializationSchema< protected transient ObjectMapper mapper; public JsonDeserializationSchema(Class clazz) { - this(clazz, () -> new ObjectMapper()); + this(clazz, JacksonMapperFactory::createObjectMapper); } public JsonDeserializationSchema(TypeInformation typeInformation) { - this(typeInformation, () -> new ObjectMapper()); + this(typeInformation, JacksonMapperFactory::createObjectMapper); } public JsonDeserializationSchema( diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java index 805b299c1..9a57bac20 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java @@ -26,6 +26,7 @@ import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.util.jackson.JacksonMapperFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.json.JsonReadFeature; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; @@ -97,7 +98,7 @@ public JsonRowDataDeserializationSchema( @Override public void open(InitializationContext context) throws Exception { objectMapper = - new ObjectMapper() + JacksonMapperFactory.createObjectMapper() .configure( JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(), true); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java index 6a8c619ee..c8b7f73b6 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java @@ -23,6 +23,7 @@ import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.jackson.JacksonMapperFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -86,7 +87,7 @@ public JsonRowDataSerializationSchema( @Override public void open(InitializationContext context) throws Exception { mapper = - new ObjectMapper() + JacksonMapperFactory.createObjectMapper() .configure( JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, encodeDecimalAsPlainNumber); diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index b2b7e6dad..dd4a9bb9f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -30,6 +30,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; import org.apache.flink.types.Row; +import org.apache.flink.util.jackson.JacksonMapperFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; @@ -121,7 +122,7 @@ private JsonRowDeserializationSchema( @Override public void open(InitializationContext context) throws Exception { - objectMapper = new ObjectMapper(); + objectMapper = JacksonMapperFactory.createObjectMapper(); if (hasDecimalType) { objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); } diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java index fe4123204..d7761851e 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.jackson.JacksonMapperFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; @@ -97,7 +98,7 @@ private JsonRowSchemaConverter() { @SuppressWarnings("unchecked") public static TypeInformation convert(String jsonSchema) { Preconditions.checkNotNull(jsonSchema, "JSON schema"); - final ObjectMapper mapper = new ObjectMapper(); + final ObjectMapper mapper = JacksonMapperFactory.createObjectMapper(); mapper.getFactory() .enable(JsonParser.Feature.ALLOW_COMMENTS) .enable(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES) diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index e78930796..f185d211b 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -28,6 +28,7 @@ import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import org.apache.flink.util.WrappingRuntimeException; +import org.apache.flink.util.jackson.JacksonMapperFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -96,7 +97,7 @@ private JsonRowSerializationSchema(TypeInformation typeInfo) { @Override public void open(InitializationContext context) throws Exception { - mapper = new ObjectMapper(); + mapper = JacksonMapperFactory.createObjectMapper(); } /** Builder for {@link JsonRowSerializationSchema}. */ diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java index 90751525f..e6b2a3e05 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java @@ -18,6 +18,7 @@ package org.apache.flink.formats.json; import org.apache.flink.connector.testutils.formats.DummyInitializationContext; +import org.apache.flink.util.jackson.JacksonMapperFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; @@ -34,7 +35,7 @@ class JsonNodeDeserializationSchemaTest { @Test void testDeserialize() throws IOException { - ObjectMapper mapper = new ObjectMapper(); + ObjectMapper mapper = JacksonMapperFactory.createObjectMapper(); ObjectNode initialValue = mapper.createObjectNode(); initialValue.put("key", 4).put("value", "world"); byte[] serializedValue = mapper.writeValueAsBytes(initialValue); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 888be3b9f..883c3f091 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -31,6 +31,7 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; +import org.apache.flink.util.jackson.JacksonMapperFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; @@ -83,6 +84,8 @@ */ class JsonRowDataSerDeSchemaTest { + private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper(); + @Test void testSerDe() throws Exception { byte tinyint = 'c'; @@ -115,11 +118,10 @@ void testSerDe() throws Exception { innerMap.put("key", 234); nestedMap.put("inner_map", innerMap); - ObjectMapper objectMapper = new ObjectMapper(); - ArrayNode doubleNode = objectMapper.createArrayNode().add(1.1D).add(2.2D).add(3.3D); + ArrayNode doubleNode = OBJECT_MAPPER.createArrayNode().add(1.1D).add(2.2D).add(3.3D); // Root - ObjectNode root = objectMapper.createObjectNode(); + ObjectNode root = OBJECT_MAPPER.createObjectNode(); root.put("bool", true); root.put("tinyint", tinyint); root.put("smallint", smallint); @@ -139,7 +141,7 @@ void testSerDe() throws Exception { root.putObject("multiSet").put("element", 2); root.putObject("map2map").putObject("inner_map").put("key", 234); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); + byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); DataType dataType = ROW( @@ -220,8 +222,7 @@ void testSlowDeserialization() throws Exception { double doubleValue = random.nextDouble(); float floatValue = random.nextFloat(); - ObjectMapper objectMapper = new ObjectMapper(); - ObjectNode root = objectMapper.createObjectNode(); + ObjectNode root = OBJECT_MAPPER.createObjectNode(); root.put("bool", String.valueOf(bool)); root.put("int", String.valueOf(integer)); root.put("bigint", String.valueOf(bigint)); @@ -230,7 +231,7 @@ void testSlowDeserialization() throws Exception { root.put("float1", String.valueOf(floatValue)); root.put("float2", new BigDecimal(floatValue)); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); + byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); DataType dataType = ROW( @@ -296,11 +297,9 @@ void testSerDeMultiRows() throws Exception { true); open(serializationSchema); - ObjectMapper objectMapper = new ObjectMapper(); - // the first row { - ObjectNode root = objectMapper.createObjectNode(); + ObjectNode root = OBJECT_MAPPER.createObjectNode(); root.put("f1", 1); root.put("f2", true); root.put("f3", "str"); @@ -312,7 +311,7 @@ void testSerDeMultiRows() throws Exception { ObjectNode row = root.putObject("f6"); row.put("f1", "this is row1"); row.put("f2", 12); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); + byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); RowData rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); assertThat(serializedJson).containsExactly(actual); @@ -320,7 +319,7 @@ void testSerDeMultiRows() throws Exception { // the second row { - ObjectNode root = objectMapper.createObjectNode(); + ObjectNode root = OBJECT_MAPPER.createObjectNode(); root.put("f1", 10); root.put("f2", false); root.put("f3", "newStr"); @@ -332,7 +331,7 @@ void testSerDeMultiRows() throws Exception { ObjectNode row = root.putObject("f6"); row.put("f1", "this is row2"); row.putNull("f2"); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); + byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); RowData rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); assertThat(serializedJson).containsExactly(actual); @@ -419,12 +418,10 @@ void testDeserializationMissingNode() throws Exception { @Test void testDeserializationMissingField() throws Exception { - ObjectMapper objectMapper = new ObjectMapper(); - // Root - ObjectNode root = objectMapper.createObjectNode(); + ObjectNode root = OBJECT_MAPPER.createObjectNode(); root.put("id", 123123123); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); + byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); DataType dataType = ROW(FIELD("name", STRING())); RowType schema = (RowType) dataType.getLogicalType(); @@ -504,14 +501,12 @@ void testSerDeSQLTimestampFormat() throws Exception { true); open(serializationSchema); - ObjectMapper objectMapper = new ObjectMapper(); - - ObjectNode root = objectMapper.createObjectNode(); + ObjectNode root = OBJECT_MAPPER.createObjectNode(); root.put("timestamp3", "1990-10-14 12:12:43.123"); root.put("timestamp9", "1990-10-14 12:12:43.123456789"); root.put("timestamp_with_local_timezone3", "1990-10-14 12:12:43.123Z"); root.put("timestamp_with_local_timezone9", "1990-10-14 12:12:43.123456789Z"); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); + byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); RowData rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); assertThat(serializedJson).containsExactly(actual); diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index deee53ae0..81e370c0f 100644 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.types.Row; +import org.apache.flink.util.jackson.JacksonMapperFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; @@ -52,6 +53,8 @@ /** Tests for the {@link JsonRowDeserializationSchema}. */ public class JsonRowDeserializationSchemaTest { + private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper(); + @Rule public ExpectedException thrown = ExpectedException.none(); /** Tests simple deserialization using type information. */ @@ -73,10 +76,8 @@ public void testTypeInfoDeserialization() throws Exception { innerMap.put("key", 234); nestedMap.put("inner_map", innerMap); - ObjectMapper objectMapper = new ObjectMapper(); - // Root - ObjectNode root = objectMapper.createObjectNode(); + ObjectNode root = OBJECT_MAPPER.createObjectNode(); root.put("id", id); root.put("name", name); root.put("bytes", bytes); @@ -89,7 +90,7 @@ public void testTypeInfoDeserialization() throws Exception { root.putObject("map").put("flink", 123); root.putObject("map2map").putObject("inner_map").put("key", 234); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); + byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder( @@ -149,10 +150,8 @@ public void testSchemaDeserialization() throws Exception { }; final String[] strings = new String[] {"one", "two", "three"}; - final ObjectMapper objectMapper = new ObjectMapper(); - // Root - ObjectNode root = objectMapper.createObjectNode(); + ObjectNode root = OBJECT_MAPPER.createObjectNode(); root.put("id", id.longValue()); root.putNull("idOrNull"); root.put("name", name); @@ -164,7 +163,7 @@ public void testSchemaDeserialization() throws Exception { root.putArray("strings").add("one").add("two").add("three"); root.putObject("nested").put("booleanField", true).put("decimalField", 12); - final byte[] serializedJson = objectMapper.writeValueAsBytes(root); + final byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder( @@ -212,12 +211,10 @@ public void testSchemaDeserialization() throws Exception { /** Tests deserialization with non-existing field name. */ @Test public void testMissingNode() throws Exception { - ObjectMapper objectMapper = new ObjectMapper(); - // Root - ObjectNode root = objectMapper.createObjectNode(); + ObjectNode root = OBJECT_MAPPER.createObjectNode(); root.put("id", 123123123); - byte[] serializedJson = objectMapper.writeValueAsBytes(root); + byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); TypeInformation rowTypeInformation = Types.ROW_NAMED(new String[] {"name"}, Types.STRING); From 077210ba3b84d1faa1d0b24b5168216e69717db8 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 10 Aug 2022 12:13:55 +0200 Subject: [PATCH 195/322] [FLINK-28060][kafka] Bump Kafka to 3.2.1 --- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 7d160a51f..4b1400686 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -32,7 +32,7 @@ under the License. Flink : Formats : Avro confluent registry - 3.1.1 + 3.2.1 6.2.2 From e71024c0fcb82383e7b1acf818d8772b5ce72a21 Mon Sep 17 00:00:00 2001 From: Benchao Li Date: Mon, 25 Jul 2022 22:35:01 +0800 Subject: [PATCH 196/322] [FLINK-28676] Update copyright year to 2014-2022 in NOTICE files --- .../src/main/resources/META-INF/NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 48394cfce..5723beb6b 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-avro-confluent-registry -Copyright 2014-2021 The Apache Software Foundation +Copyright 2014-2022 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From 54d3c6c3ba09ff8a3f4b8d536eace7eb26148ddf Mon Sep 17 00:00:00 2001 From: Ryan Skraba Date: Wed, 31 Aug 2022 01:56:16 -0700 Subject: [PATCH 197/322] [FLINK-24718] Update Avro dependency to 1.11.1 --- .../src/main/resources/META-INF/NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 5723beb6b..e259e971d 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -6,7 +6,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- org.apache.avro:avro:1.10.0 +- org.apache.avro:avro:1.11.1 - com.fasterxml.jackson.core:jackson-core:2.13.2 - com.fasterxml.jackson.core:jackson-databind:2.13.2.2 - com.fasterxml.jackson.core:jackson-annotations:2.13.2 From 4abce4b9e9ae3f0e762d7e20c3cf9bc5cc3b1d63 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Mon, 5 Sep 2022 10:43:15 +0800 Subject: [PATCH 198/322] Update version to 1.17-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 648065d66..83408f2fc 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.16-SNAPSHOT + 1.17-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 4b1400686..8745877f7 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.16-SNAPSHOT + 1.17-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index ab697cf7e..c9a1bfa80 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.16-SNAPSHOT + 1.17-SNAPSHOT flink-json diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index 123311900..d3b3373ac 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.16-SNAPSHOT + 1.17-SNAPSHOT flink-sql-avro-confluent-registry From 9975414c4cc57d2135979b77cb4db910a371484e Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 1 Feb 2022 15:09:19 +0100 Subject: [PATCH 199/322] [FLINK-25868][build] Disable japicmp in packaging/e2e modules Optimize build times and reduce noise (warnings would be printed as these modules are not being deployed, so the previous versions can not be found). --- flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index d3b3373ac..b4e956493 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -33,6 +33,10 @@ under the License. jar + + true + + confluent From a81eba698795fefa155a20a0bb78017470bad162 Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Wed, 5 Oct 2022 14:58:42 -0500 Subject: [PATCH 200/322] [FLINK-29513][Connector/Kafka] Update Kafka to version 3.2.3 which contains certain security fixes --- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 8745877f7..3303aaf19 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -32,7 +32,7 @@ under the License. Flink : Formats : Avro confluent registry - 3.2.1 + 3.2.3 6.2.2 From 85e5ca62c3d707902fe8ca744f42210c01d2152e Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Thu, 29 Sep 2022 11:46:43 +0200 Subject: [PATCH 201/322] [FLINK-29468][connectors][filesystems][formats] Update Jackson-BOM to 2.13.4. --- .../src/main/resources/META-INF/NOTICE | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index e259e971d..9b20d7171 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -7,9 +7,9 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - org.apache.avro:avro:1.11.1 -- com.fasterxml.jackson.core:jackson-core:2.13.2 -- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 -- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.4 +- com.fasterxml.jackson.core:jackson-databind:2.13.4 +- com.fasterxml.jackson.core:jackson-annotations:2.13.4 - org.apache.commons:commons-compress:1.21 - io.confluent:kafka-schema-registry-client:6.2.2 - org.apache.kafka:kafka-clients:6.2.2-ccs From d9a80bda51b96616444d672baa45308b98a49fe7 Mon Sep 17 00:00:00 2001 From: MartijnVisser Date: Tue, 18 Oct 2022 15:35:04 +0200 Subject: [PATCH 202/322] [FLINK-28405][Connector/Kafka] Update Confluent Platform images used for testing to v7.2.2. This closes #20170 * [FLINK-28405][Connector/Kafka] Update Confluent Platform images used for testing to v7.2.2 * [FLINK-28405][Connector/Kafka] Adding Guava to Schema Registry test since the Schema Registry client needs this, but doesn't bundle it * [FLINK-28405][Connector/Kafka] Refactored SchemaRegistryContainer so that we can include the Schema Registry container in DockerImageVersions * [FLINK-28405][Connector/Kafka] Make sure container gets cached --- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- .../src/main/resources/META-INF/NOTICE | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 3303aaf19..1fdc15303 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -33,7 +33,7 @@ under the License. 3.2.3 - 6.2.2 + 7.2.2 diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 9b20d7171..833e8f660 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -11,10 +11,10 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-databind:2.13.4 - com.fasterxml.jackson.core:jackson-annotations:2.13.4 - org.apache.commons:commons-compress:1.21 -- io.confluent:kafka-schema-registry-client:6.2.2 -- org.apache.kafka:kafka-clients:6.2.2-ccs -- io.confluent:common-config:6.2.2 -- io.confluent:common-utils:6.2.2 +- io.confluent:kafka-schema-registry-client:7.2.2 +- org.apache.kafka:kafka-clients:7.2.2-ccs +- io.confluent:common-config:7.2.2 +- io.confluent:common-utils:7.2.2 - org.glassfish.jersey.core:jersey-common:2.30 The binary distribution of this product bundles these dependencies under the Eclipse Public License - v 2.0 (https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt) From ad930a448b6218434bad37aa76e63995f4b9122f Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Thu, 13 Oct 2022 22:08:30 +0200 Subject: [PATCH 203/322] [FLINK-29638][connectors][filesystems][formats] Update Jackson-BOM to 2.13.4.2 --- .../src/main/resources/META-INF/NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 833e8f660..506da5509 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -8,7 +8,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.avro:avro:1.11.1 - com.fasterxml.jackson.core:jackson-core:2.13.4 -- com.fasterxml.jackson.core:jackson-databind:2.13.4 +- com.fasterxml.jackson.core:jackson-databind:2.13.4.2 - com.fasterxml.jackson.core:jackson-annotations:2.13.4 - org.apache.commons:commons-compress:1.21 - io.confluent:kafka-schema-registry-client:7.2.2 From 35ac27300119efc1c317af705e01e4d9e0e6fb9f Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Tue, 7 Feb 2023 16:27:19 +0800 Subject: [PATCH 204/322] Update version to 1.18-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-json-debezium/pom.xml | 2 +- flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 83408f2fc..fea104d9a 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.17-SNAPSHOT + 1.18-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 1fdc15303..8ced8eefc 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.17-SNAPSHOT + 1.18-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index c9a1bfa80..686ad5df3 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.17-SNAPSHOT + 1.18-SNAPSHOT flink-json diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index b4e956493..e031948cb 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.17-SNAPSHOT + 1.18-SNAPSHOT flink-sql-avro-confluent-registry From efad9b5bba94defb7127da42c5fccbe94be78126 Mon Sep 17 00:00:00 2001 From: David Moravek Date: Tue, 7 Mar 2023 13:39:51 +0100 Subject: [PATCH 205/322] [FLINK-31348][Docs] Fix unclosed shortcodes (/tabs) in the docs, that causes issues with Hugo version 0.111.0 and higher. This closes #22113 --- docs/content/docs/connectors/table/formats/debezium.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/docs/connectors/table/formats/debezium.md b/docs/content/docs/connectors/table/formats/debezium.md index e03a1535f..e025ac4de 100644 --- a/docs/content/docs/connectors/table/formats/debezium.md +++ b/docs/content/docs/connectors/table/formats/debezium.md @@ -418,6 +418,7 @@ Use format `debezium-avro-confluent` to interpret Debezium Avro messages and for {{< /tab >}} +{{< /tabs >}} Caveats ---------------- From 6f905fb67d00885cb778bf38e4714cd26e5ffb96 Mon Sep 17 00:00:00 2001 From: Fruzsina Nagy <37060328+fruska@users.noreply.github.com> Date: Wed, 15 Mar 2023 11:41:33 +0100 Subject: [PATCH 206/322] [FLINK-31085][formats] Add schema option to confluent registry avro formats --- .../table/formats/avro-confluent.md | 7 + .../docs/connectors/table/formats/debezium.md | 7 + .../table/formats/avro-confluent.md | 8 + .../docs/connectors/table/formats/debezium.md | 7 + .../confluent/AvroConfluentFormatOptions.java | 12 ++ .../confluent/RegistryAvroFormatFactory.java | 45 +++++- .../DebeziumAvroDeserializationSchema.java | 15 +- .../debezium/DebeziumAvroFormatFactory.java | 32 +++- .../DebeziumAvroSerializationSchema.java | 15 +- .../RegistryAvroFormatFactoryTest.java | 54 ++++++- .../DebeziumAvroFormatFactoryTest.java | 144 +++++++++++++++++- 11 files changed, 323 insertions(+), 23 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md index ddb3c0880..fdc79540b 100644 --- a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md @@ -259,6 +259,13 @@ Format 参数 String Password for SSL truststore + +

avro-confluent.schema
+ optional + (none) + String + The schema registered or to be registered in the Confluent Schema Registry. If no schema is provided Flink converts the table schema to avro schema. The schema provided must match the table schema. +
avro-confluent.subject
optional diff --git a/docs/content.zh/docs/connectors/table/formats/debezium.md b/docs/content.zh/docs/connectors/table/formats/debezium.md index 375968d16..a6ac486f0 100644 --- a/docs/content.zh/docs/connectors/table/formats/debezium.md +++ b/docs/content.zh/docs/connectors/table/formats/debezium.md @@ -327,6 +327,13 @@ Flink 提供了 `debezium-avro-confluent` 和 `debezium-json` 两种 format 来 String Password for SSL truststore + +
debezium-avro-confluent.schema
+ optional + (none) + String + The schema registered or to be registered in the Confluent Schema Registry. If no schema is provided Flink converts the table schema to avro schema. The schema provided must match the Debezium schema which is a nullable record type including fields 'before', 'after', 'op'. +
debezium-avro-confluent.subject
optional diff --git a/docs/content/docs/connectors/table/formats/avro-confluent.md b/docs/content/docs/connectors/table/formats/avro-confluent.md index 28b33da71..03a847713 100644 --- a/docs/content/docs/connectors/table/formats/avro-confluent.md +++ b/docs/content/docs/connectors/table/formats/avro-confluent.md @@ -263,6 +263,14 @@ Format Options String Password for SSL truststore + +
avro-confluent.schema
+ optional + no + (none) + String + The schema registered or to be registered in the Confluent Schema Registry. If no schema is provided Flink converts the table schema to avro schema. The schema provided must match the table schema. +
avro-confluent.subject
optional diff --git a/docs/content/docs/connectors/table/formats/debezium.md b/docs/content/docs/connectors/table/formats/debezium.md index e025ac4de..571992a21 100644 --- a/docs/content/docs/connectors/table/formats/debezium.md +++ b/docs/content/docs/connectors/table/formats/debezium.md @@ -321,6 +321,13 @@ Use format `debezium-avro-confluent` to interpret Debezium Avro messages and for String Password for SSL truststore + +
debezium-avro-confluent.schema
+ optional + (none) + String + The schema registered or to be registered in the Confluent Schema Registry. If no schema is provided Flink converts the table schema to avro schema. The schema provided must match the Debezium schema which is a nullable record type including fields 'before', 'after', 'op'. +
debezium-avro-confluent.subject
optional diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java index 9d06a6340..42e7353d0 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java @@ -49,6 +49,18 @@ public class AvroConfluentFormatOptions { + "is used as the value or key format. But for other connectors (e.g. 'filesystem'), " + "the subject option is required when used as sink."); + public static final ConfigOption SCHEMA = + ConfigOptions.key("schema") + .stringType() + .noDefaultValue() + .withFallbackKeys("schema-registry.schema") + .withDescription( + "The schema registered or to be registered in the Confluent Schema Registry. " + + "If no schema is provided Flink converts the table schema to avro schema. " + + "The schema provided must match the table schema ('avro-confluent') or " + + "the Debezium schema which is a nullable record type including " + + "fields 'before', 'after', 'op' ('debezium-avro-confluent')."); + // -------------------------------------------------------------------------------------------- // Commonly used options maintained by Flink for convenience // -------------------------------------------------------------------------------------------- diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java index 4030168c9..e43d5a76e 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java @@ -43,8 +43,12 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.SerializationFormatFactory; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Parser; + import javax.annotation.Nullable; import java.util.HashMap; @@ -55,11 +59,13 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static java.lang.String.format; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_CREDENTIALS_SOURCE; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_CREDENTIALS_SOURCE; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_TOKEN; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.PROPERTIES; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SCHEMA; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_LOCATION; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_PASSWORD; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_LOCATION; @@ -83,6 +89,7 @@ public DecodingFormat> createDecodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); String schemaRegistryURL = formatOptions.get(URL); + Optional schemaString = formatOptions.getOptional(SCHEMA); Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); return new ProjectableDecodingFormat>() { @@ -93,13 +100,15 @@ public DeserializationSchema createRuntimeDecoder( int[][] projections) { producedDataType = Projection.of(projections).project(producedDataType); final RowType rowType = (RowType) producedDataType.getLogicalType(); + final Schema schema = + schemaString + .map(s -> getAvroSchema(s, rowType)) + .orElse(AvroSchemaConverter.convertToSchema(rowType)); final TypeInformation rowDataTypeInfo = context.createTypeInformation(producedDataType); return new AvroRowDataDeserializationSchema( ConfluentRegistryAvroDeserializationSchema.forGeneric( - AvroSchemaConverter.convertToSchema(rowType), - schemaRegistryURL, - optionalPropertiesMap), + schema, schemaRegistryURL, optionalPropertiesMap), AvroToRowDataConverters.createRowConverter(rowType), rowDataTypeInfo); } @@ -118,6 +127,7 @@ public EncodingFormat> createEncodingFormat( String schemaRegistryURL = formatOptions.get(URL); Optional subject = formatOptions.getOptional(SUBJECT); + Optional schemaString = formatOptions.getOptional(SCHEMA); Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); if (!subject.isPresent()) { @@ -132,13 +142,14 @@ public EncodingFormat> createEncodingFormat( public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); + final Schema schema = + schemaString + .map(s -> getAvroSchema(s, rowType)) + .orElse(AvroSchemaConverter.convertToSchema(rowType)); return new AvroRowDataSerializationSchema( rowType, ConfluentRegistryAvroSerializationSchema.forGeneric( - subject.get(), - AvroSchemaConverter.convertToSchema(rowType), - schemaRegistryURL, - optionalPropertiesMap), + subject.get(), schema, schemaRegistryURL, optionalPropertiesMap), RowDataToAvroConverters.createConverter(rowType)); } @@ -165,6 +176,7 @@ public Set> requiredOptions() { public Set> optionalOptions() { Set> options = new HashSet<>(); options.add(SUBJECT); + options.add(SCHEMA); options.add(PROPERTIES); options.add(SSL_KEYSTORE_LOCATION); options.add(SSL_KEYSTORE_PASSWORD); @@ -182,6 +194,7 @@ public Set> forwardOptions() { return Stream.of( URL, SUBJECT, + SCHEMA, PROPERTIES, SSL_KEYSTORE_LOCATION, SSL_KEYSTORE_PASSWORD, @@ -230,4 +243,22 @@ public Set> forwardOptions() { } return properties; } + + private static Schema getAvroSchema(String schemaString, RowType rowType) { + LogicalType convertedDataType = + AvroSchemaConverter.convertToDataType(schemaString).getLogicalType(); + + if (convertedDataType.isNullable()) { + convertedDataType = convertedDataType.copy(false); + } + + if (!convertedDataType.equals(rowType)) { + throw new IllegalArgumentException( + format( + "Schema provided for '%s' format does not match the table schema: %s", + IDENTIFIER, schemaString)); + } + + return new Parser().parse(schemaString); + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java index 9ae5133ee..f72309e69 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java @@ -34,6 +34,9 @@ import org.apache.flink.types.RowKind; import org.apache.flink.util.Collector; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Parser; + import javax.annotation.Nullable; import java.io.IOException; @@ -41,6 +44,7 @@ import java.util.Objects; import static java.lang.String.format; +import static org.apache.flink.formats.avro.registry.confluent.debezium.DebeziumAvroFormatFactory.validateSchemaString; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** @@ -80,16 +84,21 @@ public DebeziumAvroDeserializationSchema( RowType rowType, TypeInformation producedTypeInfo, String schemaRegistryUrl, + @Nullable String schemaString, @Nullable Map registryConfigs) { this.producedTypeInfo = producedTypeInfo; RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + validateSchemaString(schemaString, debeziumAvroRowType); + Schema schema = + schemaString == null + ? AvroSchemaConverter.convertToSchema(debeziumAvroRowType) + : new Parser().parse(schemaString); + this.avroDeserializer = new AvroRowDataDeserializationSchema( ConfluentRegistryAvroDeserializationSchema.forGeneric( - AvroSchemaConverter.convertToSchema(debeziumAvroRowType), - schemaRegistryUrl, - registryConfigs), + schema, schemaRegistryUrl, registryConfigs), AvroToRowDataConverters.createRowConverter(debeziumAvroRowType), producedTypeInfo); } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java index c9773dcf4..5925f23f7 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.Projection; @@ -38,19 +39,24 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.SerializationFormatFactory; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.RowKind; +import javax.annotation.Nullable; + import java.util.HashSet; import java.util.Map; import java.util.Optional; import java.util.Set; +import static java.lang.String.format; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_CREDENTIALS_SOURCE; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_CREDENTIALS_SOURCE; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_TOKEN; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.PROPERTIES; +import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SCHEMA; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_LOCATION; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_PASSWORD; import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_LOCATION; @@ -75,6 +81,7 @@ public DecodingFormat> createDecodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); String schemaRegistryURL = formatOptions.get(URL); + String schema = formatOptions.getOptional(SCHEMA).orElse(null); Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); return new ProjectableDecodingFormat>() { @@ -88,7 +95,11 @@ public DeserializationSchema createRuntimeDecoder( final TypeInformation producedTypeInfo = context.createTypeInformation(producedDataType); return new DebeziumAvroDeserializationSchema( - rowType, producedTypeInfo, schemaRegistryURL, optionalPropertiesMap); + rowType, + producedTypeInfo, + schemaRegistryURL, + schema, + optionalPropertiesMap); } @Override @@ -110,6 +121,7 @@ public EncodingFormat> createEncodingFormat( FactoryUtil.validateFactoryOptions(this, formatOptions); String schemaRegistryURL = formatOptions.get(URL); Optional subject = formatOptions.getOptional(SUBJECT); + String schema = formatOptions.getOptional(SCHEMA).orElse(null); Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); if (!subject.isPresent()) { @@ -135,7 +147,7 @@ public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { final RowType rowType = (RowType) consumedDataType.getLogicalType(); return new DebeziumAvroSerializationSchema( - rowType, schemaRegistryURL, subject.get(), optionalPropertiesMap); + rowType, schemaRegistryURL, subject.get(), schema, optionalPropertiesMap); } }; } @@ -157,6 +169,7 @@ public Set> optionalOptions() { Set> options = new HashSet<>(); options.add(SUBJECT); options.add(PROPERTIES); + options.add(SCHEMA); options.add(SSL_KEYSTORE_LOCATION); options.add(SSL_KEYSTORE_PASSWORD); options.add(SSL_TRUSTSTORE_LOCATION); @@ -167,4 +180,19 @@ public Set> optionalOptions() { options.add(BEARER_AUTH_TOKEN); return options; } + + static void validateSchemaString(@Nullable String schemaString, RowType rowType) { + if (schemaString != null) { + LogicalType convertedDataType = + AvroSchemaConverter.convertToDataType(schemaString).getLogicalType(); + + if (!convertedDataType.equals(rowType)) { + throw new IllegalArgumentException( + format( + "Schema provided for '%s' format must be a nullable record type with fields 'before', 'after', 'op'" + + " and schema of fields 'before' and 'after' must match the table schema: %s", + IDENTIFIER, schemaString)); + } + } + } } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java index 4cecceff2..a5b4eb7ab 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java @@ -32,12 +32,16 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Parser; + import javax.annotation.Nullable; import java.util.Map; import java.util.Objects; import static java.lang.String.format; +import static org.apache.flink.formats.avro.registry.confluent.debezium.DebeziumAvroFormatFactory.validateSchemaString; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; /** @@ -62,17 +66,20 @@ public DebeziumAvroSerializationSchema( RowType rowType, String schemaRegistryUrl, String schemaRegistrySubject, + @Nullable String schemaString, @Nullable Map registryConfigs) { RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); + validateSchemaString(schemaString, debeziumAvroRowType); + Schema schema = + schemaString == null + ? AvroSchemaConverter.convertToSchema(debeziumAvroRowType) + : new Parser().parse(schemaString); this.avroSerializer = new AvroRowDataSerializationSchema( debeziumAvroRowType, ConfluentRegistryAvroSerializationSchema.forGeneric( - schemaRegistrySubject, - AvroSchemaConverter.convertToSchema(debeziumAvroRowType), - schemaRegistryUrl, - registryConfigs), + schemaRegistrySubject, schema, schemaRegistryUrl, registryConfigs), RowDataToAvroConverters.createConverter(debeziumAvroRowType)); } diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java index cf80adf3e..6ace72b19 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java @@ -37,6 +37,7 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; +import org.apache.avro.Schema.Parser; import org.junit.jupiter.api.Test; import java.util.HashMap; @@ -48,6 +49,7 @@ import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.Assert.assertThrows; /** Tests for the {@link RegistryAvroFormatFactory}. */ class RegistryAvroFormatFactoryTest { @@ -63,6 +65,37 @@ class RegistryAvroFormatFactoryTest { private static final String SUBJECT = "test-subject"; private static final String REGISTRY_URL = "http://localhost:8081"; + private static final String SCHEMA_STRING = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"test_record\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"a\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " \"string\"\n" + + " ],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\": \"b\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " \"int\"\n" + + " ],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\": \"c\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " \"boolean\"\n" + + " ],\n" + + " \"default\": null\n" + + " }\n" + + " ]\n" + + "}"; private static final Map EXPECTED_OPTIONAL_PROPERTIES = new HashMap<>(); @@ -139,7 +172,7 @@ void testDeserializationSchemaWithOptionalProperties() { final AvroRowDataDeserializationSchema expectedDeser = new AvroRowDataDeserializationSchema( ConfluentRegistryAvroDeserializationSchema.forGeneric( - AvroSchemaConverter.convertToSchema(ROW_TYPE), + new Parser().parse(SCHEMA_STRING), REGISTRY_URL, EXPECTED_OPTIONAL_PROPERTIES), AvroToRowDataConverters.createRowConverter(ROW_TYPE), @@ -164,7 +197,7 @@ void testSerializationSchemaWithOptionalProperties() { ROW_TYPE, ConfluentRegistryAvroSerializationSchema.forGeneric( SUBJECT, - AvroSchemaConverter.convertToSchema(ROW_TYPE), + new Parser().parse(SCHEMA_STRING), REGISTRY_URL, EXPECTED_OPTIONAL_PROPERTIES), RowDataToAvroConverters.createConverter(ROW_TYPE)); @@ -180,6 +213,22 @@ void testSerializationSchemaWithOptionalProperties() { assertThat(actualSer).isEqualTo(expectedSer); } + @Test + public void testSerializationSchemaWithInvalidOptionalSchema() { + Map optionalProperties = getOptionalProperties(); + optionalProperties.put("avro-confluent.schema", SCHEMA_STRING.replace("int", "string")); + + final DynamicTableSink actualSink = createTableSink(SCHEMA, optionalProperties); + TestDynamicTableFactory.DynamicTableSinkMock sinkMock = + (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; + + assertThrows( + IllegalArgumentException.class, + () -> + sinkMock.valueFormat.createRuntimeEncoder( + null, SCHEMA.toPhysicalRowDataType())); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -222,6 +271,7 @@ private Map getOptionalProperties() { properties.put(AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO.key(), "user:pwd"); // defined via general property map properties.put("properties.bearer.auth.token", "CUSTOM"); + properties.put("schema", SCHEMA_STRING); return getModifiedOptions( opts -> diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java index 50d270baf..b39cc1e28 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java +++ b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java @@ -32,6 +32,7 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; +import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.Test; import java.util.HashMap; @@ -40,6 +41,8 @@ import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; /** Tests for {@link DebeziumAvroFormatFactory}. */ class DebeziumAvroFormatFactoryTest { @@ -50,6 +53,68 @@ class DebeziumAvroFormatFactoryTest { Column.physical("b", DataTypes.INT()), Column.physical("c", DataTypes.BOOLEAN())); + private static final String RECORD_SCHEMA = + " {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"test\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"before\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"testSchema\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"a\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " \"string\"\n" + + " ],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\": \"b\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " \"int\"\n" + + " ],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\": \"c\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " \"boolean\"\n" + + " ],\n" + + " \"default\": null\n" + + " }\n" + + " ]\n" + + " }\n" + + " ],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\": \"after\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " \"testSchema\"\n" + + " ],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\": \"op\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " \"string\"\n" + + " ],\n" + + " \"default\": null\n" + + " }\n" + + " ]\n" + + " }\n"; + private static final String AVRO_SCHEMA = "[\n\"null\",\n" + RECORD_SCHEMA + "]"; + private static final RowType ROW_TYPE = (RowType) SCHEMA.toPhysicalRowDataType().getLogicalType(); @@ -59,24 +124,93 @@ class DebeziumAvroFormatFactoryTest { @Test void testSeDeSchema() { final Map options = getAllOptions(); + final Map registryConfigs = getRegistryConfigs(); - final Map registryConfigs = new HashMap<>(); - registryConfigs.put("basic.auth.user.info", "something1"); - registryConfigs.put("basic.auth.credentials.source", "something2"); + DebeziumAvroDeserializationSchema expectedDeser = + new DebeziumAvroDeserializationSchema( + ROW_TYPE, + InternalTypeInfo.of(ROW_TYPE), + REGISTRY_URL, + null, + registryConfigs); + DeserializationSchema actualDeser = createDeserializationSchema(options); + assertEquals(expectedDeser, actualDeser); + + DebeziumAvroSerializationSchema expectedSer = + new DebeziumAvroSerializationSchema( + ROW_TYPE, REGISTRY_URL, SUBJECT, null, registryConfigs); + SerializationSchema actualSer = createSerializationSchema(options); + assertEquals(expectedSer, actualSer); + } + + @Test + public void testSeDeSchemaWithSchemaOption() { + final Map options = getAllOptions(); + options.put("debezium-avro-confluent.schema", AVRO_SCHEMA); + + final Map registryConfigs = getRegistryConfigs(); DebeziumAvroDeserializationSchema expectedDeser = new DebeziumAvroDeserializationSchema( - ROW_TYPE, InternalTypeInfo.of(ROW_TYPE), REGISTRY_URL, registryConfigs); + ROW_TYPE, + InternalTypeInfo.of(ROW_TYPE), + REGISTRY_URL, + AVRO_SCHEMA, + registryConfigs); DeserializationSchema actualDeser = createDeserializationSchema(options); assertThat(actualDeser).isEqualTo(expectedDeser); DebeziumAvroSerializationSchema expectedSer = new DebeziumAvroSerializationSchema( - ROW_TYPE, REGISTRY_URL, SUBJECT, registryConfigs); + ROW_TYPE, REGISTRY_URL, SUBJECT, AVRO_SCHEMA, registryConfigs); SerializationSchema actualSer = createSerializationSchema(options); assertThat(actualSer).isEqualTo(expectedSer); } + @Test + public void testSeDeSchemaWithInvalidSchemaOption() { + final Map options = getAllOptions(); + options.put("debezium-avro-confluent.schema", RECORD_SCHEMA); + + assertThrows(IllegalArgumentException.class, () -> createDeserializationSchema(options)); + assertThrows(IllegalArgumentException.class, () -> createSerializationSchema(options)); + + String basicSchema = "[ \"null\", \"string\" ]"; + options.put("debezium-avro-confluent.schema", basicSchema); + assertThrows(IllegalArgumentException.class, () -> createDeserializationSchema(options)); + assertThrows(IllegalArgumentException.class, () -> createSerializationSchema(options)); + + String invalidSchema = + "[\"null\", " + + "{ \"type\" : \"record\"," + + "\"name\" : \"debezium\"," + + "\"fields\": [{\n" + + " \"default\": null,\n" + + " \"name\": \"op\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " \"string\"\n" + + " ]\n" + + " }]" + + "}]"; + options.put("debezium-avro-confluent.schema", invalidSchema); + assertThrows(IllegalArgumentException.class, () -> createDeserializationSchema(options)); + assertThrows(IllegalArgumentException.class, () -> createSerializationSchema(options)); + + String invalidRecordSchema = AVRO_SCHEMA.replace("int", "string"); + options.put("debezium-avro-confluent.schema", invalidRecordSchema); + assertThrows(IllegalArgumentException.class, () -> createDeserializationSchema(options)); + assertThrows(IllegalArgumentException.class, () -> createSerializationSchema(options)); + } + + @NotNull + private Map getRegistryConfigs() { + final Map registryConfigs = new HashMap<>(); + registryConfigs.put("basic.auth.user.info", "something1"); + registryConfigs.put("basic.auth.credentials.source", "something2"); + return registryConfigs; + } + private Map getAllOptions() { final Map options = new HashMap<>(); options.put("connector", TestDynamicTableFactory.IDENTIFIER); From f1a078c7bb826721d84c019a263a28468b80e546 Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Thu, 16 Mar 2023 11:29:50 +0100 Subject: [PATCH 207/322] [FLINK-31485][Confluent Avro] Change packaging of Schema Registry so that by default everything is included. This resolves the accidental exclusion of Guava, causing Avro Confluent Schema Registry to fail. --- .../flink-avro-confluent-registry/pom.xml | 28 +++++++++++++++++++ .../flink-sql-avro-confluent-registry/pom.xml | 25 +++-------------- .../src/main/resources/META-INF/NOTICE | 7 +++++ .../META-INF/licenses/LICENSE.zstd-jni | 26 +++++++++++++++++ 4 files changed, 65 insertions(+), 21 deletions(-) create mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.zstd-jni diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index 8ced8eefc..d36676066 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -65,6 +65,34 @@ under the License. io.swagger swagger-core + + com.google.guava + failureaccess + + + com.google.guava + listenablefuture + + + io.swagger.core.v3 + swagger-annotations + + + com.google.errorprone + error_prone_annotations + + + com.google.j2objc + j2objc-annotations + + + com.google.code.findbugs + jsr305 + + + org.checkerframework + checker-qual +
diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index e031948cb..10789e462 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -67,21 +67,11 @@ under the License. - io.confluent:* - - org.apache.kafka:kafka-clients - org.apache.flink:flink-avro - org.apache.flink:flink-avro-confluent-registry - org.apache.avro:avro - com.fasterxml.jackson.core:* - jakarta.ws.rs:jakarta.ws.rs-api - jakarta.annotation:jakarta.annotation-api - org.glassfish.jersey.core:jersey-common - org.glassfish.hk2.external:jakarta.inject - org.glassfish.hk2:osgi-resource-locator - org.apache.commons:commons-compress + *:* + + com.google.code.findbugs:jsr305 + @@ -108,13 +98,6 @@ under the License. - - org.glassfish.jersey.core:jersey-common - - - META-INF/versions/** - - org.apache.kafka:* diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 506da5509..f841db93c 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -16,6 +16,8 @@ This project bundles the following dependencies under the Apache Software Licens - io.confluent:common-config:7.2.2 - io.confluent:common-utils:7.2.2 - org.glassfish.jersey.core:jersey-common:2.30 +- com.google.guava:guava:30.1.1-jre +- org.xerial.snappy:snappy-java:1.1.8.3 The binary distribution of this product bundles these dependencies under the Eclipse Public License - v 2.0 (https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt) @@ -23,3 +25,8 @@ The binary distribution of this product bundles these dependencies under the Ecl - jakarta.ws.rs:jakarta.ws.rs-api:2.1.6 - org.glassfish.hk2.external:jakarta.inject:2.6.1 - org.glassfish.hk2:osgi-resource-locator:1.0.3 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- com.github.luben:zstd-jni:1.5.2-1 diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.zstd-jni b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.zstd-jni new file mode 100644 index 000000000..66abb8ae7 --- /dev/null +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.zstd-jni @@ -0,0 +1,26 @@ +Zstd-jni: JNI bindings to Zstd Library + +Copyright (c) 2015-present, Luben Karavelov/ All rights reserved. + +BSD License + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. From 453592b99f0403619ad00eb68ca1d90e3fd48263 Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Thu, 16 Mar 2023 11:31:07 +0100 Subject: [PATCH 208/322] [hotfix] Sort NOTICE dependencies in alphabetical order --- .../src/main/resources/META-INF/NOTICE | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index f841db93c..a89095bdf 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -6,17 +6,17 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- org.apache.avro:avro:1.11.1 +- com.fasterxml.jackson.core:jackson-annotations:2.13.4 - com.fasterxml.jackson.core:jackson-core:2.13.4 - com.fasterxml.jackson.core:jackson-databind:2.13.4.2 -- com.fasterxml.jackson.core:jackson-annotations:2.13.4 -- org.apache.commons:commons-compress:1.21 -- io.confluent:kafka-schema-registry-client:7.2.2 -- org.apache.kafka:kafka-clients:7.2.2-ccs +- com.google.guava:guava:30.1.1-jre - io.confluent:common-config:7.2.2 - io.confluent:common-utils:7.2.2 +- io.confluent:kafka-schema-registry-client:7.2.2 +- org.apache.avro:avro:1.11.1 +- org.apache.commons:commons-compress:1.21 +- org.apache.kafka:kafka-clients:7.2.2-ccs - org.glassfish.jersey.core:jersey-common:2.30 -- com.google.guava:guava:30.1.1-jre - org.xerial.snappy:snappy-java:1.1.8.3 The binary distribution of this product bundles these dependencies under the Eclipse Public License - v 2.0 (https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt) From 0d3088051b0fc7319e34a764cfc8f590ec728e1a Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Tue, 21 Mar 2023 12:46:07 +0100 Subject: [PATCH 209/322] [hotfix] Update copyright NOTICE year to 2023 --- .../src/main/resources/META-INF/NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index a89095bdf..6e85cfe9c 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-avro-confluent-registry -Copyright 2014-2022 The Apache Software Foundation +Copyright 2014-2023 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From d762d01a7ccb20be01818433467db90e855c9d7f Mon Sep 17 00:00:00 2001 From: mas-chen Date: Fri, 24 Mar 2023 00:42:43 -0700 Subject: [PATCH 210/322] [FLINK-30859] Externalize confluent avro related code [FLINK-30859] Fix NOTICE file to point to the right kafka-clients version [FLINK-30859] Redirect confluent-avro deps to project.version [FLINK-30859] port json debezium, rename flink-formats-kafka and cleanup poms Co-Authored-By: Tzu-Li (Gordon) Tai --- .../pom.xml | 9 +- .../pom.xml | 8 +- flink-connector-kafka/pom.xml | 7 +- .../table/KafkaDynamicTableFactoryTest.java | 2 +- .../flink-avro-confluent-registry/pom.xml | 58 +- .../62c5e4e5-2b0e-41ed-a268-ee33d5edd162 | 6 - .../flink-json-debezium/pom.xml | 61 +- .../json/JsonDeserializationSchema.java | 71 -- .../flink/formats/json/JsonFormatFactory.java | 164 ---- .../flink/formats/json/JsonFormatOptions.java | 88 -- .../formats/json/JsonFormatOptionsUtil.java | 147 --- .../json/JsonNodeDeserializationSchema.java | 37 - .../JsonRowDataDeserializationSchema.java | 163 ---- .../json/JsonRowDataSerializationSchema.java | 135 --- .../json/JsonRowDeserializationSchema.java | 587 ------------ .../formats/json/JsonRowSchemaConverter.java | 382 -------- .../json/JsonRowSerializationSchema.java | 436 --------- .../formats/json/JsonSerializationSchema.java | 59 -- .../formats/json/JsonToRowDataConverters.java | 402 -------- .../formats/json/RowDataToJsonConverters.java | 355 ------- .../json/canal/CanalJsonDecodingFormat.java | 245 ----- .../canal/CanalJsonDeserializationSchema.java | 412 --------- .../json/canal/CanalJsonFormatFactory.java | 151 --- .../json/canal/CanalJsonFormatOptions.java | 58 -- .../canal/CanalJsonSerializationSchema.java | 132 --- .../maxwell/MaxwellJsonDecodingFormat.java | 205 ---- .../MaxwellJsonDeserializationSchema.java | 282 ------ .../maxwell/MaxwellJsonFormatFactory.java | 145 --- .../maxwell/MaxwellJsonFormatOptions.java | 41 - .../MaxwellJsonSerializationSchema.java | 122 --- .../json/ogg/OggJsonDecodingFormat.java | 207 ----- .../ogg/OggJsonDeserializationSchema.java | 278 ------ .../json/ogg/OggJsonFormatFactory.java | 148 --- .../json/ogg/OggJsonFormatOptions.java | 41 - .../json/ogg/OggJsonSerializationSchema.java | 130 --- .../org.apache.flink.table.factories.Factory | 4 - .../json/JsonBatchFileSystemITCase.java | 119 --- .../json/JsonFileCompactionITCase.java | 30 - .../formats/json/JsonFormatFactoryTest.java | 232 ----- .../formats/json/JsonFsStreamSinkITCase.java | 37 - .../JsonNodeDeserializationSchemaTest.java | 50 - .../json/JsonRowDataSerDeSchemaTest.java | 872 ------------------ .../JsonRowDeserializationSchemaTest.java | 432 --------- .../json/JsonRowSchemaConverterTest.java | 152 --- .../json/JsonRowSerializationSchemaTest.java | 250 ----- .../formats/json/JsonSerDeSchemaTest.java | 111 --- .../canal/CanalJsonFormatFactoryTest.java | 206 ----- .../json/canal/CanalJsonSerDeSchemaTest.java | 321 ------- .../maxwell/MaxwellJsonFormatFactoryTest.java | 170 ---- .../json/maxwell/MaxwellJsonSerDerTest.java | 252 ----- .../json/ogg/OggJsonFileSystemITCase.java | 159 ---- .../json/ogg/OggJsonFormatFactoryTest.java | 142 --- .../json/ogg/OggJsonSerDeSchemaTest.java | 266 ------ .../utils/DeserializationSchemaMatcher.java | 161 ---- .../utils/SerializationSchemaMatcher.java | 191 ---- .../flink-sql-avro-confluent-registry/pom.xml | 4 +- .../src/main/resources/META-INF/NOTICE | 2 +- flink-formats-kafka/pom.xml | 42 + pom.xml | 5 + 59 files changed, 118 insertions(+), 9866 deletions(-) delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptionsUtil.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatOptions.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatOptions.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java create mode 100644 flink-formats-kafka/pom.xml diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index fea104d9a..75ff2cc0a 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -20,9 +20,9 @@ under the License. xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - flink-end-to-end-tests org.apache.flink - 1.18-SNAPSHOT + flink-connector-kafka-parent + 4.1-SNAPSHOT 4.0.0 @@ -44,10 +44,9 @@ under the License. org.apache.flink flink-streaming-java - ${project.version} + ${flink.version} provided - org.apache.flink flink-connector-kafka @@ -56,7 +55,7 @@ under the License. org.apache.flink flink-avro - ${project.version} + ${flink.version} org.apache.flink diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index 6a87c4623..bd7c67a13 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -37,10 +37,6 @@ under the License. - - 7.2.2 - - org.apache.flink @@ -95,7 +91,7 @@ under the License. org.apache.flink flink-sql-avro-confluent-registry - ${flink.version} + ${project.version} test @@ -198,7 +194,7 @@ under the License. org.apache.flink flink-sql-avro-confluent-registry - ${flink.version} + ${project.version} avro-confluent.jar jar ${project.build.directory}/dependencies diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 5c11705c5..f754ae26c 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -34,11 +34,6 @@ under the License. jar - - 2.12 - 2.0.9 - - @@ -241,7 +236,7 @@ under the License. org.apache.flink flink-avro-confluent-registry - ${flink.version} + ${project.version} test diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java index 7ab050359..b0d61499a 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java @@ -866,7 +866,7 @@ private SerializationSchema createConfluentAvroSerSchema( private SerializationSchema createDebeziumAvroSerSchema( RowType rowType, String subject) { - return new DebeziumAvroSerializationSchema(rowType, TEST_REGISTRY_URL, subject, null); + return new DebeziumAvroSerializationSchema(rowType, TEST_REGISTRY_URL, subject, null, null); } // -------------------------------------------------------------------------------------------- diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index d36676066..a74fe8f6d 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -21,9 +21,9 @@ under the License. xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - flink-formats org.apache.flink - 1.18-SNAPSHOT + flink-formats-kafka + 4.1-SNAPSHOT 4.0.0 @@ -31,11 +31,6 @@ under the License. Flink : Formats : Avro confluent registry - - 3.2.3 - 7.2.2 - - confluent @@ -98,18 +93,18 @@ under the License. org.apache.flink flink-core - ${project.version} + ${flink.version} provided org.apache.flink flink-avro - ${project.version} + ${flink.version} org.apache.flink flink-table-api-java - ${project.version} + ${flink.version} provided @@ -118,7 +113,7 @@ under the License. org.apache.flink flink-table-common - ${project.version} + ${flink.version} provided true @@ -126,7 +121,7 @@ under the License. org.apache.flink flink-connector-files - ${project.version} + ${flink.version} provided true @@ -135,30 +130,36 @@ under the License. org.apache.flink flink-table-api-java - ${project.version} + ${flink.version} test test-jar org.apache.flink flink-table-runtime - ${project.version} + ${flink.version} test org.apache.flink flink-table-common - ${project.version} + ${flink.version} test test-jar org.apache.flink flink-avro - ${project.version} + ${flink.version} test test-jar + + org.powermock + powermock-reflect + ${powermock.version} + test + @@ -175,4 +176,29 @@ under the License. + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + org.apache.flink:flink-format-common + + + + + + + + + diff --git a/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 b/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 index a13a3ce8f..e69de29bb 100644 --- a/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 +++ b/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 @@ -1,6 +0,0 @@ -org.apache.flink.formats.json.JsonBatchFileSystemITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml index 686ad5df3..147cf0e30 100644 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ b/flink-formats-kafka/flink-json-debezium/pom.xml @@ -24,63 +24,45 @@ under the License. org.apache.flink - flink-formats - 1.18-SNAPSHOT + flink-formats-kafka + 4.1-SNAPSHOT - flink-json - Flink : Formats : Json + flink-json-debezium + Flink : Formats : Json Debezium jar - org.apache.flink - flink-format-common - ${project.version} - - - - - - org.apache.flink - flink-shaded-jackson - provided + flink-json + ${flink.version} - - org.apache.flink - flink-core - ${project.version} - provided - - - - org.apache.flink flink-table-common - ${project.version} + ${flink.version} provided true + + + org.apache.flink flink-connector-files - ${project.version} - provided - true + ${flink.version} + test - - org.apache.flink flink-table-common - ${project.version} + ${flink.version} test test-jar @@ -89,7 +71,7 @@ under the License. org.apache.flink flink-table-planner_${scala.binary.version} - ${project.version} + ${flink.version} test @@ -97,13 +79,13 @@ under the License. org.apache.flink flink-table-api-scala-bridge_${scala.binary.version} - ${project.version} + ${flink.version} test org.apache.flink flink-table-planner_${scala.binary.version} - ${project.version} + ${flink.version} test test-jar @@ -112,20 +94,13 @@ under the License. org.apache.flink flink-connector-test-utils - ${project.version} + ${flink.version} test org.apache.flink flink-test-utils - ${project.version} - test - - - - - org.scala-lang - scala-compiler + ${flink.version} test diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java deleted file mode 100644 index cc244b0a0..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonDeserializationSchema.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.util.function.SerializableSupplier; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; - -import java.io.IOException; - -/** DeserializationSchema that deserializes a JSON String. */ -@PublicEvolving -public class JsonDeserializationSchema extends AbstractDeserializationSchema { - - private static final long serialVersionUID = 1L; - - private final Class clazz; - private final SerializableSupplier mapperFactory; - protected transient ObjectMapper mapper; - - public JsonDeserializationSchema(Class clazz) { - this(clazz, JacksonMapperFactory::createObjectMapper); - } - - public JsonDeserializationSchema(TypeInformation typeInformation) { - this(typeInformation, JacksonMapperFactory::createObjectMapper); - } - - public JsonDeserializationSchema( - Class clazz, SerializableSupplier mapperFactory) { - super(clazz); - this.clazz = clazz; - this.mapperFactory = mapperFactory; - } - - public JsonDeserializationSchema( - TypeInformation typeInformation, SerializableSupplier mapperFactory) { - super(typeInformation); - this.clazz = typeInformation.getTypeClass(); - this.mapperFactory = mapperFactory; - } - - @Override - public void open(InitializationContext context) { - mapper = mapperFactory.get(); - } - - @Override - public T deserialize(byte[] message) throws IOException { - return mapper.readValue(message, clazz); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java deleted file mode 100644 index 74d8c5310..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatFactory.java +++ /dev/null @@ -1,164 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.Projection; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.format.ProjectableDecodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DeserializationFormatFactory; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; - -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - -import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; -import static org.apache.flink.formats.json.JsonFormatOptions.FAIL_ON_MISSING_FIELD; -import static org.apache.flink.formats.json.JsonFormatOptions.IGNORE_PARSE_ERRORS; -import static org.apache.flink.formats.json.JsonFormatOptions.MAP_NULL_KEY_LITERAL; -import static org.apache.flink.formats.json.JsonFormatOptions.MAP_NULL_KEY_MODE; -import static org.apache.flink.formats.json.JsonFormatOptions.TIMESTAMP_FORMAT; - -/** - * Table format factory for providing configured instances of JSON to RowData {@link - * SerializationSchema} and {@link DeserializationSchema}. - */ -@Internal -public class JsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "json"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - JsonFormatOptionsUtil.validateDecodingFormatOptions(formatOptions); - - final boolean failOnMissingField = formatOptions.get(FAIL_ON_MISSING_FIELD); - final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - TimestampFormat timestampOption = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - - return new ProjectableDecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, - DataType physicalDataType, - int[][] projections) { - final DataType producedDataType = - Projection.of(projections).project(physicalDataType); - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation rowDataTypeInfo = - context.createTypeInformation(producedDataType); - return new JsonRowDataDeserializationSchema( - rowType, - rowDataTypeInfo, - failOnMissingField, - ignoreParseErrors, - timestampOption); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - }; - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - JsonFormatOptionsUtil.validateEncodingFormatOptions(formatOptions); - - TimestampFormat timestampOption = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - JsonFormatOptions.MapNullKeyMode mapNullKeyMode = - JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); - String mapNullKeyLiteral = formatOptions.get(MAP_NULL_KEY_LITERAL); - - final boolean encodeDecimalAsPlainNumber = - formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - - return new EncodingFormat>() { - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new JsonRowDataSerializationSchema( - rowType, - timestampOption, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return Collections.emptySet(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(FAIL_ON_MISSING_FIELD); - options.add(IGNORE_PARSE_ERRORS); - options.add(TIMESTAMP_FORMAT); - options.add(MAP_NULL_KEY_MODE); - options.add(MAP_NULL_KEY_LITERAL); - options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - return options; - } - - @Override - public Set> forwardOptions() { - Set> options = new HashSet<>(); - options.add(TIMESTAMP_FORMAT); - options.add(MAP_NULL_KEY_MODE); - options.add(MAP_NULL_KEY_LITERAL); - options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - return options; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java deleted file mode 100644 index 74d567ab9..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptions.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; - -/** Options for the JSON format. */ -@PublicEvolving -public class JsonFormatOptions { - - public static final ConfigOption FAIL_ON_MISSING_FIELD = - ConfigOptions.key("fail-on-missing-field") - .booleanType() - .defaultValue(false) - .withDescription( - "Optional flag to specify whether to fail if a field is missing or not, false by default."); - - public static final ConfigOption IGNORE_PARSE_ERRORS = - ConfigOptions.key("ignore-parse-errors") - .booleanType() - .defaultValue(false) - .withDescription( - "Optional flag to skip fields and rows with parse errors instead of failing;\n" - + "fields are set to null in case of errors, false by default."); - - public static final ConfigOption MAP_NULL_KEY_MODE = - ConfigOptions.key("map-null-key.mode") - .stringType() - .defaultValue("FAIL") - .withDescription( - "Optional flag to control the handling mode when serializing null key for map data, FAIL by default." - + " Option DROP will drop null key entries for map data." - + " Option LITERAL will use 'map-null-key.literal' as key literal."); - - public static final ConfigOption MAP_NULL_KEY_LITERAL = - ConfigOptions.key("map-null-key.literal") - .stringType() - .defaultValue("null") - .withDescription( - "Optional flag to specify string literal for null keys when 'map-null-key.mode' is LITERAL, \"null\" by default."); - - public static final ConfigOption TIMESTAMP_FORMAT = - ConfigOptions.key("timestamp-format.standard") - .stringType() - .defaultValue("SQL") - .withDescription( - "Optional flag to specify timestamp format, SQL by default." - + " Option ISO-8601 will parse input timestamp in \"yyyy-MM-ddTHH:mm:ss.s{precision}\" format and output timestamp in the same format." - + " Option SQL will parse input timestamp in \"yyyy-MM-dd HH:mm:ss.s{precision}\" format and output timestamp in the same format."); - - public static final ConfigOption ENCODE_DECIMAL_AS_PLAIN_NUMBER = - ConfigOptions.key("encode.decimal-as-plain-number") - .booleanType() - .defaultValue(false) - .withDescription( - "Optional flag to specify whether to encode all decimals as plain numbers instead of possible scientific notations, false by default."); - - // -------------------------------------------------------------------------------------------- - // Enums - // -------------------------------------------------------------------------------------------- - - /** Handling mode for map data with null key. */ - public enum MapNullKeyMode { - FAIL, - DROP, - LITERAL - } - - private JsonFormatOptions() {} -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptionsUtil.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptionsUtil.java deleted file mode 100644 index 60a953d08..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonFormatOptionsUtil.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.ValidationException; - -import java.util.Arrays; -import java.util.HashSet; -import java.util.Objects; -import java.util.Set; -import java.util.stream.Collectors; - -import static org.apache.flink.formats.json.JsonFormatOptions.FAIL_ON_MISSING_FIELD; -import static org.apache.flink.formats.json.JsonFormatOptions.IGNORE_PARSE_ERRORS; -import static org.apache.flink.formats.json.JsonFormatOptions.MAP_NULL_KEY_MODE; -import static org.apache.flink.formats.json.JsonFormatOptions.TIMESTAMP_FORMAT; - -/** Utilities for {@link JsonFormatOptions}. */ -@Internal -public class JsonFormatOptionsUtil { - - // -------------------------------------------------------------------------------------------- - // Option enumerations - // -------------------------------------------------------------------------------------------- - - public static final String SQL = "SQL"; - public static final String ISO_8601 = "ISO-8601"; - - public static final Set TIMESTAMP_FORMAT_ENUM = - new HashSet<>(Arrays.asList(SQL, ISO_8601)); - - // The handling mode of null key for map data - public static final String JSON_MAP_NULL_KEY_MODE_FAIL = "FAIL"; - public static final String JSON_MAP_NULL_KEY_MODE_DROP = "DROP"; - public static final String JSON_MAP_NULL_KEY_MODE_LITERAL = "LITERAL"; - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - public static TimestampFormat getTimestampFormat(ReadableConfig config) { - String timestampFormat = config.get(TIMESTAMP_FORMAT); - switch (timestampFormat) { - case SQL: - return TimestampFormat.SQL; - case ISO_8601: - return TimestampFormat.ISO_8601; - default: - throw new TableException( - String.format( - "Unsupported timestamp format '%s'. Validator should have checked that.", - timestampFormat)); - } - } - - /** - * Creates handling mode for null key map data. - * - *

See {@link #JSON_MAP_NULL_KEY_MODE_FAIL}, {@link #JSON_MAP_NULL_KEY_MODE_DROP}, and {@link - * #JSON_MAP_NULL_KEY_MODE_LITERAL} for more information. - */ - public static JsonFormatOptions.MapNullKeyMode getMapNullKeyMode(ReadableConfig config) { - String mapNullKeyMode = config.get(MAP_NULL_KEY_MODE); - switch (mapNullKeyMode.toUpperCase()) { - case JSON_MAP_NULL_KEY_MODE_FAIL: - return JsonFormatOptions.MapNullKeyMode.FAIL; - case JSON_MAP_NULL_KEY_MODE_DROP: - return JsonFormatOptions.MapNullKeyMode.DROP; - case JSON_MAP_NULL_KEY_MODE_LITERAL: - return JsonFormatOptions.MapNullKeyMode.LITERAL; - default: - throw new TableException( - String.format( - "Unsupported map null key handling mode '%s'. Validator should have checked that.", - mapNullKeyMode)); - } - } - - // -------------------------------------------------------------------------------------------- - // Validation - // -------------------------------------------------------------------------------------------- - - /** Validator for json decoding format. */ - public static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - boolean failOnMissingField = tableOptions.get(FAIL_ON_MISSING_FIELD); - boolean ignoreParseErrors = tableOptions.get(IGNORE_PARSE_ERRORS); - if (ignoreParseErrors && failOnMissingField) { - throw new ValidationException( - FAIL_ON_MISSING_FIELD.key() - + " and " - + IGNORE_PARSE_ERRORS.key() - + " shouldn't both be true."); - } - validateTimestampFormat(tableOptions); - } - - /** Validator for json encoding format. */ - public static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - // validator for {@link MAP_NULL_KEY_MODE} - Set nullKeyModes = - Arrays.stream(JsonFormatOptions.MapNullKeyMode.values()) - .map(Objects::toString) - .collect(Collectors.toSet()); - if (!nullKeyModes.contains(tableOptions.get(MAP_NULL_KEY_MODE).toUpperCase())) { - throw new ValidationException( - String.format( - "Unsupported value '%s' for option %s. Supported values are %s.", - tableOptions.get(MAP_NULL_KEY_MODE), - MAP_NULL_KEY_MODE.key(), - nullKeyModes)); - } - validateTimestampFormat(tableOptions); - } - - /** Validates timestamp format which value should be SQL or ISO-8601. */ - static void validateTimestampFormat(ReadableConfig tableOptions) { - String timestampFormat = tableOptions.get(TIMESTAMP_FORMAT); - if (!TIMESTAMP_FORMAT_ENUM.contains(timestampFormat)) { - throw new ValidationException( - String.format( - "Unsupported value '%s' for %s. Supported values are [SQL, ISO-8601].", - timestampFormat, TIMESTAMP_FORMAT.key())); - } - } - - private JsonFormatOptionsUtil() {} -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java deleted file mode 100644 index 928a6f1e5..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; - -/** - * DeserializationSchema that deserializes a JSON String into an ObjectNode. - * - *

Fields can be accessed by calling objectNode.get(<name>).as(<type>) - * - * @deprecated Use {@code new JsonDeserializationSchema(ObjectNode.class)} instead - */ -@Deprecated -public class JsonNodeDeserializationSchema extends JsonDeserializationSchema { - - private static final long serialVersionUID = 2L; - - public JsonNodeDeserializationSchema() { - super(ObjectNode.class); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java deleted file mode 100644 index 9a57bac20..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java +++ /dev/null @@ -1,163 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.json.JsonReadFeature; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.util.Objects; - -import static java.lang.String.format; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Deserialization schema from JSON to Flink Table/SQL internal data structure {@link RowData}. - * - *

Deserializes a byte[] message as a JSON object and reads the specified fields. - * - *

Failures during deserialization are forwarded as wrapped IOExceptions. - */ -@Internal -public class JsonRowDataDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - - /** Flag indicating whether to fail if a field is missing. */ - private final boolean failOnMissingField; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - /** TypeInformation of the produced {@link RowData}. */ - private final TypeInformation resultTypeInfo; - - /** - * Runtime converter that converts {@link JsonNode}s into objects of Flink SQL internal data - * structures. - */ - private final JsonToRowDataConverters.JsonToRowDataConverter runtimeConverter; - - /** Object mapper for parsing the JSON. */ - private transient ObjectMapper objectMapper; - - /** Timestamp format specification which is used to parse timestamp. */ - private final TimestampFormat timestampFormat; - - private final boolean hasDecimalType; - - public JsonRowDataDeserializationSchema( - RowType rowType, - TypeInformation resultTypeInfo, - boolean failOnMissingField, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - if (ignoreParseErrors && failOnMissingField) { - throw new IllegalArgumentException( - "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."); - } - this.resultTypeInfo = checkNotNull(resultTypeInfo); - this.failOnMissingField = failOnMissingField; - this.ignoreParseErrors = ignoreParseErrors; - this.runtimeConverter = - new JsonToRowDataConverters(failOnMissingField, ignoreParseErrors, timestampFormat) - .createConverter(checkNotNull(rowType)); - this.timestampFormat = timestampFormat; - this.hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t instanceof DecimalType); - } - - @Override - public void open(InitializationContext context) throws Exception { - objectMapper = - JacksonMapperFactory.createObjectMapper() - .configure( - JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(), - true); - if (hasDecimalType) { - objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); - } - } - - @Override - public RowData deserialize(@Nullable byte[] message) throws IOException { - if (message == null) { - return null; - } - try { - return convertToRowData(deserializeToJsonNode(message)); - } catch (Throwable t) { - if (ignoreParseErrors) { - return null; - } - throw new IOException( - format("Failed to deserialize JSON '%s'.", new String(message)), t); - } - } - - public JsonNode deserializeToJsonNode(byte[] message) throws IOException { - return objectMapper.readTree(message); - } - - public RowData convertToRowData(JsonNode message) { - return (RowData) runtimeConverter.convert(message); - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return resultTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - JsonRowDataDeserializationSchema that = (JsonRowDataDeserializationSchema) o; - return failOnMissingField == that.failOnMissingField - && ignoreParseErrors == that.ignoreParseErrors - && resultTypeInfo.equals(that.resultTypeInfo) - && timestampFormat.equals(that.timestampFormat); - } - - @Override - public int hashCode() { - return Objects.hash(failOnMissingField, ignoreParseErrors, resultTypeInfo, timestampFormat); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java deleted file mode 100644 index c8b7f73b6..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDataSerializationSchema.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; - -import java.util.Objects; - -/** - * Serialization schema that serializes an object of Flink internal data structure into a JSON - * bytes. - * - *

Serializes the input Flink object into a JSON string and converts it into byte[]. - * - *

Result byte[] messages can be deserialized using {@link - * JsonRowDataDeserializationSchema}. - */ -@Internal -public class JsonRowDataSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = 1L; - - /** RowType to generate the runtime converter. */ - private final RowType rowType; - - /** The converter that converts internal data formats to JsonNode. */ - private final RowDataToJsonConverters.RowDataToJsonConverter runtimeConverter; - - /** Object mapper that is used to create output JSON objects. */ - private transient ObjectMapper mapper; - - /** Reusable object node. */ - private transient ObjectNode node; - - /** Timestamp format specification which is used to parse timestamp. */ - private final TimestampFormat timestampFormat; - - /** The handling mode when serializing null keys for map data. */ - private final JsonFormatOptions.MapNullKeyMode mapNullKeyMode; - - /** The string literal when handling mode for map null key LITERAL. */ - private final String mapNullKeyLiteral; - - /** Flag indicating whether to serialize all decimals as plain numbers. */ - private final boolean encodeDecimalAsPlainNumber; - - public JsonRowDataSerializationSchema( - RowType rowType, - TimestampFormat timestampFormat, - JsonFormatOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral, - boolean encodeDecimalAsPlainNumber) { - this.rowType = rowType; - this.timestampFormat = timestampFormat; - this.mapNullKeyMode = mapNullKeyMode; - this.mapNullKeyLiteral = mapNullKeyLiteral; - this.encodeDecimalAsPlainNumber = encodeDecimalAsPlainNumber; - this.runtimeConverter = - new RowDataToJsonConverters(timestampFormat, mapNullKeyMode, mapNullKeyLiteral) - .createConverter(rowType); - } - - @Override - public void open(InitializationContext context) throws Exception { - mapper = - JacksonMapperFactory.createObjectMapper() - .configure( - JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, - encodeDecimalAsPlainNumber); - } - - @Override - public byte[] serialize(RowData row) { - if (node == null) { - node = mapper.createObjectNode(); - } - - try { - runtimeConverter.convert(mapper, node, row); - return mapper.writeValueAsBytes(node); - } catch (Throwable t) { - throw new RuntimeException(String.format("Could not serialize row '%s'.", row), t); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - JsonRowDataSerializationSchema that = (JsonRowDataSerializationSchema) o; - return rowType.equals(that.rowType) - && timestampFormat.equals(that.timestampFormat) - && mapNullKeyMode.equals(that.mapNullKeyMode) - && mapNullKeyLiteral.equals(that.mapNullKeyLiteral) - && encodeDecimalAsPlainNumber == that.encodeDecimalAsPlainNumber; - } - - @Override - public int hashCode() { - return Objects.hash( - rowType, - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java deleted file mode 100644 index dd4a9bb9f..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ /dev/null @@ -1,587 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; -import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.typeutils.MapTypeInfo; -import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; -import org.apache.flink.types.Row; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; - -import java.io.IOException; -import java.io.Serializable; -import java.lang.reflect.Array; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.ZoneOffset; -import java.time.temporal.TemporalAccessor; -import java.time.temporal.TemporalQueries; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.stream.Collectors; - -import static java.lang.String.format; -import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIME_FORMAT; -import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL; -import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Deserialization schema from JSON to Flink types. - * - *

Deserializes a byte[] message as a JSON object and reads the specified fields. - * - *

Failures during deserialization are forwarded as wrapped IOExceptions. - * - * @deprecated The format was developed for the Table API users and will not be maintained for - * DataStream API users anymore. Either use Table API or switch to Data Stream, defining your - * own {@link DeserializationSchema}. - */ -@PublicEvolving -@Deprecated -public class JsonRowDeserializationSchema implements DeserializationSchema { - - private static final long serialVersionUID = -228294330688809195L; - - /** Type information describing the result type. */ - private final RowTypeInfo typeInfo; - - private boolean failOnMissingField; - - private final boolean hasDecimalType; - - /** Object mapper for parsing the JSON. */ - private transient ObjectMapper objectMapper; - - private DeserializationRuntimeConverter runtimeConverter; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - private JsonRowDeserializationSchema( - TypeInformation typeInfo, boolean failOnMissingField, boolean ignoreParseErrors) { - checkNotNull(typeInfo, "Type information"); - checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); - if (ignoreParseErrors && failOnMissingField) { - throw new IllegalArgumentException( - "JSON format doesn't support failOnMissingField and ignoreParseErrors are both true."); - } - this.typeInfo = (RowTypeInfo) typeInfo; - this.failOnMissingField = failOnMissingField; - this.runtimeConverter = createConverter(this.typeInfo); - this.ignoreParseErrors = ignoreParseErrors; - RowType rowType = (RowType) fromLegacyInfoToDataType(this.typeInfo).getLogicalType(); - hasDecimalType = LogicalTypeChecks.hasNested(rowType, t -> t.getTypeRoot().equals(DECIMAL)); - } - - @Override - public void open(InitializationContext context) throws Exception { - objectMapper = JacksonMapperFactory.createObjectMapper(); - if (hasDecimalType) { - objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); - } - } - - /** @deprecated Use the provided {@link Builder} instead. */ - @Deprecated - public JsonRowDeserializationSchema(TypeInformation typeInfo) { - this(typeInfo, false, false); - } - - /** @deprecated Use the provided {@link Builder} instead. */ - @Deprecated - public JsonRowDeserializationSchema(String jsonSchema) { - this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)), false, false); - } - - /** @deprecated Use the provided {@link Builder} instead. */ - @Deprecated - public void setFailOnMissingField(boolean failOnMissingField) { - // TODO make this class immutable once we drop this method - this.failOnMissingField = failOnMissingField; - this.runtimeConverter = createConverter(this.typeInfo); - } - - @Override - public Row deserialize(byte[] message) throws IOException { - try { - final JsonNode root = objectMapper.readTree(message); - return (Row) runtimeConverter.convert(objectMapper, root); - } catch (Throwable t) { - if (ignoreParseErrors) { - return null; - } - throw new IOException( - format("Failed to deserialize JSON '%s'.", new String(message)), t); - } - } - - @Override - public boolean isEndOfStream(Row nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return typeInfo; - } - - /** Builder for {@link JsonRowDeserializationSchema}. */ - public static class Builder { - - private final RowTypeInfo typeInfo; - private boolean failOnMissingField = false; - private boolean ignoreParseErrors = false; - - /** - * Creates a JSON deserialization schema for the given type information. - * - * @param typeInfo Type information describing the result type. The field names of {@link - * Row} are used to parse the JSON properties. - */ - public Builder(TypeInformation typeInfo) { - checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); - this.typeInfo = (RowTypeInfo) typeInfo; - } - - /** - * Creates a JSON deserialization schema for the given JSON schema. - * - * @param jsonSchema JSON schema describing the result type - * @see http://json-schema.org/ - */ - public Builder(String jsonSchema) { - this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema))); - } - - /** - * Configures schema to fail if a JSON field is missing. - * - *

By default, a missing field is ignored and the field is set to null. - */ - public Builder failOnMissingField() { - this.failOnMissingField = true; - return this; - } - - /** - * Configures schema to fail when parsing json failed. - * - *

By default, an exception will be thrown when parsing json fails. - */ - public Builder ignoreParseErrors() { - this.ignoreParseErrors = true; - return this; - } - - public JsonRowDeserializationSchema build() { - return new JsonRowDeserializationSchema( - typeInfo, failOnMissingField, ignoreParseErrors); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - final JsonRowDeserializationSchema that = (JsonRowDeserializationSchema) o; - return Objects.equals(typeInfo, that.typeInfo) - && Objects.equals(failOnMissingField, that.failOnMissingField) - && Objects.equals(ignoreParseErrors, that.ignoreParseErrors); - } - - @Override - public int hashCode() { - return Objects.hash(typeInfo, failOnMissingField, ignoreParseErrors); - } - - /* - Runtime converter - */ - - /** Runtime converter that maps between {@link JsonNode}s and Java objects. */ - @FunctionalInterface - private interface DeserializationRuntimeConverter extends Serializable { - Object convert(ObjectMapper mapper, JsonNode jsonNode); - } - - private DeserializationRuntimeConverter createConverter(TypeInformation typeInfo) { - DeserializationRuntimeConverter baseConverter = - createConverterForSimpleType(typeInfo) - .orElseGet( - () -> - createContainerConverter(typeInfo) - .orElseGet( - () -> - createFallbackConverter( - typeInfo.getTypeClass()))); - return wrapIntoNullableConverter(baseConverter); - } - - private DeserializationRuntimeConverter wrapIntoNullableConverter( - DeserializationRuntimeConverter converter) { - return (mapper, jsonNode) -> { - if (jsonNode.isNull()) { - return null; - } - try { - return converter.convert(mapper, jsonNode); - } catch (Throwable t) { - if (!ignoreParseErrors) { - throw t; - } - return null; - } - }; - } - - private Optional createContainerConverter( - TypeInformation typeInfo) { - if (typeInfo instanceof RowTypeInfo) { - return Optional.of(createRowConverter((RowTypeInfo) typeInfo)); - } else if (typeInfo instanceof ObjectArrayTypeInfo) { - return Optional.of( - createObjectArrayConverter( - ((ObjectArrayTypeInfo) typeInfo).getComponentInfo())); - } else if (typeInfo instanceof BasicArrayTypeInfo) { - return Optional.of( - createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo())); - } else if (isPrimitiveByteArray(typeInfo)) { - return Optional.of(createByteArrayConverter()); - } else if (typeInfo instanceof MapTypeInfo) { - MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo; - return Optional.of( - createMapConverter( - mapTypeInfo.getKeyTypeInfo(), mapTypeInfo.getValueTypeInfo())); - } else { - return Optional.empty(); - } - } - - private DeserializationRuntimeConverter createMapConverter( - TypeInformation keyType, TypeInformation valueType) { - DeserializationRuntimeConverter valueConverter = createConverter(valueType); - DeserializationRuntimeConverter keyConverter = createConverter(keyType); - return (mapper, jsonNode) -> { - Iterator> fields = jsonNode.fields(); - Map result = new HashMap<>(); - while (fields.hasNext()) { - Map.Entry entry = fields.next(); - Object key = keyConverter.convert(mapper, TextNode.valueOf(entry.getKey())); - Object value = valueConverter.convert(mapper, entry.getValue()); - result.put(key, value); - } - return result; - }; - } - - private DeserializationRuntimeConverter createByteArrayConverter() { - return (mapper, jsonNode) -> { - try { - return jsonNode.binaryValue(); - } catch (IOException e) { - throw new JsonParseException("Unable to deserialize byte array.", e); - } - }; - } - - private boolean isPrimitiveByteArray(TypeInformation typeInfo) { - return typeInfo instanceof PrimitiveArrayTypeInfo - && ((PrimitiveArrayTypeInfo) typeInfo).getComponentType() == Types.BYTE; - } - - private DeserializationRuntimeConverter createObjectArrayConverter( - TypeInformation elementTypeInfo) { - DeserializationRuntimeConverter elementConverter = createConverter(elementTypeInfo); - return assembleArrayConverter(elementTypeInfo, elementConverter); - } - - private DeserializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) { - List fieldConverters = - Arrays.stream(typeInfo.getFieldTypes()) - .map(this::createConverter) - .collect(Collectors.toList()); - - return assembleRowConverter(typeInfo.getFieldNames(), fieldConverters); - } - - private DeserializationRuntimeConverter createFallbackConverter(Class valueType) { - return (mapper, jsonNode) -> { - // for types that were specified without JSON schema - // e.g. POJOs - try { - return mapper.treeToValue(jsonNode, valueType); - } catch (JsonProcessingException e) { - throw new JsonParseException(format("Could not convert node: %s", jsonNode), e); - } - }; - } - - private Optional createConverterForSimpleType( - TypeInformation simpleTypeInfo) { - if (simpleTypeInfo == Types.VOID) { - return Optional.of((mapper, jsonNode) -> null); - } else if (simpleTypeInfo == Types.BOOLEAN) { - return Optional.of(this::convertToBoolean); - } else if (simpleTypeInfo == Types.STRING) { - return Optional.of(this::convertToString); - } else if (simpleTypeInfo == Types.INT) { - return Optional.of(this::convertToInt); - } else if (simpleTypeInfo == Types.LONG) { - return Optional.of(this::convertToLong); - } else if (simpleTypeInfo == Types.DOUBLE) { - return Optional.of(this::convertToDouble); - } else if (simpleTypeInfo == Types.FLOAT) { - return Optional.of((mapper, jsonNode) -> Float.parseFloat(jsonNode.asText().trim())); - } else if (simpleTypeInfo == Types.SHORT) { - return Optional.of((mapper, jsonNode) -> Short.parseShort(jsonNode.asText().trim())); - } else if (simpleTypeInfo == Types.BYTE) { - return Optional.of((mapper, jsonNode) -> Byte.parseByte(jsonNode.asText().trim())); - } else if (simpleTypeInfo == Types.BIG_DEC) { - return Optional.of(this::convertToBigDecimal); - } else if (simpleTypeInfo == Types.BIG_INT) { - return Optional.of(this::convertToBigInteger); - } else if (simpleTypeInfo == Types.SQL_DATE) { - return Optional.of(this::convertToDate); - } else if (simpleTypeInfo == Types.SQL_TIME) { - return Optional.of(this::convertToTime); - } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) { - return Optional.of(this::convertToTimestamp); - } else if (simpleTypeInfo == Types.LOCAL_DATE) { - return Optional.of(this::convertToLocalDate); - } else if (simpleTypeInfo == Types.LOCAL_TIME) { - return Optional.of(this::convertToLocalTime); - } else if (simpleTypeInfo == Types.LOCAL_DATE_TIME) { - return Optional.of(this::convertToLocalDateTime); - } else { - return Optional.empty(); - } - } - - private String convertToString(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.isContainerNode()) { - return jsonNode.toString(); - } else { - return jsonNode.asText(); - } - } - - private boolean convertToBoolean(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.isBoolean()) { - // avoid redundant toString and parseBoolean, for better performance - return jsonNode.asBoolean(); - } else { - return Boolean.parseBoolean(jsonNode.asText().trim()); - } - } - - private int convertToInt(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.canConvertToInt()) { - // avoid redundant toString and parseInt, for better performance - return jsonNode.asInt(); - } else { - return Integer.parseInt(jsonNode.asText().trim()); - } - } - - private long convertToLong(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.canConvertToLong()) { - // avoid redundant toString and parseLong, for better performance - return jsonNode.asLong(); - } else { - return Long.parseLong(jsonNode.asText().trim()); - } - } - - private double convertToDouble(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.isDouble()) { - // avoid redundant toString and parseDouble, for better performance - return jsonNode.asDouble(); - } else { - return Double.parseDouble(jsonNode.asText().trim()); - } - } - - private BigDecimal convertToBigDecimal(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.isBigDecimal()) { - // avoid redundant toString and toDecimal, for better performance - return jsonNode.decimalValue(); - } else { - return new BigDecimal(jsonNode.asText().trim()); - } - } - - private BigInteger convertToBigInteger(ObjectMapper mapper, JsonNode jsonNode) { - if (jsonNode.isBigInteger()) { - // avoid redundant toString and toBigInteger, for better performance - return jsonNode.bigIntegerValue(); - } else { - return new BigInteger(jsonNode.asText().trim()); - } - } - - private LocalDate convertToLocalDate(ObjectMapper mapper, JsonNode jsonNode) { - return ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); - } - - private Date convertToDate(ObjectMapper mapper, JsonNode jsonNode) { - return Date.valueOf(convertToLocalDate(mapper, jsonNode)); - } - - private LocalDateTime convertToLocalDateTime(ObjectMapper mapper, JsonNode jsonNode) { - // according to RFC 3339 every date-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - TemporalAccessor parsedTimestamp = RFC3339_TIMESTAMP_FORMAT.parse(jsonNode.asText()); - - ZoneOffset zoneOffset = parsedTimestamp.query(TemporalQueries.offset()); - - if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0) { - throw new IllegalStateException( - "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " - + "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); - } - - LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); - LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); - - return LocalDateTime.of(localDate, localTime); - } - - private Timestamp convertToTimestamp(ObjectMapper mapper, JsonNode jsonNode) { - return Timestamp.valueOf(convertToLocalDateTime(mapper, jsonNode)); - } - - private LocalTime convertToLocalTime(ObjectMapper mapper, JsonNode jsonNode) { - // according to RFC 3339 every full-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - - TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText()); - - ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset()); - LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); - - if (zoneOffset != null && zoneOffset.getTotalSeconds() != 0 || localTime.getNano() != 0) { - throw new IllegalStateException( - "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet."); - } - - return localTime; - } - - private Time convertToTime(ObjectMapper mapper, JsonNode jsonNode) { - return Time.valueOf(convertToLocalTime(mapper, jsonNode)); - } - - private DeserializationRuntimeConverter assembleRowConverter( - String[] fieldNames, List fieldConverters) { - return (mapper, jsonNode) -> { - ObjectNode node = (ObjectNode) jsonNode; - int arity = fieldNames.length; - Row row = new Row(arity); - for (int i = 0; i < arity; i++) { - String fieldName = fieldNames[i]; - JsonNode field = node.get(fieldName); - Object convertField = - convertField(mapper, fieldConverters.get(i), fieldName, field); - row.setField(i, convertField); - } - - return row; - }; - } - - private Object convertField( - ObjectMapper mapper, - DeserializationRuntimeConverter fieldConverter, - String fieldName, - JsonNode field) { - if (field == null) { - if (failOnMissingField) { - throw new IllegalStateException( - "Could not find field with name '" + fieldName + "'."); - } else { - return null; - } - } else { - return fieldConverter.convert(mapper, field); - } - } - - private DeserializationRuntimeConverter assembleArrayConverter( - TypeInformation elementType, DeserializationRuntimeConverter elementConverter) { - - final Class elementClass = elementType.getTypeClass(); - - return (mapper, jsonNode) -> { - final ArrayNode node = (ArrayNode) jsonNode; - final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); - for (int i = 0; i < node.size(); i++) { - final JsonNode innerNode = node.get(i); - array[i] = elementConverter.convert(mapper, innerNode); - } - - return array; - }; - } - - /** Exception which refers to parse errors in converters. */ - private static final class JsonParseException extends RuntimeException { - private static final long serialVersionUID = 1L; - - public JsonParseException(String message, Throwable cause) { - super(message, cause); - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java deleted file mode 100644 index d7761851e..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java +++ /dev/null @@ -1,382 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; - -/** - * Converts a JSON schema into Flink's type information. It uses {@link Row} for representing - * objects and tuple arrays. - * - *

Note: This converter implements just a subset of the JSON schema specification. Union types - * (as well as "allOf", "anyOf", "not") are not supported yet. Simple references that link to a - * common definition in the document are supported. "oneOf" and arrays of types are only supported - * for specifying nullability. - * - *

This converter has been developed for JSON Schema draft-07 but also includes keywords of older - * drafts to be as compatible as possible. - */ -public final class JsonRowSchemaConverter { - - private JsonRowSchemaConverter() { - // private - } - - // see https://spacetelescope.github.io/understanding-json-schema/UnderstandingJSONSchema.pdf - private static final String PROPERTIES = "properties"; - private static final String ADDITIONAL_PROPERTIES = "additionalProperties"; - private static final String TYPE = "type"; - private static final String FORMAT = "format"; - private static final String CONTENT_ENCODING = "contentEncoding"; - private static final String ITEMS = "items"; - private static final String ADDITIONAL_ITEMS = "additionalItems"; - private static final String REF = "$ref"; - private static final String ALL_OF = "allOf"; - private static final String ANY_OF = "anyOf"; - private static final String NOT = "not"; - private static final String ONE_OF = "oneOf"; - - // from https://tools.ietf.org/html/draft-zyp-json-schema-03#page-14 - private static final String DISALLOW = "disallow"; - private static final String EXTENDS = "extends"; - - private static final String TYPE_NULL = "null"; - private static final String TYPE_BOOLEAN = "boolean"; - private static final String TYPE_OBJECT = "object"; - private static final String TYPE_ARRAY = "array"; - private static final String TYPE_NUMBER = "number"; - private static final String TYPE_INTEGER = "integer"; - private static final String TYPE_STRING = "string"; - - private static final String FORMAT_DATE = "date"; - private static final String FORMAT_TIME = "time"; - private static final String FORMAT_DATE_TIME = "date-time"; - - private static final String CONTENT_ENCODING_BASE64 = "base64"; - - /** - * Converts a JSON schema into Flink's type information. Throws an exception if the schema - * cannot converted because of loss of precision or too flexible schema. - * - *

The converter can resolve simple schema references to solve those cases where entities are - * defined at the beginning and then used throughout a document. - */ - @SuppressWarnings("unchecked") - public static TypeInformation convert(String jsonSchema) { - Preconditions.checkNotNull(jsonSchema, "JSON schema"); - final ObjectMapper mapper = JacksonMapperFactory.createObjectMapper(); - mapper.getFactory() - .enable(JsonParser.Feature.ALLOW_COMMENTS) - .enable(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES) - .enable(JsonParser.Feature.ALLOW_SINGLE_QUOTES); - final JsonNode node; - try { - node = mapper.readTree(jsonSchema); - } catch (IOException e) { - throw new IllegalArgumentException("Invalid JSON schema.", e); - } - return (TypeInformation) convertType("", node, node); - } - - private static TypeInformation convertType(String location, JsonNode node, JsonNode root) { - // we use a set here to unify types (e.g. types that just add metadata such as 'multipleOf') - final Set> typeSet = new HashSet<>(); - - // search for ref - final Optional ref; - if (node.has(REF) && node.get(REF).isTextual()) { - // try a simple ref resolver to solve those cases where entities are defined at - // the beginning and then used throughout a document - ref = Optional.of(resolveReference(node.get(REF).asText(), node, root)); - } else { - ref = Optional.empty(); - } - - // use TYPE of this node - if (node.has(TYPE)) { - final JsonNode typeNode = node.get(TYPE); - - List types = new ArrayList<>(); - // array of types - if (typeNode.isArray()) { - final Iterator elements = typeNode.elements(); - while (elements.hasNext()) { - types.add(elements.next().asText()); - } - } - // single type - else if (typeNode.isTextual()) { - types.add(typeNode.asText()); - } - - for (String type : types) { - // set field type - switch (type) { - case TYPE_NULL: - typeSet.add(Types.VOID); - break; - case TYPE_BOOLEAN: - typeSet.add(Types.BOOLEAN); - break; - case TYPE_STRING: - if (node.has(FORMAT)) { - typeSet.add(convertStringFormat(location, node.get(FORMAT))); - } else if (node.has(CONTENT_ENCODING)) { - typeSet.add( - convertStringEncoding(location, node.get(CONTENT_ENCODING))); - } else { - typeSet.add(Types.STRING); - } - break; - case TYPE_NUMBER: - typeSet.add(Types.BIG_DEC); - break; - case TYPE_INTEGER: - // use BigDecimal for easier interoperability - // without affecting the correctness of the result - typeSet.add(Types.BIG_DEC); - break; - case TYPE_OBJECT: - typeSet.add(convertObject(location, node, root)); - break; - case TYPE_ARRAY: - typeSet.add(convertArray(location, node, root)); - break; - default: - throw new IllegalArgumentException( - "Unsupported type '" - + node.get(TYPE).asText() - + "' in node: " - + location); - } - } - } - // use TYPE of reference as fallback if present - else { - ref.filter(r -> r.has(TYPE)) - .ifPresent(r -> typeSet.add(convertType(node.get(REF).asText(), r, root))); - } - - // simple interpretation of ONE_OF for supporting "object or null" - if (node.has(ONE_OF) && node.get(ONE_OF).isArray()) { - final TypeInformation[] types = - convertTypes(location + '/' + ONE_OF, node.get(ONE_OF), root); - typeSet.addAll(Arrays.asList(types)); - } - // use ONE_OF of reference as fallback - else if (ref.isPresent() && ref.get().has(ONE_OF) && ref.get().get(ONE_OF).isArray()) { - final TypeInformation[] types = - convertTypes( - node.get(REF).asText() + '/' + ONE_OF, ref.get().get(ONE_OF), root); - typeSet.addAll(Arrays.asList(types)); - } - - // validate no union types or extending - if (node.has(ALL_OF) - || node.has(ANY_OF) - || node.has(NOT) - || node.has(EXTENDS) - || node.has(DISALLOW)) { - throw new IllegalArgumentException( - "Union types are such as '" - + ALL_OF - + "', '" - + ANY_OF - + "' etc. " - + "and extending are not supported yet."); - } - - // only a type (with null) is supported yet - final List> types = new ArrayList<>(typeSet); - if (types.size() == 0) { - throw new IllegalArgumentException("No type could be found in node:" + location); - } else if (types.size() > 2 || (types.size() == 2 && !types.contains(Types.VOID))) { - throw new IllegalArgumentException( - "Union types with more than just a null type are not supported yet."); - } - - // return the first non-void type or void - if (types.size() == 2 && types.get(0) == Types.VOID) { - return types.get(1); - } else { - return types.get(0); - } - } - - private static TypeInformation convertObject( - String location, JsonNode node, JsonNode root) { - // validate properties - if (!node.has(PROPERTIES)) { - return Types.ROW(); - } - if (!node.isObject()) { - throw new IllegalArgumentException( - "Invalid '" + PROPERTIES + "' property for object type in node: " + location); - } - final JsonNode props = node.get(PROPERTIES); - final String[] names = new String[props.size()]; - final TypeInformation[] types = new TypeInformation[props.size()]; - - final Iterator> fieldIter = props.fields(); - int i = 0; - while (fieldIter.hasNext()) { - final Map.Entry subNode = fieldIter.next(); - - // set field name - names[i] = subNode.getKey(); - - // set type - types[i] = convertType(location + '/' + subNode.getKey(), subNode.getValue(), root); - - i++; - } - - // validate that object does not contain additional properties - if (node.has(ADDITIONAL_PROPERTIES) - && node.get(ADDITIONAL_PROPERTIES).isBoolean() - && node.get(ADDITIONAL_PROPERTIES).asBoolean()) { - throw new IllegalArgumentException( - "An object must not allow additional properties in node: " + location); - } - - return Types.ROW_NAMED(names, types); - } - - private static TypeInformation convertArray(String location, JsonNode node, JsonNode root) { - // validate items - if (!node.has(ITEMS)) { - throw new IllegalArgumentException( - "Arrays must specify an '" + ITEMS + "' property in node: " + location); - } - final JsonNode items = node.get(ITEMS); - - // list (translated to object array) - if (items.isObject()) { - final TypeInformation elementType = convertType(location + '/' + ITEMS, items, root); - // result type might either be ObjectArrayTypeInfo or BasicArrayTypeInfo for Strings - return Types.OBJECT_ARRAY(elementType); - } - // tuple (translated to row) - else if (items.isArray()) { - final TypeInformation[] types = convertTypes(location + '/' + ITEMS, items, root); - - // validate that array does not contain additional items - if (node.has(ADDITIONAL_ITEMS) - && node.get(ADDITIONAL_ITEMS).isBoolean() - && node.get(ADDITIONAL_ITEMS).asBoolean()) { - throw new IllegalArgumentException( - "An array tuple must not allow additional items in node: " + location); - } - - return Types.ROW(types); - } - throw new IllegalArgumentException( - "Invalid type for '" + ITEMS + "' property in node: " + location); - } - - private static TypeInformation convertStringFormat(String location, JsonNode node) { - if (!node.isTextual()) { - throw new IllegalArgumentException( - "Invalid '" + FORMAT + "' property in node: " + location); - } - - switch (node.asText()) { - case FORMAT_DATE: - return Types.SQL_DATE; - case FORMAT_TIME: - return Types.SQL_TIME; - case FORMAT_DATE_TIME: - return Types.SQL_TIMESTAMP; - default: - return Types.STRING; // unlikely that we will support other formats in the future - } - } - - private static TypeInformation convertStringEncoding(String location, JsonNode node) { - if (!node.isTextual()) { - throw new IllegalArgumentException( - "Invalid '" + CONTENT_ENCODING + "' property in node: " + location); - } - - // "If the instance value is a string, this property defines that the string SHOULD - // be interpreted as binary data and decoded using the encoding named by this property." - - switch (node.asText()) { - case CONTENT_ENCODING_BASE64: - return Types.PRIMITIVE_ARRAY(Types.BYTE); - default: - // we fail hard here: - // this gives us the chance to support more encodings in the future without problems - // of backwards compatibility - throw new IllegalArgumentException( - "Invalid encoding '" + node.asText() + "' in node: " + location); - } - } - - private static JsonNode resolveReference(String ref, JsonNode origin, JsonNode root) { - if (!ref.startsWith("#")) { - throw new IllegalArgumentException( - "Only JSON schemes with simple references " - + "(one indirection in the same document) are supported yet. But was: " - + ref); - } - final String path = ref.substring(1); - final JsonNode foundNode = root.at(path); - if (foundNode.isMissingNode()) { - throw new IllegalArgumentException("Could not find reference: " + ref); - } - // prevent obvious cyclic references - if (foundNode == origin) { - throw new IllegalArgumentException("Cyclic references are not supported:" + ref); - } - return foundNode; - } - - private static TypeInformation[] convertTypes( - String location, JsonNode arrayNode, JsonNode root) { - final TypeInformation[] types = new TypeInformation[arrayNode.size()]; - final Iterator elements = arrayNode.elements(); - int i = 0; - while (elements.hasNext()) { - final TypeInformation elementType = - convertType(location + '[' + i + ']', elements.next(), root); - types[i] = elementType; - i += 1; - } - return types; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java deleted file mode 100644 index f185d211b..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ /dev/null @@ -1,436 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; -import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.WrappingRuntimeException; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeFactory; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; - -import java.io.Serializable; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.Arrays; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import java.util.stream.Collectors; - -import static java.lang.String.format; -import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.common.TimeFormats.RFC3339_TIME_FORMAT; -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Serialization schema that serializes an object of Flink types into a JSON bytes. - * - *

Serializes the input Flink object into a JSON string and converts it into byte[]. - * - *

Result byte[] messages can be deserialized using {@link - * JsonRowDeserializationSchema}. - * - * @deprecated The format was developed for the Table API users and will not be maintained for - * DataStream API users anymore. Either use Table API or switch to Data Stream, defining your - * own {@link SerializationSchema}. - */ -@PublicEvolving -@Deprecated -public class JsonRowSerializationSchema implements SerializationSchema { - - private static final long serialVersionUID = -2885556750743978636L; - - /** Type information describing the input type. */ - private final RowTypeInfo typeInfo; - - /** Object mapper that is used to create output JSON objects. */ - private transient ObjectMapper mapper; - - private final SerializationRuntimeConverter runtimeConverter; - - /** Reusable object node. */ - private transient ObjectNode node; - - private JsonRowSerializationSchema(TypeInformation typeInfo) { - Preconditions.checkNotNull(typeInfo, "Type information"); - Preconditions.checkArgument( - typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); - this.typeInfo = (RowTypeInfo) typeInfo; - this.runtimeConverter = createConverter(typeInfo); - } - - @Override - public void open(InitializationContext context) throws Exception { - mapper = JacksonMapperFactory.createObjectMapper(); - } - - /** Builder for {@link JsonRowSerializationSchema}. */ - @PublicEvolving - public static class Builder { - - private RowTypeInfo typeInfo; - - private Builder() { - // private constructor - } - - /** - * Creates a JSON serialization schema for the given type information. - * - * @param typeInfo Type information describing the result type. The field names of {@link - * Row} are used to parse the JSON properties. - * @deprecated Use {@link JsonRowSerializationSchema#builder()} instead. - */ - @Deprecated - public Builder(TypeInformation typeInfo) { - checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); - this.typeInfo = (RowTypeInfo) typeInfo; - } - - /** - * Creates a JSON serialization schema for the given JSON schema. - * - * @param jsonSchema JSON schema describing the result type - * @see http://json-schema.org/ - * @deprecated Use {@link JsonRowSerializationSchema#builder()} instead. - */ - @Deprecated - public Builder(String jsonSchema) { - this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema))); - } - - /** - * Sets type information for JSON serialization schema. - * - * @param typeInfo Type information describing the result type. The field names of {@link - * Row} are used to parse the JSON properties. - */ - public Builder withTypeInfo(TypeInformation typeInfo) { - checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); - this.typeInfo = (RowTypeInfo) typeInfo; - return this; - } - - /** - * Finalizes the configuration and checks validity. - * - * @return Configured {@link JsonRowSerializationSchema} - */ - public JsonRowSerializationSchema build() { - checkArgument(typeInfo != null, "typeInfo should be set."); - return new JsonRowSerializationSchema(typeInfo); - } - } - - /** Creates a builder for {@link JsonRowSerializationSchema.Builder}. */ - public static Builder builder() { - return new Builder(); - } - - @Override - public byte[] serialize(Row row) { - if (node == null) { - node = mapper.createObjectNode(); - } - - try { - runtimeConverter.convert(mapper, node, row); - return mapper.writeValueAsBytes(node); - } catch (Throwable t) { - throw new RuntimeException( - "Could not serialize row '" - + row - + "'. " - + "Make sure that the schema matches the input.", - t); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - final JsonRowSerializationSchema that = (JsonRowSerializationSchema) o; - return Objects.equals(typeInfo, that.typeInfo); - } - - @Override - public int hashCode() { - return Objects.hash(typeInfo); - } - - /* - Runtime converters - */ - - /** Runtime converter that maps between Java objects and corresponding {@link JsonNode}s. */ - @FunctionalInterface - private interface SerializationRuntimeConverter extends Serializable { - JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object object); - } - - private SerializationRuntimeConverter createConverter(TypeInformation typeInfo) { - SerializationRuntimeConverter baseConverter = - createConverterForSimpleType(typeInfo) - .orElseGet( - () -> - createContainerConverter(typeInfo) - .orElseGet(this::createFallbackConverter)); - return wrapIntoNullableConverter(baseConverter); - } - - private SerializationRuntimeConverter wrapIntoNullableConverter( - SerializationRuntimeConverter converter) { - return (mapper, reuse, object) -> { - if (object == null) { - return mapper.getNodeFactory().nullNode(); - } - - return converter.convert(mapper, reuse, object); - }; - } - - private Optional createContainerConverter( - TypeInformation typeInfo) { - if (typeInfo instanceof RowTypeInfo) { - return Optional.of(createRowConverter((RowTypeInfo) typeInfo)); - } else if (typeInfo instanceof ObjectArrayTypeInfo) { - return Optional.of( - createObjectArrayConverter( - ((ObjectArrayTypeInfo) typeInfo).getComponentInfo())); - } else if (typeInfo instanceof BasicArrayTypeInfo) { - return Optional.of( - createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo())); - } else if (isPrimitiveByteArray(typeInfo)) { - return Optional.of( - (mapper, reuse, object) -> mapper.getNodeFactory().binaryNode((byte[]) object)); - } else { - return Optional.empty(); - } - } - - private boolean isPrimitiveByteArray(TypeInformation typeInfo) { - return typeInfo instanceof PrimitiveArrayTypeInfo - && ((PrimitiveArrayTypeInfo) typeInfo).getComponentType() == Types.BYTE; - } - - private SerializationRuntimeConverter createObjectArrayConverter( - TypeInformation elementTypeInfo) { - SerializationRuntimeConverter elementConverter = createConverter(elementTypeInfo); - return assembleArrayConverter(elementConverter); - } - - private SerializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) { - List fieldConverters = - Arrays.stream(typeInfo.getFieldTypes()) - .map(this::createConverter) - .collect(Collectors.toList()); - - return assembleRowConverter(typeInfo.getFieldNames(), fieldConverters); - } - - private SerializationRuntimeConverter createFallbackConverter() { - return (mapper, reuse, object) -> { - // for types that were specified without JSON schema - // e.g. POJOs - try { - return mapper.valueToTree(object); - } catch (IllegalArgumentException e) { - throw new WrappingRuntimeException( - format("Could not convert object: %s", object), e); - } - }; - } - - private Optional createConverterForSimpleType( - TypeInformation simpleTypeInfo) { - if (simpleTypeInfo == Types.VOID) { - return Optional.of((mapper, reuse, object) -> mapper.getNodeFactory().nullNode()); - } else if (simpleTypeInfo == Types.BOOLEAN) { - return Optional.of( - (mapper, reuse, object) -> - mapper.getNodeFactory().booleanNode((Boolean) object)); - } else if (simpleTypeInfo == Types.STRING) { - return Optional.of( - (mapper, reuse, object) -> mapper.getNodeFactory().textNode((String) object)); - } else if (simpleTypeInfo == Types.INT) { - return Optional.of( - (mapper, reuse, object) -> - mapper.getNodeFactory().numberNode((Integer) object)); - } else if (simpleTypeInfo == Types.LONG) { - return Optional.of( - (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Long) object)); - } else if (simpleTypeInfo == Types.DOUBLE) { - return Optional.of( - (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Double) object)); - } else if (simpleTypeInfo == Types.FLOAT) { - return Optional.of( - (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Float) object)); - } else if (simpleTypeInfo == Types.SHORT) { - return Optional.of( - (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Short) object)); - } else if (simpleTypeInfo == Types.BYTE) { - return Optional.of( - (mapper, reuse, object) -> mapper.getNodeFactory().numberNode((Byte) object)); - } else if (simpleTypeInfo == Types.BIG_DEC) { - return Optional.of(createBigDecimalConverter()); - } else if (simpleTypeInfo == Types.BIG_INT) { - return Optional.of(createBigIntegerConverter()); - } else if (simpleTypeInfo == Types.SQL_DATE) { - return Optional.of(this::convertDate); - } else if (simpleTypeInfo == Types.SQL_TIME) { - return Optional.of(this::convertTime); - } else if (simpleTypeInfo == Types.SQL_TIMESTAMP) { - return Optional.of(this::convertTimestamp); - } else if (simpleTypeInfo == Types.LOCAL_DATE) { - return Optional.of(this::convertLocalDate); - } else if (simpleTypeInfo == Types.LOCAL_TIME) { - return Optional.of(this::convertLocalTime); - } else if (simpleTypeInfo == Types.LOCAL_DATE_TIME) { - return Optional.of(this::convertLocalDateTime); - } else { - return Optional.empty(); - } - } - - private JsonNode convertLocalDate(ObjectMapper mapper, JsonNode reuse, Object object) { - return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format((LocalDate) object)); - } - - private JsonNode convertDate(ObjectMapper mapper, JsonNode reuse, Object object) { - Date date = (Date) object; - return convertLocalDate(mapper, reuse, date.toLocalDate()); - } - - private JsonNode convertLocalDateTime(ObjectMapper mapper, JsonNode reuse, Object object) { - return mapper.getNodeFactory() - .textNode(RFC3339_TIMESTAMP_FORMAT.format((LocalDateTime) object)); - } - - private JsonNode convertTimestamp(ObjectMapper mapper, JsonNode reuse, Object object) { - Timestamp timestamp = (Timestamp) object; - return convertLocalDateTime(mapper, reuse, timestamp.toLocalDateTime()); - } - - private JsonNode convertLocalTime(ObjectMapper mapper, JsonNode reuse, Object object) { - JsonNodeFactory nodeFactory = mapper.getNodeFactory(); - return nodeFactory.textNode(RFC3339_TIME_FORMAT.format((LocalTime) object)); - } - - private JsonNode convertTime(ObjectMapper mapper, JsonNode reuse, Object object) { - final Time time = (Time) object; - return convertLocalTime(mapper, reuse, time.toLocalTime()); - } - - private SerializationRuntimeConverter createBigDecimalConverter() { - return (mapper, reuse, object) -> { - // convert decimal if necessary - JsonNodeFactory nodeFactory = mapper.getNodeFactory(); - if (object instanceof BigDecimal) { - return nodeFactory.numberNode((BigDecimal) object); - } - return nodeFactory.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); - }; - } - - private SerializationRuntimeConverter createBigIntegerConverter() { - return (mapper, reuse, object) -> { - // convert decimal if necessary - JsonNodeFactory nodeFactory = mapper.getNodeFactory(); - if (object instanceof BigInteger) { - return nodeFactory.numberNode((BigInteger) object); - } - return nodeFactory.numberNode(BigInteger.valueOf(((Number) object).longValue())); - }; - } - - private SerializationRuntimeConverter assembleRowConverter( - String[] fieldNames, List fieldConverters) { - return (mapper, reuse, object) -> { - ObjectNode node; - - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createObjectNode(); - } else { - node = (ObjectNode) reuse; - } - - Row row = (Row) object; - - for (int i = 0; i < fieldNames.length; i++) { - String fieldName = fieldNames[i]; - node.set( - fieldName, - fieldConverters - .get(i) - .convert(mapper, node.get(fieldNames[i]), row.getField(i))); - } - - return node; - }; - } - - private SerializationRuntimeConverter assembleArrayConverter( - SerializationRuntimeConverter elementConverter) { - return (mapper, reuse, object) -> { - ArrayNode node; - - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createArrayNode(); - } else { - node = (ArrayNode) reuse; - node.removeAll(); - } - - Object[] array = (Object[]) object; - - for (Object element : array) { - node.add(elementConverter.convert(mapper, null, element)); - } - - return node; - }; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java deleted file mode 100644 index c029fa1dc..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonSerializationSchema.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.util.function.SerializableSupplier; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; - -/** SerializationSchema that serializes an object to a JSON String. */ -@PublicEvolving -public class JsonSerializationSchema implements SerializationSchema { - - private static final long serialVersionUID = 1L; - - private final SerializableSupplier mapperFactory; - - protected transient ObjectMapper mapper; - - public JsonSerializationSchema() { - this(() -> new ObjectMapper()); - } - - public JsonSerializationSchema(SerializableSupplier mapperFactory) { - this.mapperFactory = mapperFactory; - } - - @Override - public void open(InitializationContext context) { - mapper = mapperFactory.get(); - } - - @Override - public byte[] serialize(T element) { - try { - return mapper.writeValueAsBytes(element); - } catch (JsonProcessingException e) { - throw new RuntimeException( - String.format("Could not serialize value '%s'.", element), e); - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java deleted file mode 100644 index c89b5f583..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/JsonToRowDataConverters.java +++ /dev/null @@ -1,402 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeFamily; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.MultisetType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.utils.LogicalTypeUtils; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; - -import java.io.IOException; -import java.io.Serializable; -import java.lang.reflect.Array; -import java.math.BigDecimal; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.ZoneOffset; -import java.time.temporal.TemporalAccessor; -import java.time.temporal.TemporalQueries; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; - -import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.common.TimeFormats.SQL_TIME_FORMAT; - -/** Tool class used to convert from {@link JsonNode} to {@link RowData}. * */ -@Internal -public class JsonToRowDataConverters implements Serializable { - - private static final long serialVersionUID = 1L; - - /** Flag indicating whether to fail if a field is missing. */ - private final boolean failOnMissingField; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - /** Timestamp format specification which is used to parse timestamp. */ - private final TimestampFormat timestampFormat; - - public JsonToRowDataConverters( - boolean failOnMissingField, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - this.failOnMissingField = failOnMissingField; - this.ignoreParseErrors = ignoreParseErrors; - this.timestampFormat = timestampFormat; - } - - /** - * Runtime converter that converts {@link JsonNode}s into objects of Flink Table & SQL internal - * data structures. - */ - @FunctionalInterface - public interface JsonToRowDataConverter extends Serializable { - Object convert(JsonNode jsonNode); - } - - /** Creates a runtime converter which is null safe. */ - public JsonToRowDataConverter createConverter(LogicalType type) { - return wrapIntoNullableConverter(createNotNullConverter(type)); - } - - /** Creates a runtime converter which assuming input object is not null. */ - private JsonToRowDataConverter createNotNullConverter(LogicalType type) { - switch (type.getTypeRoot()) { - case NULL: - return jsonNode -> null; - case BOOLEAN: - return this::convertToBoolean; - case TINYINT: - return jsonNode -> Byte.parseByte(jsonNode.asText().trim()); - case SMALLINT: - return jsonNode -> Short.parseShort(jsonNode.asText().trim()); - case INTEGER: - case INTERVAL_YEAR_MONTH: - return this::convertToInt; - case BIGINT: - case INTERVAL_DAY_TIME: - return this::convertToLong; - case DATE: - return this::convertToDate; - case TIME_WITHOUT_TIME_ZONE: - return this::convertToTime; - case TIMESTAMP_WITHOUT_TIME_ZONE: - return this::convertToTimestamp; - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return this::convertToTimestampWithLocalZone; - case FLOAT: - return this::convertToFloat; - case DOUBLE: - return this::convertToDouble; - case CHAR: - case VARCHAR: - return this::convertToString; - case BINARY: - case VARBINARY: - return this::convertToBytes; - case DECIMAL: - return createDecimalConverter((DecimalType) type); - case ARRAY: - return createArrayConverter((ArrayType) type); - case MAP: - MapType mapType = (MapType) type; - return createMapConverter( - mapType.asSummaryString(), mapType.getKeyType(), mapType.getValueType()); - case MULTISET: - MultisetType multisetType = (MultisetType) type; - return createMapConverter( - multisetType.asSummaryString(), - multisetType.getElementType(), - new IntType()); - case ROW: - return createRowConverter((RowType) type); - case RAW: - default: - throw new UnsupportedOperationException("Unsupported type: " + type); - } - } - - private boolean convertToBoolean(JsonNode jsonNode) { - if (jsonNode.isBoolean()) { - // avoid redundant toString and parseBoolean, for better performance - return jsonNode.asBoolean(); - } else { - return Boolean.parseBoolean(jsonNode.asText().trim()); - } - } - - private int convertToInt(JsonNode jsonNode) { - if (jsonNode.canConvertToInt()) { - // avoid redundant toString and parseInt, for better performance - return jsonNode.asInt(); - } else { - return Integer.parseInt(jsonNode.asText().trim()); - } - } - - private long convertToLong(JsonNode jsonNode) { - if (jsonNode.canConvertToLong()) { - // avoid redundant toString and parseLong, for better performance - return jsonNode.asLong(); - } else { - return Long.parseLong(jsonNode.asText().trim()); - } - } - - private double convertToDouble(JsonNode jsonNode) { - if (jsonNode.isDouble()) { - // avoid redundant toString and parseDouble, for better performance - return jsonNode.asDouble(); - } else { - return Double.parseDouble(jsonNode.asText().trim()); - } - } - - private float convertToFloat(JsonNode jsonNode) { - if (jsonNode.isDouble()) { - // avoid redundant toString and parseDouble, for better performance - return (float) jsonNode.asDouble(); - } else { - return Float.parseFloat(jsonNode.asText().trim()); - } - } - - private int convertToDate(JsonNode jsonNode) { - LocalDate date = ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); - return (int) date.toEpochDay(); - } - - private int convertToTime(JsonNode jsonNode) { - TemporalAccessor parsedTime = SQL_TIME_FORMAT.parse(jsonNode.asText()); - LocalTime localTime = parsedTime.query(TemporalQueries.localTime()); - - // get number of milliseconds of the day - return localTime.toSecondOfDay() * 1000; - } - - private TimestampData convertToTimestamp(JsonNode jsonNode) { - TemporalAccessor parsedTimestamp; - switch (timestampFormat) { - case SQL: - parsedTimestamp = SQL_TIMESTAMP_FORMAT.parse(jsonNode.asText()); - break; - case ISO_8601: - parsedTimestamp = ISO8601_TIMESTAMP_FORMAT.parse(jsonNode.asText()); - break; - default: - throw new TableException( - String.format( - "Unsupported timestamp format '%s'. Validator should have checked that.", - timestampFormat)); - } - LocalTime localTime = parsedTimestamp.query(TemporalQueries.localTime()); - LocalDate localDate = parsedTimestamp.query(TemporalQueries.localDate()); - - return TimestampData.fromLocalDateTime(LocalDateTime.of(localDate, localTime)); - } - - private TimestampData convertToTimestampWithLocalZone(JsonNode jsonNode) { - TemporalAccessor parsedTimestampWithLocalZone; - switch (timestampFormat) { - case SQL: - parsedTimestampWithLocalZone = - SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); - break; - case ISO_8601: - parsedTimestampWithLocalZone = - ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.parse(jsonNode.asText()); - break; - default: - throw new TableException( - String.format( - "Unsupported timestamp format '%s'. Validator should have checked that.", - timestampFormat)); - } - LocalTime localTime = parsedTimestampWithLocalZone.query(TemporalQueries.localTime()); - LocalDate localDate = parsedTimestampWithLocalZone.query(TemporalQueries.localDate()); - - return TimestampData.fromInstant( - LocalDateTime.of(localDate, localTime).toInstant(ZoneOffset.UTC)); - } - - private StringData convertToString(JsonNode jsonNode) { - if (jsonNode.isContainerNode()) { - return StringData.fromString(jsonNode.toString()); - } else { - return StringData.fromString(jsonNode.asText()); - } - } - - private byte[] convertToBytes(JsonNode jsonNode) { - try { - return jsonNode.binaryValue(); - } catch (IOException e) { - throw new JsonParseException("Unable to deserialize byte array.", e); - } - } - - private JsonToRowDataConverter createDecimalConverter(DecimalType decimalType) { - final int precision = decimalType.getPrecision(); - final int scale = decimalType.getScale(); - return jsonNode -> { - BigDecimal bigDecimal; - if (jsonNode.isBigDecimal()) { - bigDecimal = jsonNode.decimalValue(); - } else { - bigDecimal = new BigDecimal(jsonNode.asText()); - } - return DecimalData.fromBigDecimal(bigDecimal, precision, scale); - }; - } - - private JsonToRowDataConverter createArrayConverter(ArrayType arrayType) { - JsonToRowDataConverter elementConverter = createConverter(arrayType.getElementType()); - final Class elementClass = - LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType()); - return jsonNode -> { - final ArrayNode node = (ArrayNode) jsonNode; - final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); - for (int i = 0; i < node.size(); i++) { - final JsonNode innerNode = node.get(i); - array[i] = elementConverter.convert(innerNode); - } - return new GenericArrayData(array); - }; - } - - private JsonToRowDataConverter createMapConverter( - String typeSummary, LogicalType keyType, LogicalType valueType) { - if (!keyType.is(LogicalTypeFamily.CHARACTER_STRING)) { - throw new UnsupportedOperationException( - "JSON format doesn't support non-string as key type of map. " - + "The type is: " - + typeSummary); - } - final JsonToRowDataConverter keyConverter = createConverter(keyType); - final JsonToRowDataConverter valueConverter = createConverter(valueType); - - return jsonNode -> { - Iterator> fields = jsonNode.fields(); - Map result = new HashMap<>(); - while (fields.hasNext()) { - Map.Entry entry = fields.next(); - Object key = keyConverter.convert(TextNode.valueOf(entry.getKey())); - Object value = valueConverter.convert(entry.getValue()); - result.put(key, value); - } - return new GenericMapData(result); - }; - } - - public JsonToRowDataConverter createRowConverter(RowType rowType) { - final JsonToRowDataConverter[] fieldConverters = - rowType.getFields().stream() - .map(RowType.RowField::getType) - .map(this::createConverter) - .toArray(JsonToRowDataConverter[]::new); - final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]); - - return jsonNode -> { - ObjectNode node = (ObjectNode) jsonNode; - int arity = fieldNames.length; - GenericRowData row = new GenericRowData(arity); - for (int i = 0; i < arity; i++) { - String fieldName = fieldNames[i]; - JsonNode field = node.get(fieldName); - try { - Object convertedField = convertField(fieldConverters[i], fieldName, field); - row.setField(i, convertedField); - } catch (Throwable t) { - throw new JsonParseException( - String.format("Fail to deserialize at field: %s.", fieldName), t); - } - } - return row; - }; - } - - private Object convertField( - JsonToRowDataConverter fieldConverter, String fieldName, JsonNode field) { - if (field == null) { - if (failOnMissingField) { - throw new JsonParseException("Could not find field with name '" + fieldName + "'."); - } else { - return null; - } - } else { - return fieldConverter.convert(field); - } - } - - private JsonToRowDataConverter wrapIntoNullableConverter(JsonToRowDataConverter converter) { - return jsonNode -> { - if (jsonNode == null || jsonNode.isNull() || jsonNode.isMissingNode()) { - return null; - } - try { - return converter.convert(jsonNode); - } catch (Throwable t) { - if (!ignoreParseErrors) { - throw t; - } - return null; - } - }; - } - - /** Exception which refers to parse errors in converters. */ - private static final class JsonParseException extends RuntimeException { - private static final long serialVersionUID = 1L; - - public JsonParseException(String message) { - super(message); - } - - public JsonParseException(String message, Throwable cause) { - super(message, cause); - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java deleted file mode 100644 index 2a1cd076e..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/RowDataToJsonConverters.java +++ /dev/null @@ -1,355 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeFamily; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.MultisetType; -import org.apache.flink.table.types.logical.RowType; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; - -import java.io.Serializable; -import java.math.BigDecimal; -import java.time.LocalDate; -import java.time.LocalTime; -import java.time.ZoneOffset; -import java.util.Arrays; - -import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.common.TimeFormats.ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_FORMAT; -import static org.apache.flink.formats.common.TimeFormats.SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT; -import static org.apache.flink.formats.common.TimeFormats.SQL_TIME_FORMAT; - -/** Tool class used to convert from {@link RowData} to {@link JsonNode}. * */ -@Internal -public class RowDataToJsonConverters implements Serializable { - - private static final long serialVersionUID = 1L; - - /** Timestamp format specification which is used to parse timestamp. */ - private final TimestampFormat timestampFormat; - - /** The handling mode when serializing null keys for map data. */ - private final JsonFormatOptions.MapNullKeyMode mapNullKeyMode; - - /** The string literal when handling mode for map null key LITERAL. is */ - private final String mapNullKeyLiteral; - - public RowDataToJsonConverters( - TimestampFormat timestampFormat, - JsonFormatOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral) { - this.timestampFormat = timestampFormat; - this.mapNullKeyMode = mapNullKeyMode; - this.mapNullKeyLiteral = mapNullKeyLiteral; - } - - /** - * Runtime converter that converts objects of Flink Table & SQL internal data structures to - * corresponding {@link JsonNode}s. - */ - public interface RowDataToJsonConverter extends Serializable { - JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value); - } - - /** Creates a runtime converter which is null safe. */ - public RowDataToJsonConverter createConverter(LogicalType type) { - return wrapIntoNullableConverter(createNotNullConverter(type)); - } - - /** Creates a runtime converter which assuming input object is not null. */ - private RowDataToJsonConverter createNotNullConverter(LogicalType type) { - switch (type.getTypeRoot()) { - case NULL: - return (mapper, reuse, value) -> mapper.getNodeFactory().nullNode(); - case BOOLEAN: - return (mapper, reuse, value) -> - mapper.getNodeFactory().booleanNode((boolean) value); - case TINYINT: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((byte) value); - case SMALLINT: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((short) value); - case INTEGER: - case INTERVAL_YEAR_MONTH: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((int) value); - case BIGINT: - case INTERVAL_DAY_TIME: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((long) value); - case FLOAT: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((float) value); - case DOUBLE: - return (mapper, reuse, value) -> mapper.getNodeFactory().numberNode((double) value); - case CHAR: - case VARCHAR: - // value is BinaryString - return (mapper, reuse, value) -> mapper.getNodeFactory().textNode(value.toString()); - case BINARY: - case VARBINARY: - return (mapper, reuse, value) -> mapper.getNodeFactory().binaryNode((byte[]) value); - case DATE: - return createDateConverter(); - case TIME_WITHOUT_TIME_ZONE: - return createTimeConverter(); - case TIMESTAMP_WITHOUT_TIME_ZONE: - return createTimestampConverter(); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return createTimestampWithLocalZone(); - case DECIMAL: - return createDecimalConverter(); - case ARRAY: - return createArrayConverter((ArrayType) type); - case MAP: - MapType mapType = (MapType) type; - return createMapConverter( - mapType.asSummaryString(), mapType.getKeyType(), mapType.getValueType()); - case MULTISET: - MultisetType multisetType = (MultisetType) type; - return createMapConverter( - multisetType.asSummaryString(), - multisetType.getElementType(), - new IntType()); - case ROW: - return createRowConverter((RowType) type); - case RAW: - default: - throw new UnsupportedOperationException("Not support to parse type: " + type); - } - } - - private RowDataToJsonConverter createDecimalConverter() { - return (mapper, reuse, value) -> { - BigDecimal bd = ((DecimalData) value).toBigDecimal(); - return mapper.getNodeFactory().numberNode(bd); - }; - } - - private RowDataToJsonConverter createDateConverter() { - return (mapper, reuse, value) -> { - int days = (int) value; - LocalDate date = LocalDate.ofEpochDay(days); - return mapper.getNodeFactory().textNode(ISO_LOCAL_DATE.format(date)); - }; - } - - private RowDataToJsonConverter createTimeConverter() { - return (mapper, reuse, value) -> { - int millisecond = (int) value; - LocalTime time = LocalTime.ofSecondOfDay(millisecond / 1000L); - return mapper.getNodeFactory().textNode(SQL_TIME_FORMAT.format(time)); - }; - } - - private RowDataToJsonConverter createTimestampConverter() { - switch (timestampFormat) { - case ISO_8601: - return (mapper, reuse, value) -> { - TimestampData timestamp = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(ISO8601_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); - }; - case SQL: - return (mapper, reuse, value) -> { - TimestampData timestamp = (TimestampData) value; - return mapper.getNodeFactory() - .textNode(SQL_TIMESTAMP_FORMAT.format(timestamp.toLocalDateTime())); - }; - default: - throw new TableException( - "Unsupported timestamp format. Validator should have checked that."); - } - } - - private RowDataToJsonConverter createTimestampWithLocalZone() { - switch (timestampFormat) { - case ISO_8601: - return (mapper, reuse, value) -> { - TimestampData timestampWithLocalZone = (TimestampData) value; - return mapper.getNodeFactory() - .textNode( - ISO8601_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format( - timestampWithLocalZone - .toInstant() - .atOffset(ZoneOffset.UTC))); - }; - case SQL: - return (mapper, reuse, value) -> { - TimestampData timestampWithLocalZone = (TimestampData) value; - return mapper.getNodeFactory() - .textNode( - SQL_TIMESTAMP_WITH_LOCAL_TIMEZONE_FORMAT.format( - timestampWithLocalZone - .toInstant() - .atOffset(ZoneOffset.UTC))); - }; - default: - throw new TableException( - "Unsupported timestamp format. Validator should have checked that."); - } - } - - private RowDataToJsonConverter createArrayConverter(ArrayType type) { - final LogicalType elementType = type.getElementType(); - final RowDataToJsonConverter elementConverter = createConverter(elementType); - final ArrayData.ElementGetter elementGetter = ArrayData.createElementGetter(elementType); - return (mapper, reuse, value) -> { - ArrayNode node; - - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createArrayNode(); - } else { - node = (ArrayNode) reuse; - node.removeAll(); - } - - ArrayData array = (ArrayData) value; - int numElements = array.size(); - for (int i = 0; i < numElements; i++) { - Object element = elementGetter.getElementOrNull(array, i); - node.add(elementConverter.convert(mapper, null, element)); - } - - return node; - }; - } - - private RowDataToJsonConverter createMapConverter( - String typeSummary, LogicalType keyType, LogicalType valueType) { - if (!keyType.is(LogicalTypeFamily.CHARACTER_STRING)) { - throw new UnsupportedOperationException( - "JSON format doesn't support non-string as key type of map. " - + "The type is: " - + typeSummary); - } - final RowDataToJsonConverter valueConverter = createConverter(valueType); - final ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(valueType); - return (mapper, reuse, object) -> { - ObjectNode node; - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createObjectNode(); - } else { - node = (ObjectNode) reuse; - node.removeAll(); - } - - MapData map = (MapData) object; - ArrayData keyArray = map.keyArray(); - ArrayData valueArray = map.valueArray(); - int numElements = map.size(); - for (int i = 0; i < numElements; i++) { - String fieldName = null; - if (keyArray.isNullAt(i)) { - // when map key is null - switch (mapNullKeyMode) { - case LITERAL: - fieldName = mapNullKeyLiteral; - break; - case DROP: - continue; - case FAIL: - throw new RuntimeException( - String.format( - "JSON format doesn't support to serialize map data with null keys. " - + "You can drop null key entries or encode null in literals by specifying %s option.", - JsonFormatOptions.MAP_NULL_KEY_MODE.key())); - default: - throw new RuntimeException( - "Unsupported map null key mode. Validator should have checked that."); - } - } else { - fieldName = keyArray.getString(i).toString(); - } - - Object value = valueGetter.getElementOrNull(valueArray, i); - node.set(fieldName, valueConverter.convert(mapper, node.get(fieldName), value)); - } - - return node; - }; - } - - private RowDataToJsonConverter createRowConverter(RowType type) { - final String[] fieldNames = type.getFieldNames().toArray(new String[0]); - final LogicalType[] fieldTypes = - type.getFields().stream() - .map(RowType.RowField::getType) - .toArray(LogicalType[]::new); - final RowDataToJsonConverter[] fieldConverters = - Arrays.stream(fieldTypes) - .map(this::createConverter) - .toArray(RowDataToJsonConverter[]::new); - final int fieldCount = type.getFieldCount(); - final RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[fieldTypes.length]; - for (int i = 0; i < fieldCount; i++) { - fieldGetters[i] = RowData.createFieldGetter(fieldTypes[i], i); - } - - return (mapper, reuse, value) -> { - ObjectNode node; - // reuse could be a NullNode if last record is null. - if (reuse == null || reuse.isNull()) { - node = mapper.createObjectNode(); - } else { - node = (ObjectNode) reuse; - } - RowData row = (RowData) value; - for (int i = 0; i < fieldCount; i++) { - String fieldName = fieldNames[i]; - try { - Object field = fieldGetters[i].getFieldOrNull(row); - node.set( - fieldName, - fieldConverters[i].convert(mapper, node.get(fieldName), field)); - } catch (Throwable t) { - throw new RuntimeException( - String.format("Fail to serialize at field: %s.", fieldName), t); - } - } - return node; - }; - } - - private RowDataToJsonConverter wrapIntoNullableConverter(RowDataToJsonConverter converter) { - return (mapper, reuse, object) -> { - if (object == null) { - return mapper.getNodeFactory().nullNode(); - } - - return converter.convert(mapper, reuse, object); - }; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java deleted file mode 100644 index 4c981d5a3..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java +++ /dev/null @@ -1,245 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.canal; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.canal.CanalJsonDeserializationSchema.MetadataConverter; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.Projection; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.ProjectableDecodingFormat; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.types.RowKind; - -import javax.annotation.Nullable; - -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -/** {@link DecodingFormat} for Canal using JSON encoding. */ -public class CanalJsonDecodingFormat - implements ProjectableDecodingFormat> { - - // -------------------------------------------------------------------------------------------- - // Mutable attributes - // -------------------------------------------------------------------------------------------- - - private List metadataKeys; - - // -------------------------------------------------------------------------------------------- - // Canal-specific attributes - // -------------------------------------------------------------------------------------------- - - private final @Nullable String database; - - private final @Nullable String table; - - private final boolean ignoreParseErrors; - - private final TimestampFormat timestampFormat; - - public CanalJsonDecodingFormat( - String database, - String table, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - this.database = database; - this.table = table; - this.ignoreParseErrors = ignoreParseErrors; - this.timestampFormat = timestampFormat; - this.metadataKeys = Collections.emptyList(); - } - - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) { - physicalDataType = Projection.of(projections).project(physicalDataType); - final List readableMetadata = - metadataKeys.stream() - .map( - k -> - Stream.of(ReadableMetadata.values()) - .filter(rm -> rm.key.equals(k)) - .findFirst() - .orElseThrow(IllegalStateException::new)) - .collect(Collectors.toList()); - final List metadataFields = - readableMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList()); - final DataType producedDataType = - DataTypeUtils.appendRowFields(physicalDataType, metadataFields); - final TypeInformation producedTypeInfo = - context.createTypeInformation(producedDataType); - return CanalJsonDeserializationSchema.builder( - physicalDataType, readableMetadata, producedTypeInfo) - .setDatabase(database) - .setTable(table) - .setIgnoreParseErrors(ignoreParseErrors) - .setTimestampFormat(timestampFormat) - .build(); - } - - @Override - public Map listReadableMetadata() { - final Map metadataMap = new LinkedHashMap<>(); - Stream.of(ReadableMetadata.values()) - .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); - return metadataMap; - } - - @Override - public void applyReadableMetadata(List metadataKeys) { - this.metadataKeys = metadataKeys; - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - // -------------------------------------------------------------------------------------------- - // Metadata handling - // -------------------------------------------------------------------------------------------- - - /** List of metadata that can be read with this format. */ - enum ReadableMetadata { - DATABASE( - "database", - DataTypes.STRING().nullable(), - DataTypes.FIELD("database", DataTypes.STRING()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getString(pos); - } - }), - - TABLE( - "table", - DataTypes.STRING().nullable(), - DataTypes.FIELD("table", DataTypes.STRING()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getString(pos); - } - }), - - SQL_TYPE( - "sql-type", - DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.INT().nullable()).nullable(), - DataTypes.FIELD( - "sqlType", - DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.INT().nullable())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getMap(pos); - } - }), - - PK_NAMES( - "pk-names", - DataTypes.ARRAY(DataTypes.STRING()).nullable(), - DataTypes.FIELD("pkNames", DataTypes.ARRAY(DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getArray(pos); - } - }), - - INGESTION_TIMESTAMP( - "ingestion-timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), - DataTypes.FIELD("ts", DataTypes.BIGINT()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - if (row.isNullAt(pos)) { - return null; - } - return TimestampData.fromEpochMillis(row.getLong(pos)); - } - }), - - EVENT_TIMESTAMP( - "event-timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), - DataTypes.FIELD("es", DataTypes.BIGINT()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - if (row.isNullAt(pos)) { - return null; - } - return TimestampData.fromEpochMillis(row.getLong(pos)); - } - }); - - final String key; - - final DataType dataType; - - final DataTypes.Field requiredJsonField; - - final MetadataConverter converter; - - ReadableMetadata( - String key, - DataType dataType, - DataTypes.Field requiredJsonField, - MetadataConverter converter) { - this.key = key; - this.dataType = dataType; - this.requiredJsonField = requiredJsonField; - this.converter = converter; - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java deleted file mode 100644 index bb9777833..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java +++ /dev/null @@ -1,412 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.canal; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; -import org.apache.flink.formats.json.canal.CanalJsonDecodingFormat.ReadableMetadata; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Collector; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.io.Serializable; -import java.util.List; -import java.util.Objects; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -import static java.lang.String.format; - -/** - * Deserialization schema from Canal JSON to Flink Table/SQL internal data structure {@link - * RowData}. The deserialization schema knows Canal's schema definition and can extract the database - * data and convert into {@link RowData} with {@link RowKind}. - * - *

Deserializes a byte[] message as a JSON object and reads the specified fields. - * - *

Failures during deserialization are forwarded as wrapped IOExceptions. - * - * @see Alibaba Canal - */ -public final class CanalJsonDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - - private static final String FIELD_OLD = "old"; - private static final String OP_INSERT = "INSERT"; - private static final String OP_UPDATE = "UPDATE"; - private static final String OP_DELETE = "DELETE"; - private static final String OP_CREATE = "CREATE"; - - /** The deserializer to deserialize Canal JSON data. */ - private final JsonRowDataDeserializationSchema jsonDeserializer; - - /** Flag that indicates that an additional projection is required for metadata. */ - private final boolean hasMetadata; - - /** Metadata to be extracted for every record. */ - private final MetadataConverter[] metadataConverters; - - /** {@link TypeInformation} of the produced {@link RowData} (physical + meta data). */ - private final TypeInformation producedTypeInfo; - - /** Only read changelogs from the specific database. */ - private final @Nullable String database; - - /** Only read changelogs from the specific table. */ - private final @Nullable String table; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - /** Names of fields. */ - private final List fieldNames; - - /** Number of fields. */ - private final int fieldCount; - - /** Pattern of the specific database. */ - private final Pattern databasePattern; - - /** Pattern of the specific table. */ - private final Pattern tablePattern; - - private CanalJsonDeserializationSchema( - DataType physicalDataType, - List requestedMetadata, - TypeInformation producedTypeInfo, - @Nullable String database, - @Nullable String table, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata); - this.jsonDeserializer = - new JsonRowDataDeserializationSchema( - jsonRowType, - // the result type is never used, so it's fine to pass in the produced type - // info - producedTypeInfo, - false, // ignoreParseErrors already contains the functionality of - // failOnMissingField - ignoreParseErrors, - timestampFormat); - this.hasMetadata = requestedMetadata.size() > 0; - this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata); - this.producedTypeInfo = producedTypeInfo; - this.database = database; - this.table = table; - this.ignoreParseErrors = ignoreParseErrors; - final RowType physicalRowType = ((RowType) physicalDataType.getLogicalType()); - this.fieldNames = physicalRowType.getFieldNames(); - this.fieldCount = physicalRowType.getFieldCount(); - this.databasePattern = database == null ? null : Pattern.compile(database); - this.tablePattern = table == null ? null : Pattern.compile(table); - } - - // ------------------------------------------------------------------------------------------ - // Builder - // ------------------------------------------------------------------------------------------ - - /** Creates A builder for building a {@link CanalJsonDeserializationSchema}. */ - public static Builder builder( - DataType physicalDataType, - List requestedMetadata, - TypeInformation producedTypeInfo) { - return new Builder(physicalDataType, requestedMetadata, producedTypeInfo); - } - - /** A builder for creating a {@link CanalJsonDeserializationSchema}. */ - @Internal - public static final class Builder { - private final DataType physicalDataType; - private final List requestedMetadata; - private final TypeInformation producedTypeInfo; - private String database = null; - private String table = null; - private boolean ignoreParseErrors = false; - private TimestampFormat timestampFormat = TimestampFormat.SQL; - - private Builder( - DataType physicalDataType, - List requestedMetadata, - TypeInformation producedTypeInfo) { - this.physicalDataType = physicalDataType; - this.requestedMetadata = requestedMetadata; - this.producedTypeInfo = producedTypeInfo; - } - - public Builder setDatabase(String database) { - this.database = database; - return this; - } - - public Builder setTable(String table) { - this.table = table; - return this; - } - - public Builder setIgnoreParseErrors(boolean ignoreParseErrors) { - this.ignoreParseErrors = ignoreParseErrors; - return this; - } - - public Builder setTimestampFormat(TimestampFormat timestampFormat) { - this.timestampFormat = timestampFormat; - return this; - } - - public CanalJsonDeserializationSchema build() { - return new CanalJsonDeserializationSchema( - physicalDataType, - requestedMetadata, - producedTypeInfo, - database, - table, - ignoreParseErrors, - timestampFormat); - } - } - - // ------------------------------------------------------------------------------------------ - - @Override - public void open(InitializationContext context) throws Exception { - jsonDeserializer.open(context); - } - - @Override - public RowData deserialize(byte[] message) throws IOException { - throw new RuntimeException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); - } - - @Override - public void deserialize(@Nullable byte[] message, Collector out) throws IOException { - if (message == null || message.length == 0) { - return; - } - try { - final JsonNode root = jsonDeserializer.deserializeToJsonNode(message); - if (database != null) { - if (!databasePattern - .matcher(root.get(ReadableMetadata.DATABASE.key).asText()) - .matches()) { - return; - } - } - if (table != null) { - if (!tablePattern - .matcher(root.get(ReadableMetadata.TABLE.key).asText()) - .matches()) { - return; - } - } - final GenericRowData row = (GenericRowData) jsonDeserializer.convertToRowData(root); - String type = row.getString(2).toString(); // "type" field - if (OP_INSERT.equals(type)) { - // "data" field is an array of row, contains inserted rows - ArrayData data = row.getArray(0); - for (int i = 0; i < data.size(); i++) { - GenericRowData insert = (GenericRowData) data.getRow(i, fieldCount); - insert.setRowKind(RowKind.INSERT); - emitRow(row, insert, out); - } - } else if (OP_UPDATE.equals(type)) { - // "data" field is an array of row, contains new rows - ArrayData data = row.getArray(0); - // "old" field is an array of row, contains old values - ArrayData old = row.getArray(1); - for (int i = 0; i < data.size(); i++) { - // the underlying JSON deserialization schema always produce GenericRowData. - GenericRowData after = (GenericRowData) data.getRow(i, fieldCount); - GenericRowData before = (GenericRowData) old.getRow(i, fieldCount); - final JsonNode oldField = root.get(FIELD_OLD); - for (int f = 0; f < fieldCount; f++) { - if (before.isNullAt(f) && oldField.findValue(fieldNames.get(f)) == null) { - // fields in "old" (before) means the fields are changed - // fields not in "old" (before) means the fields are not changed - // so we just copy the not changed fields into before - before.setField(f, after.getField(f)); - } - } - before.setRowKind(RowKind.UPDATE_BEFORE); - after.setRowKind(RowKind.UPDATE_AFTER); - emitRow(row, before, out); - emitRow(row, after, out); - } - } else if (OP_DELETE.equals(type)) { - // "data" field is an array of row, contains deleted rows - ArrayData data = row.getArray(0); - for (int i = 0; i < data.size(); i++) { - GenericRowData insert = (GenericRowData) data.getRow(i, fieldCount); - insert.setRowKind(RowKind.DELETE); - emitRow(row, insert, out); - } - } else if (OP_CREATE.equals(type)) { - // "data" field is null and "type" is "CREATE" which means - // this is a DDL change event, and we should skip it. - return; - } else { - if (!ignoreParseErrors) { - throw new IOException( - format( - "Unknown \"type\" value \"%s\". The Canal JSON message is '%s'", - type, new String(message))); - } - } - } catch (Throwable t) { - // a big try catch to protect the processing. - if (!ignoreParseErrors) { - throw new IOException( - format("Corrupt Canal JSON message '%s'.", new String(message)), t); - } - } - } - - private void emitRow( - GenericRowData rootRow, GenericRowData physicalRow, Collector out) { - // shortcut in case no output projection is required - if (!hasMetadata) { - out.collect(physicalRow); - return; - } - final int physicalArity = physicalRow.getArity(); - final int metadataArity = metadataConverters.length; - final GenericRowData producedRow = - new GenericRowData(physicalRow.getRowKind(), physicalArity + metadataArity); - for (int physicalPos = 0; physicalPos < physicalArity; physicalPos++) { - producedRow.setField(physicalPos, physicalRow.getField(physicalPos)); - } - for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { - producedRow.setField( - physicalArity + metadataPos, metadataConverters[metadataPos].convert(rootRow)); - } - out.collect(producedRow); - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return producedTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - CanalJsonDeserializationSchema that = (CanalJsonDeserializationSchema) o; - return Objects.equals(jsonDeserializer, that.jsonDeserializer) - && hasMetadata == that.hasMetadata - && Objects.equals(producedTypeInfo, that.producedTypeInfo) - && Objects.equals(database, that.database) - && Objects.equals(table, that.table) - && ignoreParseErrors == that.ignoreParseErrors - && fieldCount == that.fieldCount; - } - - @Override - public int hashCode() { - return Objects.hash( - jsonDeserializer, - hasMetadata, - producedTypeInfo, - database, - table, - ignoreParseErrors, - fieldCount); - } - - // -------------------------------------------------------------------------------------------- - - private static RowType createJsonRowType( - DataType physicalDataType, List readableMetadata) { - // Canal JSON contains other information, e.g. "ts", "sql", but we don't need them - DataType root = - DataTypes.ROW( - DataTypes.FIELD("data", DataTypes.ARRAY(physicalDataType)), - DataTypes.FIELD("old", DataTypes.ARRAY(physicalDataType)), - DataTypes.FIELD("type", DataTypes.STRING()), - ReadableMetadata.DATABASE.requiredJsonField, - ReadableMetadata.TABLE.requiredJsonField); - // append fields that are required for reading metadata in the root - final List rootMetadataFields = - readableMetadata.stream() - .filter(m -> m != ReadableMetadata.DATABASE && m != ReadableMetadata.TABLE) - .map(m -> m.requiredJsonField) - .distinct() - .collect(Collectors.toList()); - return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType(); - } - - private static MetadataConverter[] createMetadataConverters( - RowType jsonRowType, List requestedMetadata) { - return requestedMetadata.stream() - .map(m -> convert(jsonRowType, m)) - .toArray(MetadataConverter[]::new); - } - - private static MetadataConverter convert(RowType jsonRowType, ReadableMetadata metadata) { - final int pos = jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); - return new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData root, int unused) { - return metadata.converter.convert(root, pos); - } - }; - } - - // -------------------------------------------------------------------------------------------- - - /** - * Converter that extracts a metadata field from the row that comes out of the JSON schema and - * converts it to the desired data type. - */ - interface MetadataConverter extends Serializable { - - // Method for top-level access. - default Object convert(GenericRowData row) { - return convert(row, -1); - } - - Object convert(GenericRowData row, int pos); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java deleted file mode 100644 index 9f3ccad2d..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactory.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.canal; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.JsonFormatOptionsUtil; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DeserializationFormatFactory; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; - -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - -import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; -import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.DATABASE_INCLUDE; -import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.IGNORE_PARSE_ERRORS; -import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL; -import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.JSON_MAP_NULL_KEY_MODE; -import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.TABLE_INCLUDE; -import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.TIMESTAMP_FORMAT; - -/** - * Format factory for providing configured instances of Canal JSON to RowData {@link - * DeserializationSchema}. - */ -@Internal -public class CanalJsonFormatFactory - implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "canal-json"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateDecodingFormatOptions(formatOptions); - - final String database = formatOptions.getOptional(DATABASE_INCLUDE).orElse(null); - final String table = formatOptions.getOptional(TABLE_INCLUDE).orElse(null); - final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - final TimestampFormat timestampFormat = - JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - - return new CanalJsonDecodingFormat(database, table, ignoreParseErrors, timestampFormat); - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateEncodingFormatOptions(formatOptions); - - TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - JsonFormatOptions.MapNullKeyMode mapNullKeyMode = - JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); - String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); - - final boolean encodeDecimalAsPlainNumber = - formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - - return new EncodingFormat>() { - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new CanalJsonSerializationSchema( - rowType, - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return Collections.emptySet(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(IGNORE_PARSE_ERRORS); - options.add(TIMESTAMP_FORMAT); - options.add(DATABASE_INCLUDE); - options.add(TABLE_INCLUDE); - options.add(JSON_MAP_NULL_KEY_MODE); - options.add(JSON_MAP_NULL_KEY_LITERAL); - options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - return options; - } - - /** Validator for canal decoding format. */ - private static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions); - } - - /** Validator for canal encoding format. */ - private static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatOptions.java deleted file mode 100644 index 6fa11bcb9..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonFormatOptions.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.canal; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.formats.json.JsonFormatOptions; - -/** Option utils for canal-json format. */ -@PublicEvolving -public class CanalJsonFormatOptions { - - public static final ConfigOption IGNORE_PARSE_ERRORS = - JsonFormatOptions.IGNORE_PARSE_ERRORS; - - public static final ConfigOption TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT; - - public static final ConfigOption JSON_MAP_NULL_KEY_MODE = - JsonFormatOptions.MAP_NULL_KEY_MODE; - - public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = - JsonFormatOptions.MAP_NULL_KEY_LITERAL; - - public static final ConfigOption DATABASE_INCLUDE = - ConfigOptions.key("database.include") - .stringType() - .noDefaultValue() - .withDescription( - "An optional regular expression to only read the specific databases changelog rows by regular matching the \"database\" meta field in the Canal record." - + "The pattern string is compatible with Java's Pattern."); - - public static final ConfigOption TABLE_INCLUDE = - ConfigOptions.key("table.include") - .stringType() - .noDefaultValue() - .withDescription( - "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the Canal record." - + "The pattern string is compatible with Java's Pattern."); - - private CanalJsonFormatOptions() {} -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java deleted file mode 100644 index 362b9df6e..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/canal/CanalJsonSerializationSchema.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.canal; - -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.JsonRowDataSerializationSchema; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; - -import java.util.Objects; - -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; - -/** - * Serialization schema that serializes an object of Flink Table/SQL internal data structure {@link - * RowData} into a Canal JSON bytes. - * - * @see Alibaba Canal - */ -public class CanalJsonSerializationSchema implements SerializationSchema { - - private static final long serialVersionUID = 1L; - - private static final StringData OP_INSERT = StringData.fromString("INSERT"); - private static final StringData OP_DELETE = StringData.fromString("DELETE"); - - private transient GenericRowData reuse; - - /** The serializer to serialize Canal JSON data. */ - private final JsonRowDataSerializationSchema jsonSerializer; - - public CanalJsonSerializationSchema( - RowType rowType, - TimestampFormat timestampFormat, - JsonFormatOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral, - boolean encodeDecimalAsPlainNumber) { - jsonSerializer = - new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - } - - @Override - public void open(InitializationContext context) throws Exception { - jsonSerializer.open(context); - reuse = new GenericRowData(2); - } - - @Override - public byte[] serialize(RowData row) { - try { - StringData opType = rowKind2String(row.getRowKind()); - ArrayData arrayData = new GenericArrayData(new RowData[] {row}); - reuse.setField(0, arrayData); - reuse.setField(1, opType); - return jsonSerializer.serialize(reuse); - } catch (Throwable t) { - throw new RuntimeException("Could not serialize row '" + row + "'.", t); - } - } - - private StringData rowKind2String(RowKind rowKind) { - switch (rowKind) { - case INSERT: - case UPDATE_AFTER: - return OP_INSERT; - case UPDATE_BEFORE: - case DELETE: - return OP_DELETE; - default: - throw new UnsupportedOperationException( - "Unsupported operation '" + rowKind + "' for row kind."); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - CanalJsonSerializationSchema that = (CanalJsonSerializationSchema) o; - return Objects.equals(jsonSerializer, that.jsonSerializer); - } - - @Override - public int hashCode() { - return Objects.hash(jsonSerializer); - } - - private static RowType createJsonRowType(DataType databaseSchema) { - // Canal JSON contains other information, e.g. "database", "ts" - // but we don't need them - // and we don't need "old" , because can not support UPDATE_BEFORE,UPDATE_AFTER - return (RowType) - DataTypes.ROW( - DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)), - DataTypes.FIELD("type", DataTypes.STRING())) - .getLogicalType(); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java deleted file mode 100644 index 71d1ee9cc..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDecodingFormat.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.maxwell; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.maxwell.MaxwellJsonDeserializationSchema.MetadataConverter; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.Projection; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.ProjectableDecodingFormat; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.types.RowKind; - -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -/** {@link DecodingFormat} for Maxwell using JSON encoding. */ -public class MaxwellJsonDecodingFormat - implements ProjectableDecodingFormat> { - - // -------------------------------------------------------------------------------------------- - // Mutable attributes - // -------------------------------------------------------------------------------------------- - - /** The requested metadata keys. */ - private List metadataKeys; - - private final boolean ignoreParseErrors; - - private final TimestampFormat timestampFormat; - - public MaxwellJsonDecodingFormat(boolean ignoreParseErrors, TimestampFormat timestampFormat) { - this.ignoreParseErrors = ignoreParseErrors; - this.timestampFormat = timestampFormat; - this.metadataKeys = Collections.emptyList(); - } - - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) { - physicalDataType = Projection.of(projections).project(physicalDataType); - - final List readableMetadata = - metadataKeys.stream() - .map( - k -> - Stream.of(ReadableMetadata.values()) - .filter(rm -> rm.key.equals(k)) - .findFirst() - .orElseThrow( - () -> - new IllegalStateException( - String.format( - "Could not find the requested metadata key: %s", - k)))) - .collect(Collectors.toList()); - final List metadataFields = - readableMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList()); - final DataType producedDataType = - DataTypeUtils.appendRowFields(physicalDataType, metadataFields); - final TypeInformation producedTypeInfo = - context.createTypeInformation(producedDataType); - return new MaxwellJsonDeserializationSchema( - physicalDataType, - readableMetadata, - producedTypeInfo, - ignoreParseErrors, - timestampFormat); - } - - @Override - public Map listReadableMetadata() { - final Map metadataMap = new LinkedHashMap<>(); - Stream.of(ReadableMetadata.values()) - .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); - return metadataMap; - } - - @Override - public void applyReadableMetadata(List metadataKeys) { - this.metadataKeys = metadataKeys; - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - // -------------------------------------------------------------------------------------------- - // Metadata handling - // -------------------------------------------------------------------------------------------- - - /** List of metadata that can be read with this format. */ - enum ReadableMetadata { - DATABASE( - "database", - DataTypes.STRING().nullable(), - DataTypes.FIELD("database", DataTypes.STRING()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getString(pos); - } - }), - - TABLE( - "table", - DataTypes.STRING().nullable(), - DataTypes.FIELD("table", DataTypes.STRING()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getString(pos); - } - }), - - PRIMARY_KEY_COLUMNS( - "primary-key-columns", - DataTypes.ARRAY(DataTypes.STRING()).nullable(), - DataTypes.FIELD("primary_key_columns", DataTypes.ARRAY(DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getArray(pos); - } - }), - - INGESTION_TIMESTAMP( - "ingestion-timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), - DataTypes.FIELD("ts", DataTypes.BIGINT()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - if (row.isNullAt(pos)) { - return null; - } - // the `ts` field of Maxwell JSON is assigned with `timestampSeconds`. - return TimestampData.fromEpochMillis(row.getLong(pos) * 1000); - } - }); - - final String key; - - final DataType dataType; - - final DataTypes.Field requiredJsonField; - - final MetadataConverter converter; - - ReadableMetadata( - String key, - DataType dataType, - DataTypes.Field requiredJsonField, - MetadataConverter converter) { - this.key = key; - this.dataType = dataType; - this.requiredJsonField = requiredJsonField; - this.converter = converter; - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java deleted file mode 100644 index f58bfd4a1..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonDeserializationSchema.java +++ /dev/null @@ -1,282 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.maxwell; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; -import org.apache.flink.formats.json.maxwell.MaxwellJsonDecodingFormat.ReadableMetadata; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Collector; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; - -import java.io.IOException; -import java.io.Serializable; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; - -import static java.lang.String.format; - -/** - * Deserialization schema from Maxwell JSON to Flink Table/SQL internal data structure {@link - * RowData}. The deserialization schema knows Maxwell's schema definition and can extract the - * database data and convert into {@link RowData} with {@link RowKind}. - * - *

Deserializes a byte[] message as a JSON object and reads the specified fields. - * - *

Failures during deserialization are forwarded as wrapped IOExceptions. - * - * @see Maxwell - */ -public class MaxwellJsonDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 2L; - - private static final String FIELD_OLD = "old"; - private static final String OP_INSERT = "insert"; - private static final String OP_UPDATE = "update"; - private static final String OP_DELETE = "delete"; - - /** The deserializer to deserialize Maxwell JSON data. */ - private final JsonRowDataDeserializationSchema jsonDeserializer; - - /** Flag that indicates that an additional projection is required for metadata. */ - private final boolean hasMetadata; - - /** Metadata to be extracted for every record. */ - private final MetadataConverter[] metadataConverters; - - /** {@link TypeInformation} of the produced {@link RowData} (physical + meta data). */ - private final TypeInformation producedTypeInfo; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - /** Names of physical fields. */ - private final List fieldNames; - - /** Number of physical fields. */ - private final int fieldCount; - - public MaxwellJsonDeserializationSchema( - DataType physicalDataType, - List requestedMetadata, - TypeInformation producedTypeInfo, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata); - this.jsonDeserializer = - new JsonRowDataDeserializationSchema( - jsonRowType, - // the result type is never used, so it's fine to pass in the produced type - // info - producedTypeInfo, - // ignoreParseErrors already contains the functionality of - // failOnMissingField - false, - ignoreParseErrors, - timestampFormat); - this.hasMetadata = requestedMetadata.size() > 0; - this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata); - this.producedTypeInfo = producedTypeInfo; - this.ignoreParseErrors = ignoreParseErrors; - final RowType physicalRowType = ((RowType) physicalDataType.getLogicalType()); - this.fieldNames = physicalRowType.getFieldNames(); - this.fieldCount = physicalRowType.getFieldCount(); - } - - @Override - public void open(InitializationContext context) throws Exception { - jsonDeserializer.open(context); - } - - @Override - public RowData deserialize(byte[] message) throws IOException { - throw new RuntimeException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); - } - - @Override - public void deserialize(byte[] message, Collector out) throws IOException { - if (message == null || message.length == 0) { - return; - } - try { - final JsonNode root = jsonDeserializer.deserializeToJsonNode(message); - final GenericRowData row = (GenericRowData) jsonDeserializer.convertToRowData(root); - String type = row.getString(2).toString(); // "type" field - if (OP_INSERT.equals(type)) { - // "data" field is a row, contains inserted rows - GenericRowData insert = (GenericRowData) row.getRow(0, fieldCount); - insert.setRowKind(RowKind.INSERT); - emitRow(row, insert, out); - } else if (OP_UPDATE.equals(type)) { - // "data" field is a row, contains new rows - // "old" field is a row, contains old values - // the underlying JSON deserialization schema always produce GenericRowData. - GenericRowData after = (GenericRowData) row.getRow(0, fieldCount); // "data" field - GenericRowData before = (GenericRowData) row.getRow(1, fieldCount); // "old" field - final JsonNode oldField = root.get(FIELD_OLD); - for (int f = 0; f < fieldCount; f++) { - if (before.isNullAt(f) && oldField.findValue(fieldNames.get(f)) == null) { - // not null fields in "old" (before) means the fields are changed - // null/empty fields in "old" (before) means the fields are not changed - // so we just copy the not changed fields into before - before.setField(f, after.getField(f)); - } - } - before.setRowKind(RowKind.UPDATE_BEFORE); - after.setRowKind(RowKind.UPDATE_AFTER); - emitRow(row, before, out); - emitRow(row, after, out); - } else if (OP_DELETE.equals(type)) { - // "data" field is a row, contains deleted rows - GenericRowData delete = (GenericRowData) row.getRow(0, fieldCount); - delete.setRowKind(RowKind.DELETE); - emitRow(row, delete, out); - } else { - if (!ignoreParseErrors) { - throw new IOException( - format( - "Unknown \"type\" value \"%s\". The Maxwell JSON message is '%s'", - type, new String(message))); - } - } - } catch (Throwable t) { - // a big try catch to protect the processing. - if (!ignoreParseErrors) { - throw new IOException( - format("Corrupt Maxwell JSON message '%s'.", new String(message)), t); - } - } - } - - private void emitRow( - GenericRowData rootRow, GenericRowData physicalRow, Collector out) { - // shortcut in case no output projection is required - if (!hasMetadata) { - out.collect(physicalRow); - return; - } - final int metadataArity = metadataConverters.length; - final GenericRowData producedRow = - new GenericRowData(physicalRow.getRowKind(), fieldCount + metadataArity); - for (int physicalPos = 0; physicalPos < fieldCount; physicalPos++) { - producedRow.setField(physicalPos, physicalRow.getField(physicalPos)); - } - for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { - producedRow.setField( - fieldCount + metadataPos, metadataConverters[metadataPos].convert(rootRow)); - } - out.collect(producedRow); - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return producedTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - MaxwellJsonDeserializationSchema that = (MaxwellJsonDeserializationSchema) o; - return Objects.equals(jsonDeserializer, that.jsonDeserializer) - && hasMetadata == that.hasMetadata - && Objects.equals(producedTypeInfo, that.producedTypeInfo) - && ignoreParseErrors == that.ignoreParseErrors - && fieldCount == that.fieldCount; - } - - @Override - public int hashCode() { - return Objects.hash( - jsonDeserializer, hasMetadata, producedTypeInfo, ignoreParseErrors, fieldCount); - } - - // -------------------------------------------------------------------------------------------- - - private static RowType createJsonRowType( - DataType physicalDataType, List readableMetadata) { - DataType root = - DataTypes.ROW( - DataTypes.FIELD("data", physicalDataType), - DataTypes.FIELD("old", physicalDataType), - DataTypes.FIELD("type", DataTypes.STRING())); - // append fields that are required for reading metadata in the root - final List rootMetadataFields = - readableMetadata.stream() - .map(m -> m.requiredJsonField) - .distinct() - .collect(Collectors.toList()); - return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType(); - } - - private static MetadataConverter[] createMetadataConverters( - RowType jsonRowType, List requestedMetadata) { - return requestedMetadata.stream() - .map(m -> convert(jsonRowType, m)) - .toArray(MetadataConverter[]::new); - } - - private static MetadataConverter convert(RowType jsonRowType, ReadableMetadata metadata) { - final int pos = jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); - return new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData root, int unused) { - return metadata.converter.convert(root, pos); - } - }; - } - - // -------------------------------------------------------------------------------------------- - - /** - * Converter that extracts a metadata field from the row that comes out of the JSON schema and - * converts it to the desired data type. - */ - interface MetadataConverter extends Serializable { - - // Method for top-level access. - default Object convert(GenericRowData row) { - return convert(row, -1); - } - - Object convert(GenericRowData row, int pos); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java deleted file mode 100644 index 1bbbec844..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactory.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.maxwell; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.JsonFormatOptionsUtil; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DeserializationFormatFactory; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; - -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - -import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; -import static org.apache.flink.formats.json.maxwell.MaxwellJsonFormatOptions.IGNORE_PARSE_ERRORS; -import static org.apache.flink.formats.json.maxwell.MaxwellJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL; -import static org.apache.flink.formats.json.maxwell.MaxwellJsonFormatOptions.JSON_MAP_NULL_KEY_MODE; -import static org.apache.flink.formats.json.maxwell.MaxwellJsonFormatOptions.TIMESTAMP_FORMAT; - -/** - * Format factory for providing configured instances of Maxwell JSON to RowData {@link - * DeserializationSchema}. - */ -@Internal -public class MaxwellJsonFormatFactory - implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "maxwell-json"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateDecodingFormatOptions(formatOptions); - - final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - final TimestampFormat timestampFormat = - JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - - return new MaxwellJsonDecodingFormat(ignoreParseErrors, timestampFormat); - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateEncodingFormatOptions(formatOptions); - - TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - JsonFormatOptions.MapNullKeyMode mapNullKeyMode = - JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); - String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); - - final boolean encodeDecimalAsPlainNumber = - formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - - return new EncodingFormat>() { - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new MaxwellJsonSerializationSchema( - rowType, - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return Collections.emptySet(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(IGNORE_PARSE_ERRORS); - options.add(TIMESTAMP_FORMAT); - options.add(JSON_MAP_NULL_KEY_MODE); - options.add(JSON_MAP_NULL_KEY_LITERAL); - options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - return options; - } - - /** Validator for maxwell decoding format. */ - private static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions); - } - - /** Validator for maxwell encoding format. */ - private static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatOptions.java deleted file mode 100644 index 011fa7b79..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatOptions.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.maxwell; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.formats.json.JsonFormatOptions; - -/** Option utils for maxwell-json format. */ -@PublicEvolving -public class MaxwellJsonFormatOptions { - - public static final ConfigOption IGNORE_PARSE_ERRORS = - JsonFormatOptions.IGNORE_PARSE_ERRORS; - - public static final ConfigOption TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT; - - public static final ConfigOption JSON_MAP_NULL_KEY_MODE = - JsonFormatOptions.MAP_NULL_KEY_MODE; - - public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = - JsonFormatOptions.MAP_NULL_KEY_LITERAL; - - private MaxwellJsonFormatOptions() {} -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java deleted file mode 100644 index 1fe567b08..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerializationSchema.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.maxwell; - -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.JsonRowDataSerializationSchema; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; - -import java.util.Objects; - -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; - -/** - * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to - * maxwell-Json. - */ -public class MaxwellJsonSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = 1L; - - private static final StringData OP_INSERT = StringData.fromString("insert"); - private static final StringData OP_DELETE = StringData.fromString("delete"); - - private final JsonRowDataSerializationSchema jsonSerializer; - - /** Timestamp format specification which is used to parse timestamp. */ - private final TimestampFormat timestampFormat; - - private transient GenericRowData reuse; - - public MaxwellJsonSerializationSchema( - RowType rowType, - TimestampFormat timestampFormat, - JsonFormatOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral, - boolean encodeDecimalAsPlainNumber) { - this.jsonSerializer = - new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - this.timestampFormat = timestampFormat; - } - - @Override - public void open(InitializationContext context) throws Exception { - jsonSerializer.open(context); - this.reuse = new GenericRowData(2); - } - - @Override - public byte[] serialize(RowData element) { - reuse.setField(0, element); - reuse.setField(1, rowKind2String(element.getRowKind())); - return jsonSerializer.serialize(reuse); - } - - private StringData rowKind2String(RowKind rowKind) { - switch (rowKind) { - case INSERT: - case UPDATE_AFTER: - return OP_INSERT; - case UPDATE_BEFORE: - case DELETE: - return OP_DELETE; - default: - throw new UnsupportedOperationException( - "Unsupported operation '" + rowKind + "' for row kind."); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - MaxwellJsonSerializationSchema that = (MaxwellJsonSerializationSchema) o; - return Objects.equals(jsonSerializer, that.jsonSerializer) - && timestampFormat == that.timestampFormat; - } - - @Override - public int hashCode() { - return Objects.hash(jsonSerializer, timestampFormat); - } - - private RowType createJsonRowType(DataType databaseSchema) { - DataType payload = - DataTypes.ROW( - DataTypes.FIELD("data", databaseSchema), - DataTypes.FIELD("type", DataTypes.STRING())); - return (RowType) payload.getLogicalType(); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java deleted file mode 100644 index 6730fdca7..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDecodingFormat.java +++ /dev/null @@ -1,207 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.ogg; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.ogg.OggJsonDeserializationSchema.MetadataConverter; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.table.utils.DateTimeUtils; -import org.apache.flink.types.RowKind; - -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -/** {@link DecodingFormat} for Ogg using JSON encoding. */ -public class OggJsonDecodingFormat implements DecodingFormat> { - - // -------------------------------------------------------------------------------------------- - // Mutable attributes - // -------------------------------------------------------------------------------------------- - - private List metadataKeys; - - // -------------------------------------------------------------------------------------------- - // Ogg-specific attributes - // -------------------------------------------------------------------------------------------- - - private final boolean ignoreParseErrors; - private final TimestampFormat timestampFormat; - - public OggJsonDecodingFormat(boolean ignoreParseErrors, TimestampFormat timestampFormat) { - this.ignoreParseErrors = ignoreParseErrors; - this.timestampFormat = timestampFormat; - this.metadataKeys = Collections.emptyList(); - } - - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType physicalDataType) { - - final List readableMetadata = - metadataKeys.stream() - .map( - k -> - Stream.of(ReadableMetadata.values()) - .filter(rm -> rm.key.equals(k)) - .findFirst() - .orElseThrow(IllegalStateException::new)) - .collect(Collectors.toList()); - - final List metadataFields = - readableMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList()); - - final DataType producedDataType = - DataTypeUtils.appendRowFields(physicalDataType, metadataFields); - - final TypeInformation producedTypeInfo = - context.createTypeInformation(producedDataType); - - return new OggJsonDeserializationSchema( - physicalDataType, - readableMetadata, - producedTypeInfo, - ignoreParseErrors, - timestampFormat); - } - - // -------------------------------------------------------------------------------------------- - // Metadata handling - // -------------------------------------------------------------------------------------------- - - @Override - public Map listReadableMetadata() { - final Map metadataMap = new LinkedHashMap<>(); - Stream.of(ReadableMetadata.values()) - .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); - return metadataMap; - } - - @Override - public void applyReadableMetadata(List metadataKeys) { - this.metadataKeys = metadataKeys; - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - /** List of metadata that can be read with this format. */ - enum ReadableMetadata { - TABLE( - "table", - DataTypes.STRING().nullable(), - DataTypes.FIELD("table", DataTypes.STRING()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getString(pos); - } - }), - - PRIMARY_KEYS( - "primary-keys", - DataTypes.ARRAY(DataTypes.STRING()).nullable(), - DataTypes.FIELD("primary_keys", DataTypes.ARRAY(DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getArray(pos); - } - }), - - INGESTION_TIMESTAMP( - "ingestion-timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(6).nullable(), - DataTypes.FIELD("current_ts", DataTypes.STRING()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - if (row.isNullAt(pos)) { - return null; - } - // the timestamp follows the ISO-8601 format - return DateTimeUtils.parseTimestampData( - row.getString(pos).toString(), "yyyy-MM-dd'T'HH:mm:ss.SSSSSS"); - } - }), - - EVENT_TIMESTAMP( - "event-timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(6).nullable(), - DataTypes.FIELD("op_ts", DataTypes.STRING()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - if (row.isNullAt(pos)) { - return null; - } - // the timestamp format is like "yyyy-MM-dd HH:mm:ss.SSSSSS" - return DateTimeUtils.parseTimestampData(row.getString(pos).toString()); - } - }); - - final String key; - - final DataType dataType; - - final DataTypes.Field requiredJsonField; - - final MetadataConverter converter; - - ReadableMetadata( - String key, - DataType dataType, - DataTypes.Field requiredJsonField, - MetadataConverter converter) { - this.key = key; - this.dataType = dataType; - this.requiredJsonField = requiredJsonField; - this.converter = converter; - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java deleted file mode 100644 index 85bd52dc0..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java +++ /dev/null @@ -1,278 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.ogg; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; -import org.apache.flink.formats.json.ogg.OggJsonDecodingFormat.ReadableMetadata; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Collector; - -import java.io.IOException; -import java.io.Serializable; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; - -import static java.lang.String.format; - -/** - * Deserialization schema from Ogg JSON to Flink Table/SQL internal data structure {@link RowData}. - * The deserialization schema knows Ogg's schema definition and can extract the database data and - * convert into {@link RowData} with {@link RowKind}. - * - *

Deserializes a byte[] message as a JSON object and reads the specified fields. - * - *

Failures during deserialization are forwarded as wrapped IOExceptions. - * - * @see Ogg - */ -@Internal -public final class OggJsonDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - - private static final String OP_CREATE = "I"; // insert - private static final String OP_UPDATE = "U"; // update - private static final String OP_DELETE = "D"; // delete - private static final String OP_TRUNCATE = "T"; // truncate - - private static final String REPLICA_IDENTITY_EXCEPTION = - "The \"before\" field of %s message is null, " - + "if you are using Ogg Postgres Connector, " - + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; - - /** The deserializer to deserialize Ogg JSON data. */ - private final JsonRowDataDeserializationSchema jsonDeserializer; - - /** Flag that indicates that an additional projection is required for metadata. */ - private final boolean hasMetadata; - - /** Metadata to be extracted for every record. */ - private final MetadataConverter[] metadataConverters; - - /** {@link TypeInformation} of the produced {@link RowData} (physical + metadata). */ - private final TypeInformation producedTypeInfo; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - public OggJsonDeserializationSchema( - DataType physicalDataType, - List requestedMetadata, - TypeInformation producedTypeInfo, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata); - this.jsonDeserializer = - new JsonRowDataDeserializationSchema( - jsonRowType, - // the result type is never used, so it's fine to pass in the produced type - // info - producedTypeInfo, - false, // ignoreParseErrors already contains the functionality of - // failOnMissingField - ignoreParseErrors, - timestampFormat); - this.hasMetadata = requestedMetadata.size() > 0; - this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata); - this.producedTypeInfo = producedTypeInfo; - this.ignoreParseErrors = ignoreParseErrors; - } - - private static RowType createJsonRowType( - DataType physicalDataType, List readableMetadata) { - DataType root = - DataTypes.ROW( - DataTypes.FIELD("before", physicalDataType), - DataTypes.FIELD("after", physicalDataType), - DataTypes.FIELD("op_type", DataTypes.STRING())); - // append fields that are required for reading metadata in the root - final List rootMetadataFields = - readableMetadata.stream() - .map(m -> m.requiredJsonField) - .distinct() - .collect(Collectors.toList()); - return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType(); - } - - private static MetadataConverter[] createMetadataConverters( - RowType jsonRowType, List requestedMetadata) { - return requestedMetadata.stream() - .map(m -> convertInRoot(jsonRowType, m)) - .toArray(MetadataConverter[]::new); - } - - private static MetadataConverter convertInRoot(RowType jsonRowType, ReadableMetadata metadata) { - final int pos = findFieldPos(metadata, jsonRowType); - return new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData root, int unused) { - return metadata.converter.convert(root, pos); - } - }; - } - - private static int findFieldPos(ReadableMetadata metadata, RowType jsonRowType) { - return jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); - } - - @Override - public void open(InitializationContext context) throws Exception { - jsonDeserializer.open(context); - } - - @Override - public RowData deserialize(byte[] message) { - throw new RuntimeException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); - } - - @Override - public void deserialize(byte[] message, Collector out) throws IOException { - if (message == null || message.length == 0) { - // skip tombstone messages - return; - } - try { - GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); - - GenericRowData before = (GenericRowData) row.getField(0); - GenericRowData after = (GenericRowData) row.getField(1); - String op = row.getField(2).toString(); - if (OP_CREATE.equals(op)) { - after.setRowKind(RowKind.INSERT); - emitRow(row, after, out); - } else if (OP_UPDATE.equals(op)) { - if (before == null) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - before.setRowKind(RowKind.UPDATE_BEFORE); - after.setRowKind(RowKind.UPDATE_AFTER); - emitRow(row, before, out); - emitRow(row, after, out); - } else if (OP_DELETE.equals(op)) { - if (before == null) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); - } - before.setRowKind(RowKind.DELETE); - emitRow(row, before, out); - } else { - if (!ignoreParseErrors) { - throw new IOException( - format( - "Unknown \"op_type\" value \"%s\". The Ogg JSON message is '%s'", - op, new String(message))); - } - } - } catch (Throwable t) { - // a big try catch to protect the processing. - if (!ignoreParseErrors) { - throw new IOException( - format("Corrupt Ogg JSON message '%s'.", new String(message)), t); - } - } - } - - // -------------------------------------------------------------------------------------------- - - private void emitRow( - GenericRowData rootRow, GenericRowData physicalRow, Collector out) { - // shortcut in case no output projection is required - if (!hasMetadata) { - out.collect(physicalRow); - return; - } - - final int physicalArity = physicalRow.getArity(); - final int metadataArity = metadataConverters.length; - - final GenericRowData producedRow = - new GenericRowData(physicalRow.getRowKind(), physicalArity + metadataArity); - - for (int physicalPos = 0; physicalPos < physicalArity; physicalPos++) { - producedRow.setField(physicalPos, physicalRow.getField(physicalPos)); - } - - for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { - producedRow.setField( - physicalArity + metadataPos, metadataConverters[metadataPos].convert(rootRow)); - } - - out.collect(producedRow); - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return producedTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - OggJsonDeserializationSchema that = (OggJsonDeserializationSchema) o; - return Objects.equals(jsonDeserializer, that.jsonDeserializer) - && hasMetadata == that.hasMetadata - && Objects.equals(producedTypeInfo, that.producedTypeInfo) - && ignoreParseErrors == that.ignoreParseErrors; - } - - @Override - public int hashCode() { - return Objects.hash(jsonDeserializer, hasMetadata, producedTypeInfo, ignoreParseErrors); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Converter that extracts a metadata field from the row (root or payload) that comes out of the - * JSON schema and converts it to the desired data type. - */ - interface MetadataConverter extends Serializable { - - // Method for top-level access. - default Object convert(GenericRowData row) { - return convert(row, -1); - } - - Object convert(GenericRowData row, int pos); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java deleted file mode 100644 index f853983d4..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactory.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.ogg; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.JsonFormatOptionsUtil; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DeserializationFormatFactory; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; - -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - -import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; -import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.IGNORE_PARSE_ERRORS; -import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL; -import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.JSON_MAP_NULL_KEY_MODE; -import static org.apache.flink.formats.json.ogg.OggJsonFormatOptions.TIMESTAMP_FORMAT; - -/** - * Format factory for providing configured instances of Ogg JSON to RowData {@link - * DeserializationSchema}. - */ -@Internal -public class OggJsonFormatFactory - implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "ogg-json"; - - /** Validator for ogg decoding format. */ - private static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions); - } - - /** Validator for ogg encoding format. */ - private static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions); - } - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateDecodingFormatOptions(formatOptions); - - final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - - final TimestampFormat timestampFormat = - JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - - return new OggJsonDecodingFormat(ignoreParseErrors, timestampFormat); - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateEncodingFormatOptions(formatOptions); - - TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - JsonFormatOptions.MapNullKeyMode mapNullKeyMode = - JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); - String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); - - final boolean encodeDecimalAsPlainNumber = - formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - - return new EncodingFormat>() { - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new OggJsonSerializationSchema( - rowType, - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return Collections.emptySet(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(IGNORE_PARSE_ERRORS); - options.add(TIMESTAMP_FORMAT); - options.add(JSON_MAP_NULL_KEY_MODE); - options.add(JSON_MAP_NULL_KEY_LITERAL); - options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - return options; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java deleted file mode 100644 index feeca7975..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonFormatOptions.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.ogg; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.formats.json.JsonFormatOptions; - -/** Option utils for ogg-json format. */ -@PublicEvolving -public class OggJsonFormatOptions { - - public static final ConfigOption IGNORE_PARSE_ERRORS = - JsonFormatOptions.IGNORE_PARSE_ERRORS; - - public static final ConfigOption TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT; - - public static final ConfigOption JSON_MAP_NULL_KEY_MODE = - JsonFormatOptions.MAP_NULL_KEY_MODE; - - public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = - JsonFormatOptions.MAP_NULL_KEY_LITERAL; - - private OggJsonFormatOptions() {} -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java deleted file mode 100644 index 635ff3dc7..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/ogg/OggJsonSerializationSchema.java +++ /dev/null @@ -1,130 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.ogg; - -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.JsonRowDataSerializationSchema; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; - -import java.util.Objects; - -import static java.lang.String.format; -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; - -/** - * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Ogg JSON. - * - * @see Ogg - * JSON Message - */ -public class OggJsonSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = 1L; - - private static final StringData OP_INSERT = StringData.fromString("I"); // insert - private static final StringData OP_DELETE = StringData.fromString("D"); // delete - - /** The serializer to serialize Ogg JSON data. * */ - private final JsonRowDataSerializationSchema jsonSerializer; - - private transient GenericRowData genericRowData; - - public OggJsonSerializationSchema( - RowType rowType, - TimestampFormat timestampFormat, - JsonFormatOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral, - boolean encodeDecimalAsPlainNumber) { - jsonSerializer = - new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - } - - private static RowType createJsonRowType(DataType databaseSchema) { - // Ogg JSON contains some other information, e.g. "source", "ts_ms" - // but we don't need them. - return (RowType) - DataTypes.ROW( - DataTypes.FIELD("before", databaseSchema), - DataTypes.FIELD("after", databaseSchema), - DataTypes.FIELD("op_type", DataTypes.STRING())) - .getLogicalType(); - } - - @Override - public void open(InitializationContext context) throws Exception { - jsonSerializer.open(context); - genericRowData = new GenericRowData(3); - } - - @Override - public byte[] serialize(RowData rowData) { - try { - switch (rowData.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - genericRowData.setField(0, null); - genericRowData.setField(1, rowData); - genericRowData.setField(2, OP_INSERT); - return jsonSerializer.serialize(genericRowData); - case UPDATE_BEFORE: - case DELETE: - genericRowData.setField(0, rowData); - genericRowData.setField(1, null); - genericRowData.setField(2, OP_DELETE); - return jsonSerializer.serialize(genericRowData); - default: - throw new UnsupportedOperationException( - format( - "Unsupported operation '%s' for row kind.", - rowData.getRowKind())); - } - } catch (Throwable t) { - throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - OggJsonSerializationSchema that = (OggJsonSerializationSchema) o; - return Objects.equals(jsonSerializer, that.jsonSerializer); - } - - @Override - public int hashCode() { - return Objects.hash(jsonSerializer); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 35e647de6..3b83658fc 100644 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,8 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.formats.json.JsonFormatFactory org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory -org.apache.flink.formats.json.canal.CanalJsonFormatFactory -org.apache.flink.formats.json.maxwell.MaxwellJsonFormatFactory -org.apache.flink.formats.json.ogg.OggJsonFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java deleted file mode 100644 index b2efe8c86..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.table.api.TableResult; -import org.apache.flink.table.planner.runtime.batch.sql.BatchFileSystemITCaseBase; -import org.apache.flink.table.utils.LegacyRowResource; -import org.apache.flink.types.Row; -import org.apache.flink.util.FileUtils; - -import org.junit.Rule; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.io.PrintWriter; -import java.net.URI; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - -/** ITCase to test json format for {@link JsonFormatFactory}. */ -public class JsonBatchFileSystemITCase extends BatchFileSystemITCaseBase { - - @Rule public final LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE; - - @Override - public String[] formatProperties() { - List ret = new ArrayList<>(); - ret.add("'format'='json'"); - ret.add("'json.ignore-parse-errors'='true'"); - return ret.toArray(new String[0]); - } - - @Test - public void testParseError() throws Exception { - String path = new URI(resultPath()).getPath(); - new File(path).mkdirs(); - File file = new File(path, "temp_file"); - file.createNewFile(); - FileUtils.writeFileUtf8( - file, - "{\"x\":\"x5\",\"y\":5,\"a\":1,\"b\":1}\n" - + "{I am a wrong json.}\n" - + "{\"x\":\"x5\",\"y\":5,\"a\":1,\"b\":1}"); - - check( - "select * from nonPartitionedTable", - Arrays.asList(Row.of("x5,5,1,1"), Row.of("x5,5,1,1"))); - } - - @Test - public void bigDataTest() throws IOException { - int numRecords = 1000; - File dir = generateTestData(numRecords); - - env().setParallelism(1); - - String sql = - String.format( - "CREATE TABLE bigdata_source ( " - + " id INT, " - + " content STRING" - + ") PARTITIONED by (id) WITH (" - + " 'connector' = 'filesystem'," - + " 'path' = '%s'," - + " 'format' = 'json'" - + ")", - dir); - tEnv().executeSql(sql); - TableResult result = tEnv().executeSql("select * from bigdata_source"); - List elements = new ArrayList<>(); - result.collect().forEachRemaining(r -> elements.add((String) r.getField(1))); - assertThat(elements).hasSize(numRecords); - elements.sort(String::compareTo); - - List expected = new ArrayList<>(); - for (int i = 0; i < numRecords; i++) { - expected.add(String.valueOf(i)); - } - expected.sort(String::compareTo); - - assertThat(elements).isEqualTo(expected); - } - - private static File generateTestData(int numRecords) throws IOException { - File tempDir = TEMPORARY_FOLDER.newFolder(); - - File root = new File(tempDir, "id=0"); - root.mkdir(); - - File dataFile = new File(root, "testdata"); - try (PrintWriter writer = new PrintWriter(dataFile)) { - for (int i = 0; i < numRecords; ++i) { - writer.println(String.format("{\"content\":\"%s\"}", i)); - } - } - - return tempDir; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java deleted file mode 100644 index 9e2cde3a2..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFileCompactionITCase.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.table.planner.runtime.stream.sql.FileCompactionITCaseBase; - -/** Compaction it case for json. */ -public class JsonFileCompactionITCase extends FileCompactionITCaseBase { - - @Override - protected String format() { - return "json"; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java deleted file mode 100644 index 7c1c553a6..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFormatFactoryTest.java +++ /dev/null @@ -1,232 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.TestDynamicTableFactory; -import org.apache.flink.table.factories.utils.FactoryMocks; -import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; -import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; - -import org.assertj.core.api.AbstractThrowableAssert; -import org.junit.jupiter.api.Test; - -import java.util.HashMap; -import java.util.Map; -import java.util.function.Consumer; - -import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; -import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; -import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; -import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for the {@link JsonFormatFactory}. */ -class JsonFormatFactoryTest { - - @Test - void testSeDeSchema() { - final Map tableOptions = getAllOptions(); - - testSchemaSerializationSchema(tableOptions); - testSchemaDeserializationSchema(tableOptions); - } - - @Test - void testFailOnMissingField() { - final Map tableOptions = - getModifyOptions(options -> options.put("json.fail-on-missing-field", "true")); - - assertThatCreateRuntimeDecoder(tableOptions) - .satisfies( - anyCauseMatches( - ValidationException.class, - "fail-on-missing-field and ignore-parse-errors shouldn't both be true.")); - } - - @Test - void testInvalidOptionForIgnoreParseErrors() { - final Map tableOptions = - getModifyOptions(options -> options.put("json.ignore-parse-errors", "abc")); - - assertThatCreateRuntimeDecoder(tableOptions) - .satisfies( - anyCauseMatches( - IllegalArgumentException.class, - "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)")); - } - - @Test - void testInvalidOptionForTimestampFormat() { - final Map tableOptions = - getModifyOptions(options -> options.put("json.timestamp-format.standard", "test")); - - assertThatCreateRuntimeDecoder(tableOptions) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'test' for timestamp-format.standard. Supported values are [SQL, ISO-8601].")); - } - - @Test - void testLowerCaseOptionForTimestampFormat() { - final Map tableOptions = - getModifyOptions( - options -> options.put("json.timestamp-format.standard", "iso-8601")); - - assertThatCreateRuntimeDecoder(tableOptions) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'iso-8601' for timestamp-format.standard. Supported values are [SQL, ISO-8601].")); - } - - @Test - void testInvalidOptionForMapNullKeyMode() { - final Map tableOptions = - getModifyOptions(options -> options.put("json.map-null-key.mode", "invalid")); - - assertThatCreateRuntimeEncoder(tableOptions) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'invalid' for option map-null-key.mode. Supported values are [LITERAL, FAIL, DROP].")); - } - - @Test - void testLowerCaseOptionForMapNullKeyMode() { - final Map tableOptions = - getModifyOptions(options -> options.put("json.map-null-key.mode", "fail")); - - testSchemaDeserializationSchema(tableOptions); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - private AbstractThrowableAssert assertThatCreateRuntimeDecoder( - Map options) { - return assertThatThrownBy( - () -> - createTableSource(options) - .valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, - SCHEMA.toPhysicalRowDataType())); - } - - private AbstractThrowableAssert assertThatCreateRuntimeEncoder( - Map options) { - return assertThatThrownBy( - () -> - createTableSink(options) - .valueFormat - .createRuntimeEncoder( - new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE)); - } - - private void testSchemaDeserializationSchema(Map options) { - final JsonRowDataDeserializationSchema expectedDeser = - new JsonRowDataDeserializationSchema( - PHYSICAL_TYPE, - InternalTypeInfo.of(PHYSICAL_TYPE), - false, - true, - TimestampFormat.ISO_8601); - - DeserializationSchema actualDeser = - createTableSource(options) - .valueFormat - .createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, - SCHEMA.toPhysicalRowDataType()); - - assertThat(actualDeser).isEqualTo(expectedDeser); - } - - private void testSchemaSerializationSchema(Map options) { - final JsonRowDataSerializationSchema expectedSer = - new JsonRowDataSerializationSchema( - PHYSICAL_TYPE, - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - - SerializationSchema actualSer = - createTableSink(options) - .valueFormat - .createRuntimeEncoder( - new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); - - assertThat(actualSer).isEqualTo(expectedSer); - } - - private TestDynamicTableFactory.DynamicTableSinkMock createTableSink( - Map options) { - final DynamicTableSink actualSink = FactoryMocks.createTableSink(SCHEMA, options); - assertThat(actualSink).isInstanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class); - - return (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - } - - private TestDynamicTableFactory.DynamicTableSourceMock createTableSource( - Map options) { - final DynamicTableSource actualSource = FactoryMocks.createTableSource(SCHEMA, options); - assertThat(actualSource).isInstanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class); - - return (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - } - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifyOptions(Consumer> optionModifier) { - Map options = getAllOptions(); - optionModifier.accept(options); - return options; - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", JsonFormatFactory.IDENTIFIER); - options.put("json.fail-on-missing-field", "false"); - options.put("json.ignore-parse-errors", "true"); - options.put("json.timestamp-format.standard", "ISO-8601"); - options.put("json.map-null-key.mode", "LITERAL"); - options.put("json.map-null-key.literal", "null"); - options.put("json.encode.decimal-as-plain-number", "true"); - return options; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java deleted file mode 100644 index 253707a30..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonFsStreamSinkITCase.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.table.planner.runtime.stream.FsStreamingSinkITCaseBase; - -import java.util.ArrayList; -import java.util.List; - -/** Test checkpoint for file system table factory with json format. */ -public class JsonFsStreamSinkITCase extends FsStreamingSinkITCaseBase { - - @Override - public String[] additionalProperties() { - List ret = new ArrayList<>(); - ret.add("'format'='json'"); - // for test purpose - ret.add("'sink.rolling-policy.file-size'='1b'"); - return ret.toArray(new String[0]); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java deleted file mode 100644 index e6b2a3e05..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonNodeDeserializationSchemaTest.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.connector.testutils.formats.DummyInitializationContext; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; - -import org.junit.jupiter.api.Test; - -import java.io.IOException; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for the {@link JsonNodeDeserializationSchema}. */ -@SuppressWarnings("deprecation") -class JsonNodeDeserializationSchemaTest { - - @Test - void testDeserialize() throws IOException { - ObjectMapper mapper = JacksonMapperFactory.createObjectMapper(); - ObjectNode initialValue = mapper.createObjectNode(); - initialValue.put("key", 4).put("value", "world"); - byte[] serializedValue = mapper.writeValueAsBytes(initialValue); - - JsonNodeDeserializationSchema schema = new JsonNodeDeserializationSchema(); - schema.open(new DummyInitializationContext()); - ObjectNode deserializedValue = schema.deserialize(serializedValue); - - assertThat(deserializedValue.get("key").asInt()).isEqualTo(4); - assertThat(deserializedValue.get("value").asText()).isEqualTo("world"); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java deleted file mode 100644 index 883c3f091..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ /dev/null @@ -1,872 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.connector.testutils.formats.DummyInitializationContext; -import org.apache.flink.core.testutils.FlinkAssertions; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.Row; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; - -import org.junit.jupiter.api.Test; - -import java.math.BigDecimal; -import java.sql.Timestamp; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.ZoneOffset; -import java.util.Arrays; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.ThreadLocalRandom; - -import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; -import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; -import static org.apache.flink.table.api.DataTypes.ARRAY; -import static org.apache.flink.table.api.DataTypes.BIGINT; -import static org.apache.flink.table.api.DataTypes.BOOLEAN; -import static org.apache.flink.table.api.DataTypes.BYTES; -import static org.apache.flink.table.api.DataTypes.DATE; -import static org.apache.flink.table.api.DataTypes.DECIMAL; -import static org.apache.flink.table.api.DataTypes.DOUBLE; -import static org.apache.flink.table.api.DataTypes.FIELD; -import static org.apache.flink.table.api.DataTypes.FLOAT; -import static org.apache.flink.table.api.DataTypes.INT; -import static org.apache.flink.table.api.DataTypes.MAP; -import static org.apache.flink.table.api.DataTypes.MULTISET; -import static org.apache.flink.table.api.DataTypes.ROW; -import static org.apache.flink.table.api.DataTypes.SMALLINT; -import static org.apache.flink.table.api.DataTypes.STRING; -import static org.apache.flink.table.api.DataTypes.TIME; -import static org.apache.flink.table.api.DataTypes.TIMESTAMP; -import static org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE; -import static org.apache.flink.table.api.DataTypes.TINYINT; -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** - * Tests for {@link JsonRowDataDeserializationSchema} and {@link JsonRowDataSerializationSchema}. - */ -class JsonRowDataSerDeSchemaTest { - - private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper(); - - @Test - void testSerDe() throws Exception { - byte tinyint = 'c'; - short smallint = 128; - int intValue = 45536; - float floatValue = 33.333F; - long bigint = 1238123899121L; - String name = "asdlkjasjkdla998y1122"; - byte[] bytes = new byte[1024]; - ThreadLocalRandom.current().nextBytes(bytes); - BigDecimal decimal = new BigDecimal("123.456789"); - Double[] doubles = new Double[] {1.1, 2.2, 3.3}; - LocalDate date = LocalDate.parse("1990-10-14"); - LocalTime time = LocalTime.parse("12:12:43"); - Timestamp timestamp3 = Timestamp.valueOf("1990-10-14 12:12:43.123"); - Timestamp timestamp9 = Timestamp.valueOf("1990-10-14 12:12:43.123456789"); - Instant timestampWithLocalZone = - LocalDateTime.of(1990, 10, 14, 12, 12, 43, 123456789) - .atOffset(ZoneOffset.of("Z")) - .toInstant(); - - Map map = new HashMap<>(); - map.put("element", 123L); - - Map multiSet = new HashMap<>(); - multiSet.put("element", 2); - - Map> nestedMap = new HashMap<>(); - Map innerMap = new HashMap<>(); - innerMap.put("key", 234); - nestedMap.put("inner_map", innerMap); - - ArrayNode doubleNode = OBJECT_MAPPER.createArrayNode().add(1.1D).add(2.2D).add(3.3D); - - // Root - ObjectNode root = OBJECT_MAPPER.createObjectNode(); - root.put("bool", true); - root.put("tinyint", tinyint); - root.put("smallint", smallint); - root.put("int", intValue); - root.put("bigint", bigint); - root.put("float", floatValue); - root.put("name", name); - root.put("bytes", bytes); - root.put("decimal", decimal); - root.set("doubles", doubleNode); - root.put("date", "1990-10-14"); - root.put("time", "12:12:43"); - root.put("timestamp3", "1990-10-14T12:12:43.123"); - root.put("timestamp9", "1990-10-14T12:12:43.123456789"); - root.put("timestampWithLocalZone", "1990-10-14T12:12:43.123456789Z"); - root.putObject("map").put("element", 123); - root.putObject("multiSet").put("element", 2); - root.putObject("map2map").putObject("inner_map").put("key", 234); - - byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); - - DataType dataType = - ROW( - FIELD("bool", BOOLEAN()), - FIELD("tinyint", TINYINT()), - FIELD("smallint", SMALLINT()), - FIELD("int", INT()), - FIELD("bigint", BIGINT()), - FIELD("float", FLOAT()), - FIELD("name", STRING()), - FIELD("bytes", BYTES()), - FIELD("decimal", DECIMAL(9, 6)), - FIELD("doubles", ARRAY(DOUBLE())), - FIELD("date", DATE()), - FIELD("time", TIME(0)), - FIELD("timestamp3", TIMESTAMP(3)), - FIELD("timestamp9", TIMESTAMP(9)), - FIELD("timestampWithLocalZone", TIMESTAMP_WITH_LOCAL_TIME_ZONE(9)), - FIELD("map", MAP(STRING(), BIGINT())), - FIELD("multiSet", MULTISET(STRING())), - FIELD("map2map", MAP(STRING(), MAP(STRING(), INT())))); - RowType schema = (RowType) dataType.getLogicalType(); - TypeInformation resultTypeInfo = InternalTypeInfo.of(schema); - - JsonRowDataDeserializationSchema deserializationSchema = - new JsonRowDataDeserializationSchema( - schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); - open(deserializationSchema); - - Row expected = new Row(18); - expected.setField(0, true); - expected.setField(1, tinyint); - expected.setField(2, smallint); - expected.setField(3, intValue); - expected.setField(4, bigint); - expected.setField(5, floatValue); - expected.setField(6, name); - expected.setField(7, bytes); - expected.setField(8, decimal); - expected.setField(9, doubles); - expected.setField(10, date); - expected.setField(11, time); - expected.setField(12, timestamp3.toLocalDateTime()); - expected.setField(13, timestamp9.toLocalDateTime()); - expected.setField(14, timestampWithLocalZone); - expected.setField(15, map); - expected.setField(16, multiSet); - expected.setField(17, nestedMap); - - RowData rowData = deserializationSchema.deserialize(serializedJson); - Row actual = convertToExternal(rowData, dataType); - assertThat(actual).isEqualTo(expected); - - // test serialization - JsonRowDataSerializationSchema serializationSchema = - new JsonRowDataSerializationSchema( - schema, - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - open(serializationSchema); - - byte[] actualBytes = serializationSchema.serialize(rowData); - assertThat(serializedJson).containsExactly(actualBytes); - } - - /** - * Tests the deserialization slow path, e.g. convert into string and use {@link - * Double#parseDouble(String)}. - */ - @Test - void testSlowDeserialization() throws Exception { - Random random = new Random(); - boolean bool = random.nextBoolean(); - int integer = random.nextInt(); - long bigint = random.nextLong(); - double doubleValue = random.nextDouble(); - float floatValue = random.nextFloat(); - - ObjectNode root = OBJECT_MAPPER.createObjectNode(); - root.put("bool", String.valueOf(bool)); - root.put("int", String.valueOf(integer)); - root.put("bigint", String.valueOf(bigint)); - root.put("double1", String.valueOf(doubleValue)); - root.put("double2", new BigDecimal(doubleValue)); - root.put("float1", String.valueOf(floatValue)); - root.put("float2", new BigDecimal(floatValue)); - - byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); - - DataType dataType = - ROW( - FIELD("bool", BOOLEAN()), - FIELD("int", INT()), - FIELD("bigint", BIGINT()), - FIELD("double1", DOUBLE()), - FIELD("double2", DOUBLE()), - FIELD("float1", FLOAT()), - FIELD("float2", FLOAT())); - RowType rowType = (RowType) dataType.getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = - new JsonRowDataDeserializationSchema( - rowType, - InternalTypeInfo.of(rowType), - false, - false, - TimestampFormat.ISO_8601); - open(deserializationSchema); - - Row expected = new Row(7); - expected.setField(0, bool); - expected.setField(1, integer); - expected.setField(2, bigint); - expected.setField(3, doubleValue); - expected.setField(4, doubleValue); - expected.setField(5, floatValue); - expected.setField(6, floatValue); - - RowData rowData = deserializationSchema.deserialize(serializedJson); - Row actual = convertToExternal(rowData, dataType); - assertThat(actual).isEqualTo(expected); - } - - @Test - void testSerDeMultiRows() throws Exception { - RowType rowType = - (RowType) - ROW( - FIELD("f1", INT()), - FIELD("f2", BOOLEAN()), - FIELD("f3", STRING()), - FIELD("f4", MAP(STRING(), STRING())), - FIELD("f5", ARRAY(STRING())), - FIELD("f6", ROW(FIELD("f1", STRING()), FIELD("f2", INT())))) - .getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = - new JsonRowDataDeserializationSchema( - rowType, - InternalTypeInfo.of(rowType), - false, - false, - TimestampFormat.ISO_8601); - open(deserializationSchema); - JsonRowDataSerializationSchema serializationSchema = - new JsonRowDataSerializationSchema( - rowType, - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - open(serializationSchema); - - // the first row - { - ObjectNode root = OBJECT_MAPPER.createObjectNode(); - root.put("f1", 1); - root.put("f2", true); - root.put("f3", "str"); - ObjectNode map = root.putObject("f4"); - map.put("hello1", "flink"); - ArrayNode array = root.putArray("f5"); - array.add("element1"); - array.add("element2"); - ObjectNode row = root.putObject("f6"); - row.put("f1", "this is row1"); - row.put("f2", 12); - byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); - RowData rowData = deserializationSchema.deserialize(serializedJson); - byte[] actual = serializationSchema.serialize(rowData); - assertThat(serializedJson).containsExactly(actual); - } - - // the second row - { - ObjectNode root = OBJECT_MAPPER.createObjectNode(); - root.put("f1", 10); - root.put("f2", false); - root.put("f3", "newStr"); - ObjectNode map = root.putObject("f4"); - map.put("hello2", "json"); - ArrayNode array = root.putArray("f5"); - array.add("element3"); - array.add("element4"); - ObjectNode row = root.putObject("f6"); - row.put("f1", "this is row2"); - row.putNull("f2"); - byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); - RowData rowData = deserializationSchema.deserialize(serializedJson); - byte[] actual = serializationSchema.serialize(rowData); - assertThat(serializedJson).containsExactly(actual); - } - } - - @Test - void testSerDeMultiRowsWithNullValues() throws Exception { - String[] jsons = - new String[] { - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{\"k1\":10.01,\"k2\":\"invalid\"}}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, " - + "\"ids\":[1, 2, 3]}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"metrics\":{}}", - }; - - String[] expected = - new String[] { - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null,\"metrics\":{\"k1\":10.01,\"k2\":null}}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}," - + "\"ids\":[1,2,3],\"metrics\":null}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null,\"metrics\":{}}", - }; - - RowType rowType = - (RowType) - ROW( - FIELD("svt", STRING()), - FIELD("ops", ROW(FIELD("id", STRING()))), - FIELD("ids", ARRAY(INT())), - FIELD("metrics", MAP(STRING(), DOUBLE()))) - .getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = - new JsonRowDataDeserializationSchema( - rowType, - InternalTypeInfo.of(rowType), - false, - true, - TimestampFormat.ISO_8601); - open(deserializationSchema); - JsonRowDataSerializationSchema serializationSchema = - new JsonRowDataSerializationSchema( - rowType, - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - open(serializationSchema); - - for (int i = 0; i < jsons.length; i++) { - String json = jsons[i]; - RowData row = deserializationSchema.deserialize(json.getBytes()); - String result = new String(serializationSchema.serialize(row)); - assertThat(result).isEqualTo(expected[i]); - } - } - - @Test - void testDeserializationNullRow() throws Exception { - DataType dataType = ROW(FIELD("name", STRING())); - RowType schema = (RowType) dataType.getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = - new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); - open(deserializationSchema); - - assertThat(deserializationSchema.deserialize(null)).isNull(); - } - - @Test - void testDeserializationMissingNode() throws Exception { - DataType dataType = ROW(FIELD("name", STRING())); - RowType schema = (RowType) dataType.getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = - new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); - open(deserializationSchema); - RowData rowData = deserializationSchema.deserialize("".getBytes()); - assertThat(rowData).isNull(); - } - - @Test - void testDeserializationMissingField() throws Exception { - // Root - ObjectNode root = OBJECT_MAPPER.createObjectNode(); - root.put("id", 123123123); - byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); - - DataType dataType = ROW(FIELD("name", STRING())); - RowType schema = (RowType) dataType.getLogicalType(); - - // pass on missing field - JsonRowDataDeserializationSchema deserializationSchema = - new JsonRowDataDeserializationSchema( - schema, - InternalTypeInfo.of(schema), - false, - false, - TimestampFormat.ISO_8601); - open(deserializationSchema); - - Row expected = new Row(1); - Row actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); - assertThat(actual).isEqualTo(expected); - - // fail on missing field - deserializationSchema = - new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), true, false, TimestampFormat.ISO_8601); - open(deserializationSchema); - - String errorMessage = "Failed to deserialize JSON '{\"id\":123123123}'."; - - JsonRowDataDeserializationSchema finalDeserializationSchema = deserializationSchema; - assertThatThrownBy(() -> finalDeserializationSchema.deserialize(serializedJson)) - .hasMessage(errorMessage); - - // ignore on parse error - deserializationSchema = - new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), false, true, TimestampFormat.ISO_8601); - open(deserializationSchema); - actual = convertToExternal(deserializationSchema.deserialize(serializedJson), dataType); - assertThat(actual).isEqualTo(expected); - - errorMessage = - "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."; - assertThatThrownBy( - () -> - new JsonRowDataDeserializationSchema( - schema, - InternalTypeInfo.of(schema), - true, - true, - TimestampFormat.ISO_8601)) - .hasMessage(errorMessage); - } - - @Test - void testSerDeSQLTimestampFormat() throws Exception { - RowType rowType = - (RowType) - ROW( - FIELD("timestamp3", TIMESTAMP(3)), - FIELD("timestamp9", TIMESTAMP(9)), - FIELD( - "timestamp_with_local_timezone3", - TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), - FIELD( - "timestamp_with_local_timezone9", - TIMESTAMP_WITH_LOCAL_TIME_ZONE(9))) - .getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = - new JsonRowDataDeserializationSchema( - rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); - open(deserializationSchema); - JsonRowDataSerializationSchema serializationSchema = - new JsonRowDataSerializationSchema( - rowType, - TimestampFormat.SQL, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - open(serializationSchema); - - ObjectNode root = OBJECT_MAPPER.createObjectNode(); - root.put("timestamp3", "1990-10-14 12:12:43.123"); - root.put("timestamp9", "1990-10-14 12:12:43.123456789"); - root.put("timestamp_with_local_timezone3", "1990-10-14 12:12:43.123Z"); - root.put("timestamp_with_local_timezone9", "1990-10-14 12:12:43.123456789Z"); - byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); - RowData rowData = deserializationSchema.deserialize(serializedJson); - byte[] actual = serializationSchema.serialize(rowData); - assertThat(serializedJson).containsExactly(actual); - } - - @Test - void testSerializationMapNullKey() { - RowType rowType = - (RowType) - ROW(FIELD("nestedMap", MAP(STRING(), MAP(STRING(), INT())))) - .getLogicalType(); - - // test data - // use LinkedHashMap to make sure entries order - Map map = new LinkedHashMap<>(); - map.put(StringData.fromString("no-null key"), 1); - map.put(StringData.fromString(null), 2); - GenericMapData mapData = new GenericMapData(map); - - Map nestedMap = new LinkedHashMap<>(); - nestedMap.put(StringData.fromString("no-null key"), mapData); - nestedMap.put(StringData.fromString(null), mapData); - - GenericMapData nestedMapData = new GenericMapData(nestedMap); - GenericRowData rowData = new GenericRowData(1); - rowData.setField(0, nestedMapData); - - JsonRowDataSerializationSchema serializationSchema1 = - new JsonRowDataSerializationSchema( - rowType, - TimestampFormat.SQL, - JsonFormatOptions.MapNullKeyMode.FAIL, - "null", - true); - open(serializationSchema1); - // expect message for serializationSchema1 - String errorMessage1 = - "JSON format doesn't support to serialize map data with null keys." - + " You can drop null key entries or encode null in literals by specifying map-null-key.mode option."; - - JsonRowDataSerializationSchema serializationSchema2 = - new JsonRowDataSerializationSchema( - rowType, - TimestampFormat.SQL, - JsonFormatOptions.MapNullKeyMode.DROP, - "null", - true); - open(serializationSchema2); - // expect result for serializationSchema2 - String expectResult2 = "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1}}}"; - - JsonRowDataSerializationSchema serializationSchema3 = - new JsonRowDataSerializationSchema( - rowType, - TimestampFormat.SQL, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "nullKey", - true); - open(serializationSchema3); - // expect result for serializationSchema3 - String expectResult3 = - "{\"nestedMap\":{\"no-null key\":{\"no-null key\":1,\"nullKey\":2},\"nullKey\":{\"no-null key\":1,\"nullKey\":2}}}"; - - assertThatThrownBy(() -> serializationSchema1.serialize(rowData)) - .satisfies(FlinkAssertions.anyCauseMatches(errorMessage1)); - - // mapNullKey Mode is drop - byte[] actual2 = serializationSchema2.serialize(rowData); - assertThat(new String(actual2)).isEqualTo(expectResult2); - - // mapNullKey Mode is literal - byte[] actual3 = serializationSchema3.serialize(rowData); - assertThat(new String(actual3)).isEqualTo(expectResult3); - } - - @Test - void testSerializationDecimalEncode() throws Exception { - RowType schema = - (RowType) - ROW( - FIELD("decimal1", DECIMAL(9, 6)), - FIELD("decimal2", DECIMAL(20, 0)), - FIELD("decimal3", DECIMAL(11, 9))) - .getLogicalType(); - - TypeInformation resultTypeInfo = InternalTypeInfo.of(schema); - - JsonRowDataDeserializationSchema deserializer = - new JsonRowDataDeserializationSchema( - schema, resultTypeInfo, false, false, TimestampFormat.ISO_8601); - deserializer.open(new DummyInitializationContext()); - - JsonRowDataSerializationSchema plainDecimalSerializer = - new JsonRowDataSerializationSchema( - schema, - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - plainDecimalSerializer.open(new DummyInitializationContext()); - JsonRowDataSerializationSchema scientificDecimalSerializer = - new JsonRowDataSerializationSchema( - schema, - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - false); - scientificDecimalSerializer.open(new DummyInitializationContext()); - - String plainDecimalJson = - "{\"decimal1\":123.456789,\"decimal2\":454621864049246170,\"decimal3\":0.000000027}"; - RowData rowData = deserializer.deserialize(plainDecimalJson.getBytes()); - - String plainDecimalResult = new String(plainDecimalSerializer.serialize(rowData)); - assertThat(plainDecimalResult).isEqualTo(plainDecimalJson); - - String scientificDecimalJson = - "{\"decimal1\":123.456789,\"decimal2\":4.5462186404924617E+17,\"decimal3\":2.7E-8}"; - - String scientificDecimalResult = new String(scientificDecimalSerializer.serialize(rowData)); - assertThat(scientificDecimalResult).isEqualTo(scientificDecimalJson); - } - - @Test - void testJsonParse() throws Exception { - for (TestSpec spec : testData) { - testIgnoreParseErrors(spec); - if (spec.errorMessage != null) { - testParseErrors(spec); - } - } - } - - @Test - void testSerializationWithTypesMismatch() { - RowType rowType = (RowType) ROW(FIELD("f0", INT()), FIELD("f1", STRING())).getLogicalType(); - GenericRowData genericRowData = new GenericRowData(2); - genericRowData.setField(0, 1); - genericRowData.setField(1, 1); - JsonRowDataSerializationSchema serializationSchema = - new JsonRowDataSerializationSchema( - rowType, - TimestampFormat.SQL, - JsonFormatOptions.MapNullKeyMode.FAIL, - "null", - true); - open(serializationSchema); - String errorMessage = "Fail to serialize at field: f1."; - - assertThatThrownBy(() -> serializationSchema.serialize(genericRowData)) - .satisfies(anyCauseMatches(RuntimeException.class, errorMessage)); - } - - @Test - void testDeserializationWithTypesMismatch() { - RowType rowType = (RowType) ROW(FIELD("f0", STRING()), FIELD("f1", INT())).getLogicalType(); - String json = "{\"f0\":\"abc\", \"f1\": \"abc\"}"; - JsonRowDataDeserializationSchema deserializationSchema = - new JsonRowDataDeserializationSchema( - rowType, InternalTypeInfo.of(rowType), false, false, TimestampFormat.SQL); - open(deserializationSchema); - String errorMessage = "Fail to deserialize at field: f1."; - - assertThatThrownBy(() -> deserializationSchema.deserialize(json.getBytes())) - .satisfies(anyCauseMatches(errorMessage)); - } - - private void testIgnoreParseErrors(TestSpec spec) throws Exception { - // the parsing field should be null and no exception is thrown - JsonRowDataDeserializationSchema ignoreErrorsSchema = - new JsonRowDataDeserializationSchema( - spec.rowType, - InternalTypeInfo.of(spec.rowType), - false, - true, - spec.timestampFormat); - ignoreErrorsSchema.open(new DummyInitializationContext()); - - Row expected; - if (spec.expected != null) { - expected = spec.expected; - } else { - expected = new Row(1); - } - RowData rowData = ignoreErrorsSchema.deserialize(spec.json.getBytes()); - Row actual = convertToExternal(rowData, fromLogicalToDataType(spec.rowType)); - assertThat(actual) - .isEqualTo(expected) - .withFailMessage("Test Ignore Parse Error: " + spec.json); - } - - private void testParseErrors(TestSpec spec) { - // expect exception if parse error is not ignored - JsonRowDataDeserializationSchema failingSchema = - new JsonRowDataDeserializationSchema( - spec.rowType, - InternalTypeInfo.of(spec.rowType), - false, - false, - spec.timestampFormat); - open(failingSchema); - - assertThatThrownBy(() -> failingSchema.deserialize(spec.json.getBytes())) - .hasMessageContaining(spec.errorMessage); - } - - private static List testData = - Arrays.asList( - TestSpec.json("{\"id\": \"trueA\"}") - .rowType(ROW(FIELD("id", BOOLEAN()))) - .expect(Row.of(false)), - TestSpec.json("{\"id\": true}") - .rowType(ROW(FIELD("id", BOOLEAN()))) - .expect(Row.of(true)), - TestSpec.json("{\"id\":\"abc\"}") - .rowType(ROW(FIELD("id", INT()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'."), - TestSpec.json("{\"id\":112.013}") - .rowType(ROW(FIELD("id", BIGINT()))) - .expect(Row.of(112L)), - TestSpec.json("{\"id\":\"long\"}") - .rowType(ROW(FIELD("id", BIGINT()))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'."), - TestSpec.json("{\"id\":\"112.013.123\"}") - .rowType(ROW(FIELD("id", FLOAT()))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'."), - TestSpec.json("{\"id\":\"112.013.123\"}") - .rowType(ROW(FIELD("id", DOUBLE()))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'."), - TestSpec.json("{\"id\":\"18:00:243\"}") - .rowType(ROW(FIELD("id", TIME()))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"18:00:243\"}'."), - TestSpec.json("{\"id\":\"18:00:243\"}") - .rowType(ROW(FIELD("id", TIME()))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"18:00:243\"}'."), - TestSpec.json("{\"id\":\"20191112\"}") - .rowType(ROW(FIELD("id", DATE()))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"20191112\"}'."), - TestSpec.json("{\"id\":\"20191112\"}") - .rowType(ROW(FIELD("id", DATE()))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"20191112\"}'."), - TestSpec.json("{\"id\":true}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("true")), - TestSpec.json("{\"id\":123.234}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("123.234")), - TestSpec.json("{\"id\":1234567}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("1234567")), - TestSpec.json("{\"id\":\"string field\"}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("string field")), - TestSpec.json("{\"id\":[\"array data1\",\"array data2\",123,234.345]}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("[\"array data1\",\"array data2\",123,234.345]")), - TestSpec.json("{\"id\":{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}}") - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}")), - TestSpec.json("{\"id\":\"2019-11-12 18:00:12\"}") - .rowType(ROW(FIELD("id", TIMESTAMP(0)))) - .timestampFormat(TimestampFormat.ISO_8601) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'."), - TestSpec.json("{\"id\":\"2019-11-12T18:00:12\"}") - .rowType(ROW(FIELD("id", TIMESTAMP(0)))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."), - TestSpec.json("{\"id\":\"2019-11-12T18:00:12Z\"}") - .rowType(ROW(FIELD("id", TIMESTAMP(0)))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'."), - TestSpec.json("{\"id\":\"2019-11-12T18:00:12Z\"}") - .rowType(ROW(FIELD("id", TIMESTAMP(0)))) - .timestampFormat(TimestampFormat.ISO_8601) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12Z\"}'."), - TestSpec.json("{\"id\":\"abc\"}") - .rowType(ROW(FIELD("id", DECIMAL(10, 3)))) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'."), - TestSpec.json("{\"row\":{\"id\":\"abc\"}}") - .rowType(ROW(FIELD("row", ROW(FIELD("id", BOOLEAN()))))) - .expect(Row.of(Row.of(false))), - TestSpec.json("{\"array\":[123, \"abc\"]}") - .rowType(ROW(FIELD("array", ARRAY(INT())))) - .expect(Row.of((Object) new Integer[] {123, null})) - .expectErrorMessage( - "Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'."), - TestSpec.json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") - .rowType(ROW(FIELD("map", MAP(STRING(), INT())))) - .expect(Row.of(createHashMap("key1", 123, "key2", null))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'."), - TestSpec.json("{\"id\":\"2019-11-12T18:00:12\"}") - .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0)))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."), - TestSpec.json("{\"id\":\"2019-11-12T18:00:12+0800\"}") - .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0)))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12+0800\"}'."), - TestSpec.json("{\"id\":1,\"factor\":799.929496989092949698}") - .rowType(ROW(FIELD("id", INT()), FIELD("factor", DECIMAL(38, 18)))) - .expect(Row.of(1, new BigDecimal("799.929496989092949698"))), - TestSpec.json("{\"id\":\"\tstring field\"}") // test to parse control chars - .rowType(ROW(FIELD("id", STRING()))) - .expect(Row.of("\tstring field"))); - - private static Map createHashMap( - String k1, Integer v1, String k2, Integer v2) { - Map map = new HashMap<>(); - map.put(k1, v1); - map.put(k2, v2); - return map; - } - - @SuppressWarnings("unchecked") - private static Row convertToExternal(RowData rowData, DataType dataType) { - return (Row) DataFormatConverters.getConverterForDataType(dataType).toExternal(rowData); - } - - private static class TestSpec { - private final String json; - private RowType rowType; - private TimestampFormat timestampFormat = TimestampFormat.SQL; - private Row expected; - private String errorMessage; - - private TestSpec(String json) { - this.json = json; - } - - public static TestSpec json(String json) { - return new TestSpec(json); - } - - TestSpec expect(Row row) { - this.expected = row; - return this; - } - - TestSpec rowType(DataType rowType) { - this.rowType = (RowType) rowType.getLogicalType(); - return this; - } - - TestSpec expectErrorMessage(String errorMessage) { - this.errorMessage = errorMessage; - return this; - } - - TestSpec timestampFormat(TimestampFormat timestampFormat) { - this.timestampFormat = timestampFormat; - return this; - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java deleted file mode 100644 index 81e370c0f..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ /dev/null @@ -1,432 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.types.Row; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; - -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -import javax.annotation.Nullable; - -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; - -import static org.apache.flink.formats.utils.DeserializationSchemaMatcher.whenDeserializedWith; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.instanceOf; -import static org.junit.Assert.assertThat; -import static org.junit.internal.matchers.ThrowableCauseMatcher.hasCause; -import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; - -/** Tests for the {@link JsonRowDeserializationSchema}. */ -public class JsonRowDeserializationSchemaTest { - - private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper(); - - @Rule public ExpectedException thrown = ExpectedException.none(); - - /** Tests simple deserialization using type information. */ - @Test - public void testTypeInfoDeserialization() throws Exception { - long id = 1238123899121L; - String name = "asdlkjasjkdla998y1122"; - byte[] bytes = new byte[1024]; - ThreadLocalRandom.current().nextBytes(bytes); - Timestamp timestamp = Timestamp.valueOf("1990-10-14 12:12:43"); - Date date = Date.valueOf("1990-10-14"); - Time time = Time.valueOf("12:12:43"); - - Map map = new HashMap<>(); - map.put("flink", 123L); - - Map> nestedMap = new HashMap<>(); - Map innerMap = new HashMap<>(); - innerMap.put("key", 234); - nestedMap.put("inner_map", innerMap); - - // Root - ObjectNode root = OBJECT_MAPPER.createObjectNode(); - root.put("id", id); - root.put("name", name); - root.put("bytes", bytes); - root.put("date1", "1990-10-14"); - root.put("date2", "1990-10-14"); - root.put("time1", "12:12:43Z"); - root.put("time2", "12:12:43Z"); - root.put("timestamp1", "1990-10-14T12:12:43Z"); - root.put("timestamp2", "1990-10-14T12:12:43Z"); - root.putObject("map").put("flink", 123); - root.putObject("map2map").putObject("inner_map").put("key", 234); - - byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); - - JsonRowDeserializationSchema deserializationSchema = - new JsonRowDeserializationSchema.Builder( - Types.ROW_NAMED( - new String[] { - "id", - "name", - "bytes", - "date1", - "date2", - "time1", - "time2", - "timestamp1", - "timestamp2", - "map", - "map2map" - }, - Types.LONG, - Types.STRING, - Types.PRIMITIVE_ARRAY(Types.BYTE), - Types.SQL_DATE, - Types.LOCAL_DATE, - Types.SQL_TIME, - Types.LOCAL_TIME, - Types.SQL_TIMESTAMP, - Types.LOCAL_DATE_TIME, - Types.MAP(Types.STRING, Types.LONG), - Types.MAP( - Types.STRING, Types.MAP(Types.STRING, Types.INT)))) - .build(); - - Row row = new Row(11); - row.setField(0, id); - row.setField(1, name); - row.setField(2, bytes); - row.setField(3, date); - row.setField(4, date.toLocalDate()); - row.setField(5, time); - row.setField(6, time.toLocalTime()); - row.setField(7, timestamp); - row.setField(8, timestamp.toLocalDateTime()); - row.setField(9, map); - row.setField(10, nestedMap); - - assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); - } - - @Test - public void testSchemaDeserialization() throws Exception { - final BigDecimal id = BigDecimal.valueOf(1238123899121L); - final String name = "asdlkjasjkdla998y1122"; - final byte[] bytes = new byte[1024]; - ThreadLocalRandom.current().nextBytes(bytes); - final BigDecimal[] numbers = - new BigDecimal[] { - BigDecimal.valueOf(1), BigDecimal.valueOf(2), BigDecimal.valueOf(3) - }; - final String[] strings = new String[] {"one", "two", "three"}; - - // Root - ObjectNode root = OBJECT_MAPPER.createObjectNode(); - root.put("id", id.longValue()); - root.putNull("idOrNull"); - root.put("name", name); - root.put("date", "1990-10-14"); - root.put("time", "12:12:43Z"); - root.put("timestamp", "1990-10-14T12:12:43Z"); - root.put("bytes", bytes); - root.putArray("numbers").add(1).add(2).add(3); - root.putArray("strings").add("one").add("two").add("three"); - root.putObject("nested").put("booleanField", true).put("decimalField", 12); - - final byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); - - JsonRowDeserializationSchema deserializationSchema = - new JsonRowDeserializationSchema.Builder( - "{" - + " type: 'object'," - + " properties: {" - + " id: { type: 'integer' }," - + " idOrNull: { type: ['integer', 'null'] }," - + " name: { type: 'string' }," - + " date: { type: 'string', format: 'date' }," - + " time: { type: 'string', format: 'time' }," - + " timestamp: { type: 'string', format: 'date-time' }," - + " bytes: { type: 'string', contentEncoding: 'base64' }," - + " numbers: { type: 'array', items: { type: 'integer' } }," - + " strings: { type: 'array', items: { type: 'string' } }," - + " nested: { " - + " type: 'object'," - + " properties: { " - + " booleanField: { type: 'boolean' }," - + " decimalField: { type: 'number' }" - + " }" - + " }" - + " }" - + "}") - .build(); - - final Row expected = new Row(10); - expected.setField(0, id); - expected.setField(1, null); - expected.setField(2, name); - expected.setField(3, Date.valueOf("1990-10-14")); - expected.setField(4, Time.valueOf("12:12:43")); - expected.setField(5, Timestamp.valueOf("1990-10-14 12:12:43")); - expected.setField(6, bytes); - expected.setField(7, numbers); - expected.setField(8, strings); - final Row nestedRow = new Row(2); - nestedRow.setField(0, true); - nestedRow.setField(1, BigDecimal.valueOf(12)); - expected.setField(9, nestedRow); - - assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(expected)); - } - - /** Tests deserialization with non-existing field name. */ - @Test - public void testMissingNode() throws Exception { - // Root - ObjectNode root = OBJECT_MAPPER.createObjectNode(); - root.put("id", 123123123); - byte[] serializedJson = OBJECT_MAPPER.writeValueAsBytes(root); - - TypeInformation rowTypeInformation = - Types.ROW_NAMED(new String[] {"name"}, Types.STRING); - - JsonRowDeserializationSchema deserializationSchema = - new JsonRowDeserializationSchema.Builder(rowTypeInformation).build(); - - Row row = new Row(1); - assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); - - deserializationSchema = - new JsonRowDeserializationSchema.Builder(rowTypeInformation) - .failOnMissingField() - .build(); - - assertThat( - serializedJson, - whenDeserializedWith(deserializationSchema) - .failsWithException(hasCause(instanceOf(IllegalStateException.class)))); - - // ignore-parse-errors ignores missing field exception too - deserializationSchema = - new JsonRowDeserializationSchema.Builder(rowTypeInformation) - .ignoreParseErrors() - .build(); - assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage( - "JSON format doesn't support failOnMissingField and ignoreParseErrors are both true"); - new JsonRowDeserializationSchema.Builder(rowTypeInformation) - .failOnMissingField() - .ignoreParseErrors() - .build(); - } - - /** Tests that number of field names and types has to match. */ - @Test - public void testNumberOfFieldNamesAndTypesMismatch() { - try { - new JsonRowDeserializationSchema.Builder( - Types.ROW_NAMED(new String[] {"one", "two", "three"}, Types.LONG)) - .build(); - Assert.fail("Did not throw expected Exception"); - } catch (IllegalArgumentException ignored) { - // Expected - } - } - - @Test - public void testJsonParse() { - for (TestSpec spec : testData) { - testIgnoreParseErrors(spec); - if (spec.errorMessage != null) { - testParseErrors(spec); - } - } - } - - private void testIgnoreParseErrors(TestSpec spec) { - // the parsing field should be null and no exception is thrown - JsonRowDeserializationSchema ignoreErrorsSchema = - new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation) - .ignoreParseErrors() - .build(); - Row expected; - if (spec.expected != null) { - expected = spec.expected; - } else { - expected = new Row(1); - } - assertThat( - "Test Ignore Parse Error: " + spec.json, - spec.json.getBytes(), - whenDeserializedWith(ignoreErrorsSchema).equalsTo(expected)); - } - - private void testParseErrors(TestSpec spec) { - // expect exception if parse error is not ignored - JsonRowDeserializationSchema failingSchema = - new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation).build(); - assertThat( - "Test Parse Error: " + spec.json, - spec.json.getBytes(), - whenDeserializedWith(failingSchema) - .failsWithException(hasMessage(containsString(spec.errorMessage)))); - } - - private static List testData = - Arrays.asList( - TestSpec.json("{\"id\": \"trueA\"}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.BOOLEAN)) - .expect(Row.of(false)), - TestSpec.json("{\"id\": true}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.BOOLEAN)) - .expect(Row.of(true)), - TestSpec.json("{\"id\":\"abc\"}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.INT)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), - TestSpec.json("{\"id\":112.013}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.LONG)) - .expect(Row.of(112L)), - TestSpec.json("{\"id\":true}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) - .expect(Row.of("true")), - TestSpec.json("{\"id\":123.234}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) - .expect(Row.of("123.234")), - TestSpec.json("{\"id\":1234567}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) - .expect(Row.of("1234567")), - TestSpec.json("{\"id\":\"string field\"}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) - .expect(Row.of("string field")), - TestSpec.json("{\"id\":[\"array data1\",\"array data2\",123,234.345]}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) - .expect(Row.of("[\"array data1\",\"array data2\",123,234.345]")), - TestSpec.json("{\"id\":{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.STRING)) - .expect(Row.of("{\"k1\":123,\"k2\":234.234,\"k3\":\"string data\"}")), - TestSpec.json("{\"id\":\"long\"}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.LONG)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'"), - TestSpec.json("{\"id\":\"112.013.123\"}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.FLOAT)) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), - TestSpec.json("{\"id\":\"112.013.123\"}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.DOUBLE)) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), - TestSpec.json("{\"id\":\"18:00:243\"}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.SQL_TIME)) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"), - TestSpec.json("{\"id\":\"20191112\"}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.SQL_DATE)) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"20191112\"}'"), - TestSpec.json("{\"id\":\"2019-11-12 18:00:12\"}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.SQL_TIMESTAMP)) - .expectErrorMessage( - "Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'"), - TestSpec.json("{\"id\":\"abc\"}") - .typeInfo(Types.ROW_NAMED(new String[] {"id"}, Types.BIG_DEC)) - .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), - TestSpec.json("{\"row\":{\"id\":\"abc\"}}") - .typeInfo( - Types.ROW_NAMED( - new String[] {"row"}, - Types.ROW_NAMED(new String[] {"id"}, Types.INT))) - .expect(Row.of(new Row(1))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"row\":{\"id\":\"abc\"}}'"), - TestSpec.json("{\"array\":[123, \"abc\"]}") - .typeInfo( - Types.ROW_NAMED( - new String[] {"array"}, Types.OBJECT_ARRAY(Types.INT))) - .expect(Row.of((Object) new Integer[] {123, null})) - .expectErrorMessage( - "Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'"), - TestSpec.json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") - .typeInfo( - Types.ROW_NAMED( - new String[] {"map"}, - Types.MAP(Types.STRING, Types.INT))) - .expect(Row.of(createHashMap("key1", 123, "key2", null))) - .expectErrorMessage( - "Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'"), - TestSpec.json("{\"id\":1,\"factor\":799.929496989092949698}") - .typeInfo( - Types.ROW_NAMED( - new String[] {"id", "factor"}, - Types.INT, - Types.BIG_DEC)) - .expect(Row.of(1, new BigDecimal("799.929496989092949698")))); - - private static Map createHashMap( - String k1, Integer v1, String k2, Integer v2) { - Map map = new HashMap<>(); - map.put(k1, v1); - map.put(k2, v2); - return map; - } - - private static class TestSpec { - private final String json; - private @Nullable TypeInformation rowTypeInformation; - private @Nullable Row expected; - private @Nullable String errorMessage; - - private TestSpec(String json) { - this.json = json; - } - - public static TestSpec json(String json) { - return new TestSpec(json); - } - - TestSpec expect(Row row) { - this.expected = row; - return this; - } - - TestSpec typeInfo(TypeInformation rowTypeInformation) { - this.rowTypeInformation = rowTypeInformation; - return this; - } - - TestSpec expectErrorMessage(String errorMessage) { - this.errorMessage = errorMessage; - return this; - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java deleted file mode 100644 index b15444dc5..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.util.FileUtils; - -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.net.URL; -import java.util.Objects; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link JsonRowSchemaConverter}. */ -class JsonRowSchemaConverterTest { - - @Test - void testComplexSchema() throws Exception { - final URL url = getClass().getClassLoader().getResource("complex-schema.json"); - Objects.requireNonNull(url); - final String schema = FileUtils.readFileUtf8(new File(url.getFile())); - final TypeInformation result = JsonRowSchemaConverter.convert(schema); - - final TypeInformation expected = - Types.ROW_NAMED( - new String[] { - "fn", - "familyName", - "additionalName", - "tuples", - "honorificPrefix", - "url", - "email", - "tel", - "sound", - "org" - }, - Types.STRING, - Types.STRING, - Types.BOOLEAN, - Types.ROW(Types.BIG_DEC, Types.STRING, Types.STRING, Types.STRING), - Types.OBJECT_ARRAY(Types.STRING), - Types.STRING, - Types.ROW_NAMED(new String[] {"type", "value"}, Types.STRING, Types.STRING), - Types.ROW_NAMED( - new String[] {"type", "value"}, Types.BIG_DEC, Types.STRING), - Types.VOID, - Types.ROW_NAMED(new String[] {"organizationUnit"}, Types.ROW())); - - assertThat(result).isEqualTo(expected); - } - - @Test - void testReferenceSchema() throws Exception { - final URL url = getClass().getClassLoader().getResource("reference-schema.json"); - Objects.requireNonNull(url); - final String schema = FileUtils.readFileUtf8(new File(url.getFile())); - final TypeInformation result = JsonRowSchemaConverter.convert(schema); - - final TypeInformation expected = - Types.ROW_NAMED( - new String[] {"billing_address", "shipping_address", "optional_address"}, - Types.ROW_NAMED( - new String[] {"street_address", "city", "state"}, - Types.STRING, - Types.STRING, - Types.STRING), - Types.ROW_NAMED( - new String[] {"street_address", "city", "state"}, - Types.STRING, - Types.STRING, - Types.STRING), - Types.ROW_NAMED( - new String[] {"street_address", "city", "state"}, - Types.STRING, - Types.STRING, - Types.STRING)); - - assertThat(result).isEqualTo(expected); - } - - @Test - void testAtomicType() { - final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'number' }"); - - assertThat(result).isEqualTo(Types.BIG_DEC); - } - - @Test - void testMissingType() { - assertThatThrownBy(() -> JsonRowSchemaConverter.convert("{ }")) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testWrongType() { - assertThatThrownBy(() -> JsonRowSchemaConverter.convert("{ type: 'whatever' }")) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testArrayWithAdditionalItems() { - assertThatThrownBy( - () -> - JsonRowSchemaConverter.convert( - "{ type: 'array', items: [{type: 'integer'}], additionalItems: true }")) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testMissingProperties() { - final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'object' }"); - - assertThat(result).isEqualTo(Types.ROW()); - } - - @Test - void testNullUnionTypes() { - final TypeInformation result = - JsonRowSchemaConverter.convert("{ type: ['string', 'null'] }"); - - assertThat(result).isEqualTo(Types.STRING); - } - - @Test - void testTimestamp() { - final TypeInformation result = - JsonRowSchemaConverter.convert("{ type: 'string', format: 'date-time' }"); - - assertThat(result).isEqualTo(Types.SQL_TIMESTAMP); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java deleted file mode 100644 index ac1fe1818..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java +++ /dev/null @@ -1,250 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.types.Row; - -import org.junit.Test; - -import java.io.IOException; -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.util.concurrent.ThreadLocalRandom; - -import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; -import static org.apache.flink.formats.utils.SerializationSchemaMatcher.whenSerializedWith; -import static org.hamcrest.Matchers.instanceOf; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; - -/** Tests for the {@link JsonRowSerializationSchema}. */ -public class JsonRowSerializationSchemaTest { - - @Test - public void testRowSerialization() { - final TypeInformation rowSchema = - Types.ROW_NAMED( - new String[] {"f1", "f2", "f3", "f4", "f5"}, - Types.INT, - Types.BOOLEAN, - Types.STRING, - Types.SQL_TIMESTAMP, - Types.LOCAL_DATE_TIME); - - final Row row = new Row(5); - row.setField(0, 1); - row.setField(1, true); - row.setField(2, "str"); - row.setField(3, Timestamp.valueOf("1990-10-14 12:12:43")); - row.setField(4, Timestamp.valueOf("1990-10-14 12:12:43").toLocalDateTime()); - - final JsonRowSerializationSchema serializationSchema = - new JsonRowSerializationSchema.Builder(rowSchema).build(); - final JsonRowDeserializationSchema deserializationSchema = - new JsonRowDeserializationSchema.Builder(rowSchema).build(); - - assertThat( - row, - whenSerializedWith(serializationSchema) - .andDeserializedWith(deserializationSchema) - .equalsTo(row)); - } - - @Test - public void testSerializationOfTwoRows() throws IOException { - final TypeInformation rowSchema = - Types.ROW_NAMED( - new String[] {"f1", "f2", "f3"}, Types.INT, Types.BOOLEAN, Types.STRING); - - final Row row1 = new Row(3); - row1.setField(0, 1); - row1.setField(1, true); - row1.setField(2, "str"); - - final JsonRowSerializationSchema serializationSchema = - new JsonRowSerializationSchema.Builder(rowSchema).build(); - open(serializationSchema); - final JsonRowDeserializationSchema deserializationSchema = - new JsonRowDeserializationSchema.Builder(rowSchema).build(); - open(deserializationSchema); - - byte[] bytes = serializationSchema.serialize(row1); - assertEquals(row1, deserializationSchema.deserialize(bytes)); - - final Row row2 = new Row(3); - row2.setField(0, 10); - row2.setField(1, false); - row2.setField(2, "newStr"); - - bytes = serializationSchema.serialize(row2); - assertEquals(row2, deserializationSchema.deserialize(bytes)); - } - - @Test - public void testMultiRowsWithNullValues() throws IOException { - String[] jsons = - new String[] { - "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, " - + "\"ids\":[1, 2, 3]}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}", - }; - - String[] expected = - new String[] { - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}," - + "\"ids\":[1,2,3]}", - "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}", - }; - - TypeInformation schema = - Types.ROW_NAMED( - new String[] {"svt", "ops", "ids"}, - Types.STRING, - Types.ROW_NAMED(new String[] {"id"}, Types.STRING), - Types.PRIMITIVE_ARRAY(Types.INT)); - JsonRowDeserializationSchema deserializationSchema = - new JsonRowDeserializationSchema.Builder(schema).build(); - open(deserializationSchema); - JsonRowSerializationSchema serializationSchema = - JsonRowSerializationSchema.builder().withTypeInfo(schema).build(); - open(serializationSchema); - - for (int i = 0; i < jsons.length; i++) { - String json = jsons[i]; - Row row = deserializationSchema.deserialize(json.getBytes()); - String result = new String(serializationSchema.serialize(row)); - assertEquals(expected[i], result); - } - } - - @Test - public void testNestedSchema() { - final TypeInformation rowSchema = - Types.ROW_NAMED( - new String[] {"f1", "f2", "f3"}, - Types.INT, - Types.BOOLEAN, - Types.ROW(Types.INT, Types.DOUBLE)); - - final Row row = new Row(3); - row.setField(0, 42); - row.setField(1, false); - final Row nested = new Row(2); - nested.setField(0, 22); - nested.setField(1, 2.3); - row.setField(2, nested); - - final JsonRowSerializationSchema serializationSchema = - new JsonRowSerializationSchema.Builder(rowSchema).build(); - final JsonRowDeserializationSchema deserializationSchema = - new JsonRowDeserializationSchema.Builder(rowSchema).build(); - - assertThat( - row, - whenSerializedWith(serializationSchema) - .andDeserializedWith(deserializationSchema) - .equalsTo(row)); - } - - @Test - public void testSerializeRowWithInvalidNumberOfFields() { - final TypeInformation rowSchema = - Types.ROW_NAMED( - new String[] {"f1", "f2", "f3"}, Types.INT, Types.BOOLEAN, Types.STRING); - - final Row row = new Row(1); - row.setField(0, 1); - - final JsonRowSerializationSchema serializationSchema = - new JsonRowSerializationSchema.Builder(rowSchema).build(); - open(serializationSchema); - assertThat( - row, - whenSerializedWith(serializationSchema) - .failsWithException(instanceOf(RuntimeException.class))); - } - - @Test - public void testSchema() { - final TypeInformation rowSchema = - JsonRowSchemaConverter.convert( - "{" - + " type: 'object'," - + " properties: {" - + " id: { type: 'integer' }," - + " idNumber: { type: 'number' }," - + " idOrNull: { type: ['integer', 'null'] }," - + " name: { type: 'string' }," - + " date: { type: 'string', format: 'date' }," - + " time: { type: 'string', format: 'time' }," - + " timestamp: { type: 'string', format: 'date-time' }," - + " bytes: { type: 'string', contentEncoding: 'base64' }," - + " numbers: { type: 'array', items: { type: 'integer' } }," - + " strings: { type: 'array', items: { type: 'string' } }," - + " nested: { " - + " type: 'object'," - + " properties: { " - + " booleanField: { type: 'boolean' }," - + " decimalField: { type: 'number' }" - + " }" - + " }" - + " }" - + "}"); - - final Row row = new Row(11); - row.setField(0, BigDecimal.valueOf(-333)); - row.setField(1, BigDecimal.valueOf(12.2222)); - row.setField(2, null); - row.setField(3, ""); - row.setField(4, Date.valueOf("1990-10-14")); - row.setField(5, Time.valueOf("12:12:43")); - row.setField(6, Timestamp.valueOf("1990-10-14 12:12:43")); - - final byte[] bytes = new byte[1024]; - ThreadLocalRandom.current().nextBytes(bytes); - row.setField(7, bytes); - final BigDecimal[] numbers = - new BigDecimal[] { - BigDecimal.valueOf(1), BigDecimal.valueOf(2), BigDecimal.valueOf(3) - }; - row.setField(8, numbers); - final String[] strings = new String[] {"one", "two", "three"}; - row.setField(9, strings); - final Row nestedRow = new Row(2); - nestedRow.setField(0, true); - nestedRow.setField(1, BigDecimal.valueOf(12)); - row.setField(10, nestedRow); - - final JsonRowSerializationSchema serializationSchema = - new JsonRowSerializationSchema.Builder(rowSchema).build(); - final JsonRowDeserializationSchema deserializationSchema = - new JsonRowDeserializationSchema.Builder(rowSchema).build(); - - assertThat( - row, - whenSerializedWith(serializationSchema) - .andDeserializedWith(deserializationSchema) - .equalsTo(row)); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java deleted file mode 100644 index 5ed992c1e..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/JsonSerDeSchemaTest.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json; - -import org.apache.flink.connector.testutils.formats.DummyInitializationContext; - -import org.junit.jupiter.api.Test; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Objects; - -import static org.assertj.core.api.Assertions.assertThat; - -class JsonSerDeSchemaTest { - private static final JsonSerializationSchema SERIALIZATION_SCHEMA; - private static final JsonDeserializationSchema DESERIALIZATION_SCHEMA; - private static final String JSON = "{\"x\":34,\"y\":\"hello\"}"; - - static { - SERIALIZATION_SCHEMA = new JsonSerializationSchema<>(); - SERIALIZATION_SCHEMA.open(new DummyInitializationContext()); - DESERIALIZATION_SCHEMA = new JsonDeserializationSchema<>(Event.class); - DESERIALIZATION_SCHEMA.open(new DummyInitializationContext()); - } - - @Test - void testSrialization() throws IOException { - final byte[] serialized = SERIALIZATION_SCHEMA.serialize(new Event(34, "hello")); - assertThat(serialized).isEqualTo(JSON.getBytes(StandardCharsets.UTF_8)); - } - - @Test - void testDeserialization() throws IOException { - final Event deserialized = - DESERIALIZATION_SCHEMA.deserialize(JSON.getBytes(StandardCharsets.UTF_8)); - assertThat(deserialized).isEqualTo(new Event(34, "hello")); - } - - @Test - void testRoundTrip() throws IOException { - final Event original = new Event(34, "hello"); - - final byte[] serialized = SERIALIZATION_SCHEMA.serialize(original); - - final Event deserialized = DESERIALIZATION_SCHEMA.deserialize(serialized); - - assertThat(deserialized).isEqualTo(original); - } - - private static class Event { - - private int x; - private String y = null; - - public Event() {} - - public Event(int x, String y) { - this.x = x; - this.y = y; - } - - public int getX() { - return x; - } - - public void setX(int x) { - this.x = x; - } - - public String getY() { - return y; - } - - public void setY(String y) { - this.y = y; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Event event = (Event) o; - return x == event.x && Objects.equals(y, event.y); - } - - @Override - public int hashCode() { - return Objects.hash(x, y); - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java deleted file mode 100644 index 00bd5a062..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonFormatFactoryTest.java +++ /dev/null @@ -1,206 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.canal; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.TestDynamicTableFactory; -import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; -import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.function.Consumer; - -import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; -import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; -import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; -import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link CanalJsonFormatFactory}. */ -class CanalJsonFormatFactoryTest { - - private static final InternalTypeInfo ROW_TYPE_INFO = - InternalTypeInfo.of(PHYSICAL_TYPE); - - @Test - void testDefaultOptions() { - Map options = getAllOptions(); - - // test Deser - CanalJsonDeserializationSchema expectedDeser = - CanalJsonDeserializationSchema.builder( - PHYSICAL_DATA_TYPE, Collections.emptyList(), ROW_TYPE_INFO) - .setIgnoreParseErrors(false) - .setTimestampFormat(TimestampFormat.SQL) - .build(); - DeserializationSchema actualDeser = createDeserializationSchema(options); - assertThat(actualDeser).isEqualTo(expectedDeser); - - // test Ser - CanalJsonSerializationSchema expectedSer = - new CanalJsonSerializationSchema( - PHYSICAL_TYPE, - TimestampFormat.SQL, - JsonFormatOptions.MapNullKeyMode.FAIL, - "null", - false); - SerializationSchema actualSer = createSerializationSchema(options); - assertThat(actualSer).isEqualTo(expectedSer); - } - - @Test - void testUserDefinedOptions() { - Map options = getAllOptions(); - options.put("canal-json.ignore-parse-errors", "true"); - options.put("canal-json.timestamp-format.standard", "ISO-8601"); - options.put("canal-json.database.include", "mydb"); - options.put("canal-json.table.include", "mytable"); - options.put("canal-json.map-null-key.mode", "LITERAL"); - options.put("canal-json.map-null-key.literal", "nullKey"); - options.put("canal-json.encode.decimal-as-plain-number", "true"); - - // test Deser - CanalJsonDeserializationSchema expectedDeser = - CanalJsonDeserializationSchema.builder( - PHYSICAL_DATA_TYPE, Collections.emptyList(), ROW_TYPE_INFO) - .setIgnoreParseErrors(true) - .setTimestampFormat(TimestampFormat.ISO_8601) - .setDatabase("mydb") - .setTable("mytable") - .build(); - DeserializationSchema actualDeser = createDeserializationSchema(options); - assertThat(actualDeser).isEqualTo(expectedDeser); - - // test Ser - CanalJsonSerializationSchema expectedSer = - new CanalJsonSerializationSchema( - PHYSICAL_TYPE, - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "nullKey", - true); - SerializationSchema actualSer = createSerializationSchema(options); - assertThat(actualSer).isEqualTo(expectedSer); - } - - @Test - void testInvalidIgnoreParseError() { - final Map options = - getModifiedOptions(opts -> opts.put("canal-json.ignore-parse-errors", "abc")); - - assertThatThrownBy(() -> createDeserializationSchema(options)) - .satisfies( - anyCauseMatches( - IllegalArgumentException.class, - "Unrecognized option for boolean: abc. Expected either true" - + " or false(case insensitive)")); - } - - @Test - void testInvalidOptionForTimestampFormat() { - final Map tableOptions = - getModifiedOptions( - opts -> opts.put("canal-json.timestamp-format.standard", "test")); - - assertThatThrownBy(() -> createDeserializationSchema(tableOptions)) - .isInstanceOf(ValidationException.class) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'test' for timestamp-format.standard. " - + "Supported values are [SQL, ISO-8601].")); - } - - @Test - void testInvalidOptionForMapNullKeyMode() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("canal-json.map-null-key.mode", "invalid")); - - assertThatThrownBy(() -> createSerializationSchema(tableOptions)) - .isInstanceOf(ValidationException.class) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'invalid' for option map-null-key.mode. " - + "Supported values are [LITERAL, FAIL, DROP].")); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifiedOptions(Consumer> optionModifier) { - Map options = getAllOptions(); - optionModifier.accept(options); - return options; - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - options.put("format", "canal-json"); - return options; - } - - private static DeserializationSchema createDeserializationSchema( - Map options) { - DynamicTableSource source = createTableSource(SCHEMA, options); - - assert source instanceof TestDynamicTableFactory.DynamicTableSourceMock; - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) source; - - return scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); - } - - private static SerializationSchema createSerializationSchema( - Map options) { - DynamicTableSink sink = createTableSink(SCHEMA, options); - - assert sink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) sink; - - return sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java deleted file mode 100644 index e45bfcc5e..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/canal/CanalJsonSerDeSchemaTest.java +++ /dev/null @@ -1,321 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.canal; - -import org.apache.flink.connector.testutils.formats.DummyInitializationContext; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.canal.CanalJsonDecodingFormat.ReadableMetadata; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.function.Consumer; -import java.util.stream.Collectors; - -import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; -import static org.apache.flink.table.api.DataTypes.FIELD; -import static org.apache.flink.table.api.DataTypes.FLOAT; -import static org.apache.flink.table.api.DataTypes.INT; -import static org.apache.flink.table.api.DataTypes.ROW; -import static org.apache.flink.table.api.DataTypes.STRING; -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link CanalJsonSerializationSchema} and {@link CanalJsonDeserializationSchema}. */ -class CanalJsonSerDeSchemaTest { - - private static final DataType PHYSICAL_DATA_TYPE = - ROW( - FIELD("id", INT().notNull()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", FLOAT())); - - @Test - void testFilteringTables() throws Exception { - List lines = readLines("canal-data-filter-table.txt"); - CanalJsonDeserializationSchema deserializationSchema = - CanalJsonDeserializationSchema.builder( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType())) - .setDatabase("^my.*") - .setTable("^prod.*") - .build(); - runTest(lines, deserializationSchema); - } - - @Test - void testDeserializeNullRow() throws Exception { - final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); - final CanalJsonDeserializationSchema deserializationSchema = - createCanalJsonDeserializationSchema(null, null, requestedMetadata); - open(deserializationSchema); - final SimpleCollector collector = new SimpleCollector(); - - deserializationSchema.deserialize(null, collector); - deserializationSchema.deserialize(new byte[0], collector); - assertThat(collector.list).hasSize(0); - } - - @Test - void testDeserializationWithMetadata() throws Exception { - testDeserializationWithMetadata( - "canal-data.txt", - null, - null, - row -> { - assertThat(row.getInt(0)).isEqualTo(101); - assertThat(row.getString(1).toString()).isEqualTo("scooter"); - assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter"); - assertThat(row.getFloat(3)).isEqualTo(3.14f); - assertThat(row.getString(4).toString()).isEqualTo("inventory"); - assertThat(row.getString(5).toString()).isEqualTo("products2"); - assertThat(row.getMap(6).size()).isEqualTo(4); - assertThat(row.getArray(7).getString(0).toString()).isEqualTo("id"); - assertThat(row.getTimestamp(8, 3).getMillisecond()).isEqualTo(1589373515477L); - assertThat(row.getTimestamp(9, 3).getMillisecond()).isEqualTo(1589373515000L); - }); - testDeserializationWithMetadata( - "canal-data-filter-table.txt", - "mydb", - "product", - row -> { - assertThat(row.getInt(0)).isEqualTo(101); - assertThat(row.getString(1).toString()).isEqualTo("scooter"); - assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter"); - assertThat(row.getFloat(3)).isEqualTo(3.14f); - assertThat(row.getString(4).toString()).isEqualTo("mydb"); - assertThat(row.getString(5).toString()).isEqualTo("product"); - assertThat(row.getMap(6).size()).isEqualTo(4); - assertThat(row.getArray(7).getString(0).toString()).isEqualTo("id"); - assertThat(row.getTimestamp(8, 3).getMillisecond()).isEqualTo(1598944146308L); - assertThat(row.getTimestamp(9, 3).getMillisecond()).isEqualTo(1598944132000L); - }); - } - - @Test - void testSerializationDeserialization() throws Exception { - List lines = readLines("canal-data.txt"); - CanalJsonDeserializationSchema deserializationSchema = - CanalJsonDeserializationSchema.builder( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType())) - .setIgnoreParseErrors(false) - .setTimestampFormat(TimestampFormat.ISO_8601) - .build(); - runTest(lines, deserializationSchema); - } - - public void runTest(List lines, CanalJsonDeserializationSchema deserializationSchema) - throws Exception { - open(deserializationSchema); - SimpleCollector collector = new SimpleCollector(); - for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); - } - - // Canal captures change data (`canal-data.txt`) on the `product` table: - // - // CREATE TABLE product ( - // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - // name VARCHAR(255), - // description VARCHAR(512), - // weight FLOAT - // ); - // ALTER TABLE product AUTO_INCREMENT = 101; - // - // INSERT INTO product - // VALUES (default,"scooter","Small 2-wheel scooter",3.14), - // (default,"car battery","12V car battery",8.1), - // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 - // to #3",0.8), - // (default,"hammer","12oz carpenter's hammer",0.75), - // (default,"hammer","14oz carpenter's hammer",0.875), - // (default,"hammer","16oz carpenter's hammer",1.0), - // (default,"rocks","box of assorted rocks",5.3), - // (default,"jacket","water resistent black wind breaker",0.1), - // (default,"spare tire","24 inch spare tire",22.2); - // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; - // UPDATE product SET weight='5.1' WHERE id=107; - // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); - // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); - // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' - // WHERE id=110; - // UPDATE product SET weight='5.17' WHERE id=111; - // DELETE FROM product WHERE id=111; - // UPDATE product SET weight='5.17' WHERE id=102 or id = 101; - // DELETE FROM product WHERE id=102 or id = 103; - List expected = - Arrays.asList( - "+I(101,scooter,Small 2-wheel scooter,3.14)", - "+I(102,car battery,12V car battery,8.1)", - "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", - "+I(104,hammer,12oz carpenter's hammer,0.75)", - "+I(105,hammer,14oz carpenter's hammer,0.875)", - "+I(106,hammer,null,1.0)", - "+I(107,rocks,box of assorted rocks,5.3)", - "+I(108,jacket,water resistent black wind breaker,0.1)", - "+I(109,spare tire,24 inch spare tire,22.2)", - "-U(106,hammer,null,1.0)", - "+U(106,hammer,18oz carpenter hammer,1.0)", - "-U(107,rocks,box of assorted rocks,5.3)", - "+U(107,rocks,box of assorted rocks,5.1)", - "+I(110,jacket,water resistent white wind breaker,0.2)", - "+I(111,scooter,Big 2-wheel scooter ,5.18)", - "-U(110,jacket,water resistent white wind breaker,0.2)", - "+U(110,jacket,new water resistent white wind breaker,0.5)", - "-U(111,scooter,Big 2-wheel scooter ,5.18)", - "+U(111,scooter,Big 2-wheel scooter ,5.17)", - "-D(111,scooter,Big 2-wheel scooter ,5.17)", - "-U(101,scooter,Small 2-wheel scooter,3.14)", - "+U(101,scooter,Small 2-wheel scooter,5.17)", - "-U(102,car battery,12V car battery,8.1)", - "+U(102,car battery,12V car battery,5.17)", - "-D(102,car battery,12V car battery,5.17)", - "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)"); - List actual = - collector.list.stream().map(Object::toString).collect(Collectors.toList()); - assertThat(actual).isEqualTo(expected); - - // test Serialization - CanalJsonSerializationSchema serializationSchema = - new CanalJsonSerializationSchema( - (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - serializationSchema.open(new DummyInitializationContext()); - - List result = new ArrayList<>(); - for (RowData rowData : collector.list) { - result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); - } - - List expectedResult = - Arrays.asList( - "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"INSERT\"}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"DELETE\"}", - "{\"data\":[{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8}],\"type\":\"DELETE\"}"); - - assertThat(result).isEqualTo(expectedResult); - } - - private void testDeserializationWithMetadata( - String resourceFile, String database, String table, Consumer testConsumer) - throws Exception { - // we only read the first line for keeping the test simple - final String firstLine = readLines(resourceFile).get(0); - final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); - final CanalJsonDeserializationSchema deserializationSchema = - createCanalJsonDeserializationSchema(database, table, requestedMetadata); - open(deserializationSchema); - final SimpleCollector collector = new SimpleCollector(); - - deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - assertThat(collector.list).hasSize(9); - assertThat(collector.list.get(0)).satisfies(testConsumer); - } - - private CanalJsonDeserializationSchema createCanalJsonDeserializationSchema( - String database, String table, List requestedMetadata) { - final DataType producedDataType = - DataTypeUtils.appendRowFields( - PHYSICAL_DATA_TYPE, - requestedMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList())); - return CanalJsonDeserializationSchema.builder( - PHYSICAL_DATA_TYPE, - requestedMetadata, - InternalTypeInfo.of(producedDataType.getLogicalType())) - .setDatabase(database) - .setTable(table) - .setIgnoreParseErrors(false) - .setTimestampFormat(TimestampFormat.ISO_8601) - .build(); - } - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - private static List readLines(String resource) throws IOException { - final URL url = CanalJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return Files.readAllLines(path); - } - - private static class SimpleCollector implements Collector { - - private final List list = new ArrayList<>(); - - @Override - public void collect(RowData record) { - list.add(record); - } - - @Override - public void close() { - // do nothing - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java deleted file mode 100644 index bc47d1e68..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonFormatFactoryTest.java +++ /dev/null @@ -1,170 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.maxwell; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.TestDynamicTableFactory; -import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; -import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.function.Consumer; - -import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; -import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; -import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; -import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link MaxwellJsonFormatFactory}. */ -class MaxwellJsonFormatFactoryTest { - - private static final InternalTypeInfo ROW_TYPE_INFO = - InternalTypeInfo.of(PHYSICAL_TYPE); - - @Test - void testSeDeSchema() { - final MaxwellJsonDeserializationSchema expectedDeser = - new MaxwellJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - ROW_TYPE_INFO, - true, - TimestampFormat.ISO_8601); - - final MaxwellJsonSerializationSchema expectedSer = - new MaxwellJsonSerializationSchema( - PHYSICAL_TYPE, - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - - final Map options = getAllOptions(); - - final DynamicTableSource actualSource = createTableSource(SCHEMA, options); - assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - DeserializationSchema actualDeser = - scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); - - assertThat(actualDeser).isEqualTo(expectedDeser); - - final DynamicTableSink actualSink = createTableSink(SCHEMA, options); - assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = - sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), SCHEMA.toPhysicalRowDataType()); - - assertThat(actualSer).isEqualTo(expectedSer); - } - - @Test - void testInvalidIgnoreParseError() { - final Map options = - getModifiedOptions(opts -> opts.put("maxwell-json.ignore-parse-errors", "abc")); - - assertThatThrownBy(() -> createTableSource(SCHEMA, options)) - .satisfies( - anyCauseMatches( - IllegalArgumentException.class, - "Unrecognized option for boolean: abc. " - + "Expected either true or false(case insensitive)")); - } - - @Test - void testInvalidOptionForTimestampFormat() { - final Map tableOptions = - getModifiedOptions( - opts -> opts.put("maxwell-json.timestamp-format.standard", "test")); - - assertThatThrownBy(() -> createTableSource(SCHEMA, tableOptions)) - .isInstanceOf(ValidationException.class) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'test' for timestamp-format.standard. " - + "Supported values are [SQL, ISO-8601].")); - } - - @Test - void testInvalidOptionForMapNullKeyMode() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("maxwell-json.map-null-key.mode", "invalid")); - - assertThatThrownBy(() -> createTableSink(SCHEMA, tableOptions)) - .isInstanceOf(ValidationException.class) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'invalid' for option map-null-key.mode. " - + "Supported values are [LITERAL, FAIL, DROP].")); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifiedOptions(Consumer> optionModifier) { - Map options = getAllOptions(); - optionModifier.accept(options); - return options; - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", "maxwell-json"); - options.put("maxwell-json.ignore-parse-errors", "true"); - options.put("maxwell-json.timestamp-format.standard", "ISO-8601"); - options.put("maxwell-json.map-null-key.mode", "LITERAL"); - options.put("maxwell-json.map-null-key.literal", "null"); - options.put("maxwell-json.encode.decimal-as-plain-number", "true"); - return options; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java deleted file mode 100644 index 12d64fd99..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/maxwell/MaxwellJsonSerDerTest.java +++ /dev/null @@ -1,252 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.maxwell; - -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.maxwell.MaxwellJsonDecodingFormat.ReadableMetadata; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; -import static org.apache.flink.table.api.DataTypes.FIELD; -import static org.apache.flink.table.api.DataTypes.FLOAT; -import static org.apache.flink.table.api.DataTypes.INT; -import static org.apache.flink.table.api.DataTypes.ROW; -import static org.apache.flink.table.api.DataTypes.STRING; -import static org.assertj.core.api.Assertions.assertThat; - -/** - * Tests for {@link MaxwellJsonSerializationSchema} and {@link MaxwellJsonDeserializationSchema}. - */ -class MaxwellJsonSerDerTest { - - private static final DataType PHYSICAL_DATA_TYPE = - ROW( - FIELD("id", INT().notNull()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", FLOAT())); - - @Test - void testDeserializationWithMetadata() throws Exception { - // we only read the first line for keeping the test simple - final String firstLine = readLines("maxwell-data.txt").get(0); - final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); - final DataType producedDataType = - DataTypeUtils.appendRowFields( - PHYSICAL_DATA_TYPE, - requestedMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList())); - final MaxwellJsonDeserializationSchema deserializationSchema = - new MaxwellJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - requestedMetadata, - InternalTypeInfo.of(producedDataType.getLogicalType()), - false, - TimestampFormat.ISO_8601); - open(deserializationSchema); - final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - assertThat(collector.list).hasSize(1); - assertThat(collector.list.get(0)) - .satisfies( - row -> { - assertThat(row.getInt(0)).isEqualTo(101); - assertThat(row.getString(1).toString()).isEqualTo("scooter"); - assertThat(row.getString(2).toString()) - .isEqualTo("Small 2-wheel scooter"); - assertThat(row.getFloat(3)).isEqualTo(3.14f); - assertThat(row.getString(4).toString()).isEqualTo("test"); - assertThat(row.getString(5).toString()).isEqualTo("product"); - assertThat(row.getArray(6).getString(0).toString()).isEqualTo("id"); - assertThat(row.getTimestamp(7, 3).getMillisecond()) - .isEqualTo(1596684883000L); - }); - } - - @Test - void testSerializationDeserialization() throws Exception { - List lines = readLines("maxwell-data.txt"); - MaxwellJsonDeserializationSchema deserializationSchema = - new MaxwellJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - false, - TimestampFormat.ISO_8601); - open(deserializationSchema); - - SimpleCollector collector = new SimpleCollector(); - for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); - } - - // Maxwell captures change data (`maxwell-data.txt`) on the `product` table: - // - // CREATE TABLE product ( - // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - // name VARCHAR(255), - // description VARCHAR(512), - // weight FLOAT - // ); - // ALTER TABLE product AUTO_INCREMENT = 101; - // - // INSERT INTO product - // VALUES (default,"scooter","Small 2-wheel scooter",3.14), - // (default,"car battery","12V car battery",8.1), - // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 - // to #3",0.8), - // (default,"hammer","12oz carpenter's hammer",0.75), - // (default,"hammer","14oz carpenter's hammer",0.875), - // (default,"hammer","16oz carpenter's hammer",1.0), - // (default,"rocks","box of assorted rocks",5.3), - // (default,"jacket","water resistent black wind breaker",0.1), - // (default,"spare tire","24 inch spare tire",22.2); - // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; - // UPDATE product SET weight='5.1' WHERE id=107; - // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); - // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); - // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' - // WHERE id=110; - // UPDATE product SET weight='5.17' WHERE id=111; - // DELETE FROM product WHERE id=111; - // UPDATE product SET weight='5.17' WHERE id=102 or id = 101; - // DELETE FROM product WHERE id=102 or id = 103; - List expected = - Arrays.asList( - "+I(101,scooter,Small 2-wheel scooter,3.14)", - "+I(102,car battery,12V car battery,8.1)", - "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", - "+I(104,hammer,12oz carpenter's hammer,0.75)", - "+I(105,hammer,14oz carpenter's hammer,0.875)", - "+I(106,hammer,16oz carpenter's hammer,1.0)", - "+I(107,rocks,box of assorted rocks,5.3)", - "+I(108,jacket,water resistent black wind breaker,0.1)", - "+I(109,spare tire,24 inch spare tire,22.2)", - "-U(106,hammer,16oz carpenter's hammer,1.0)", - "+U(106,hammer,18oz carpenter hammer,1.0)", - "-U(107,rocks,box of assorted rocks,5.3)", - "+U(107,rocks,box of assorted rocks,5.1)", - "+I(110,jacket,water resistent white wind breaker,0.2)", - "+I(111,scooter,Big 2-wheel scooter ,5.18)", - "-U(110,jacket,water resistent white wind breaker,0.2)", - "+U(110,jacket,new water resistent white wind breaker,0.5)", - "-U(111,scooter,Big 2-wheel scooter ,5.18)", - "+U(111,scooter,Big 2-wheel scooter ,5.17)", - "-D(111,scooter,Big 2-wheel scooter ,5.17)", - "-U(101,scooter,Small 2-wheel scooter,3.14)", - "+U(101,scooter,Small 2-wheel scooter,5.17)", - "-U(102,car battery,12V car battery,8.1)", - "+U(102,car battery,12V car battery,5.17)", - "-D(102,car battery,12V car battery,5.17)", - "-D(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)"); - List actual = - collector.list.stream().map(Object::toString).collect(Collectors.toList()); - assertThat(actual).isEqualTo(expected); - - MaxwellJsonSerializationSchema serializationSchema = - new MaxwellJsonSerializationSchema( - (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), - TimestampFormat.SQL, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - open(serializationSchema); - List result = new ArrayList<>(); - for (RowData rowData : collector.list) { - result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); - } - List expectedResult = - Arrays.asList( - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"insert\"}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"insert\"}", - "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"insert\"}", - "{\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"type\":\"insert\"}", - "{\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"type\":\"insert\"}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"insert\"}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"insert\"}", - "{\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"type\":\"insert\"}", - "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"type\":\"insert\"}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"delete\"}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"type\":\"insert\"}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"delete\"}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"type\":\"insert\"}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"insert\"}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"insert\"}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"delete\"}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"type\":\"insert\"}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"delete\"}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"insert\"}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"delete\"}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"delete\"}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17},\"type\":\"insert\"}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"delete\"}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"insert\"}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"delete\"}", - "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"type\":\"delete\"}"); - assertThat(result).isEqualTo(expectedResult); - } - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - private static List readLines(String resource) throws IOException { - final URL url = MaxwellJsonSerDerTest.class.getClassLoader().getResource(resource); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return Files.readAllLines(path); - } - - private static class SimpleCollector implements Collector { - - private List list = new ArrayList<>(); - - @Override - public void collect(RowData record) { - list.add(record); - } - - @Override - public void close() { - // do nothing - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java deleted file mode 100644 index 84b293cd9..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFileSystemITCase.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.ogg; - -import org.apache.flink.table.planner.runtime.utils.StreamingTestBase; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.CollectionUtil; - -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; - -import static java.lang.String.format; -import static org.assertj.core.api.Assertions.assertThat; - -/** Test Filesystem connector with OGG Json. */ -class OggJsonFileSystemITCase extends StreamingTestBase { - - private static final List EXPECTED = - Arrays.asList( - "+I[101, SCOOTER, Small 2-wheel scooter, 3.14]", - "+I[102, CAR BATTERY, 12V car battery, 8.1]", - "+I[103, 12-PACK DRILL BITS, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+I[104, HAMMER, 12oz carpenter's hammer, 0.75]", - "+I[105, HAMMER, 14oz carpenter's hammer, 0.875]", - "+I[106, HAMMER, 16oz carpenter's hammer, 1.0]", - "+I[107, ROCKS, box of assorted rocks, 5.3]", - "+I[108, JACKET, water resistent black wind breaker, 0.1]", - "+I[109, SPARE TIRE, 24 inch spare tire, 22.2]", - "-D[106, HAMMER, 16oz carpenter's hammer, 1.0]", // -U - "+I[106, HAMMER, 18oz carpenter hammer, 1.0]", // +U - "-D[107, ROCKS, box of assorted rocks, 5.3]", // -U - "+I[107, ROCKS, box of assorted rocks, 5.1]", // +U - "+I[110, JACKET, water resistent white wind breaker, 0.2]", - "+I[111, SCOOTER, Big 2-wheel scooter , 5.18]", - "-D[110, JACKET, water resistent white wind breaker, 0.2]", // -U - "+I[110, JACKET, new water resistent white wind breaker, 0.5]", // +U - "-D[111, SCOOTER, Big 2-wheel scooter , 5.18]", // -U - "+I[111, SCOOTER, Big 2-wheel scooter , 5.17]", // +U - "-D[111, SCOOTER, Big 2-wheel scooter , 5.17]"); - - private File source; - private File sink; - - private static byte[] readBytes(String resource) throws IOException { - final URL url = OggJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return Files.readAllBytes(path); - } - - private void prepareTables(boolean isPartition, Path tempSourceDir, Path tempSinkDir) - throws IOException { - byte[] bytes = readBytes("ogg-data.txt"); - source = tempSourceDir.toFile(); - File file; - if (isPartition) { - File partition = new File(source, "p=1"); - partition.mkdirs(); - file = new File(partition, "my_file"); - } else { - file = new File(source, "my_file"); - } - file.createNewFile(); - Files.write(file.toPath(), bytes); - - sink = tempSinkDir.toFile(); - - env().setParallelism(1); - } - - private void createTable(boolean isSink, String path, boolean isPartition) { - tEnv().executeSql( - format("create table %s (", isSink ? "sink" : "source") - + "id int, name string," - + (isSink ? "upper_name string," : "") - + " description string, weight float" - + (isPartition ? ", p int) partitioned by (p) " : ")") - + " with (" - + "'connector'='filesystem'," - + "'format'='ogg-json'," - + format("'path'='%s'", path) - + ")"); - } - - @Test - void testNonPartition(@TempDir Path tempSourceDir, @TempDir Path tempSinkDir) throws Exception { - prepareTables(true, tempSourceDir, tempSinkDir); - createTable(false, source.toURI().toString(), false); - createTable(true, sink.toURI().toString(), false); - - tEnv().executeSql( - "insert into sink select id,name,UPPER(name),description,weight from source") - .await(); - CloseableIterator iter = - tEnv().executeSql("select id,upper_name,description,weight from sink").collect(); - - List results = - CollectionUtil.iteratorToList(iter).stream() - .map(Row::toString) - .collect(Collectors.toList()); - iter.close(); - - assertThat(results).isEqualTo(EXPECTED); - } - - @Test - void testPartition(@TempDir Path tempSourceDir, @TempDir Path tempSinkDir) throws Exception { - prepareTables(true, tempSourceDir, tempSinkDir); - createTable(false, source.toURI().toString(), true); - createTable(true, sink.toURI().toString(), true); - - tEnv().executeSql( - "insert into sink select id,name,UPPER(name),description,weight,p from source") - .await(); - CloseableIterator iter = - tEnv().executeSql("select id,upper_name,description,weight,p from sink").collect(); - List list = CollectionUtil.iteratorToList(iter); - iter.close(); - - List results = - list.stream() - .map(row -> Row.project(row, new int[] {0, 1, 2, 3})) - .map(Row::toString) - .collect(Collectors.toList()); - - assertThat(results).isEqualTo(EXPECTED); - - // check partition value - for (Row row : list) { - assertThat(row.getField(4)).isEqualTo(1); - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java deleted file mode 100644 index c04e991a2..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonFormatFactoryTest.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.ogg; - -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.TestDynamicTableFactory; -import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; -import org.apache.flink.table.types.logical.RowType; - -import org.junit.jupiter.api.Test; - -import java.util.HashMap; -import java.util.Map; -import java.util.function.Consumer; - -import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; -import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; -import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link OggJsonFormatFactory}. */ -class OggJsonFormatFactoryTest { - - @Test - void testSeDeSchema() { - final Map options = getAllOptions(); - - final OggJsonSerializationSchema expectedSer = - new OggJsonSerializationSchema( - (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - - final DynamicTableSink actualSink = createTableSink(SCHEMA, options); - assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = - sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); - - assertThat(actualSer).isEqualTo(expectedSer); - } - - @Test - void testInvalidIgnoreParseError() { - final Map options = - getModifiedOptions(opts -> opts.put("ogg-json.ignore-parse-errors", "abc")); - - assertThatThrownBy(() -> createTableSource(SCHEMA, options)) - .satisfies( - anyCauseMatches( - IllegalArgumentException.class, - "Unrecognized option for boolean: abc. " - + "Expected either true or false(case insensitive)")); - } - - @Test - void testInvalidOptionForTimestampFormat() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("ogg-json.timestamp-format.standard", "test")); - - assertThatThrownBy(() -> createTableSource(SCHEMA, tableOptions)) - .isInstanceOf(ValidationException.class) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'test' for timestamp-format.standard. " - + "Supported values are [SQL, ISO-8601].")); - } - - @Test - void testInvalidOptionForMapNullKeyMode() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("ogg-json.map-null-key.mode", "invalid")); - - assertThatThrownBy(() -> createTableSink(SCHEMA, tableOptions)) - .isInstanceOf(ValidationException.class) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'invalid' for option map-null-key.mode. " - + "Supported values are [LITERAL, FAIL, DROP].")); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifiedOptions(Consumer> optionModifier) { - Map options = getAllOptions(); - optionModifier.accept(options); - return options; - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", "ogg-json"); - options.put("ogg-json.ignore-parse-errors", "true"); - options.put("ogg-json.timestamp-format.standard", "ISO-8601"); - options.put("ogg-json.map-null-key.mode", "LITERAL"); - options.put("ogg-json.map-null-key.literal", "null"); - options.put("ogg-json.encode.decimal-as-plain-number", "true"); - return options; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java deleted file mode 100644 index 2fa78c894..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/ogg/OggJsonSerDeSchemaTest.java +++ /dev/null @@ -1,266 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.ogg; - -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.ogg.OggJsonDecodingFormat.ReadableMetadata; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.util.Collector; - -import org.assertj.core.data.Percentage; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; -import static org.apache.flink.table.api.DataTypes.FIELD; -import static org.apache.flink.table.api.DataTypes.FLOAT; -import static org.apache.flink.table.api.DataTypes.INT; -import static org.apache.flink.table.api.DataTypes.ROW; -import static org.apache.flink.table.api.DataTypes.STRING; -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link OggJsonSerializationSchema} and {@link OggJsonDeserializationSchema}. */ -class OggJsonSerDeSchemaTest { - - private static final DataType PHYSICAL_DATA_TYPE = - ROW( - FIELD("id", INT().notNull()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", FLOAT())); - - private static List readLines(String resource) throws IOException { - final URL url = OggJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return Files.readAllLines(path); - } - - @Test - void testSerializationAndDeserialization() throws Exception { - testSerializationDeserialization("ogg-data.txt"); - } - - @Test - void testDeserializationWithMetadata() throws Exception { - testDeserializationWithMetadata("ogg-data.txt"); - } - - @Test - void testTombstoneMessages() throws Exception { - OggJsonDeserializationSchema deserializationSchema = - new OggJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - false, - TimestampFormat.ISO_8601); - open(deserializationSchema); - SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(null, collector); - deserializationSchema.deserialize(new byte[] {}, collector); - assertThat(collector.list).isEmpty(); - } - - public void testDeserializationWithMetadata(String resourceFile) throws Exception { - // we only read the first line for keeping the test simple - final String firstLine = readLines(resourceFile).get(0); - - final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); - - final DataType producedDataTypes = - DataTypeUtils.appendRowFields( - PHYSICAL_DATA_TYPE, - requestedMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList())); - final OggJsonDeserializationSchema deserializationSchema = - new OggJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - requestedMetadata, - InternalTypeInfo.of(producedDataTypes.getLogicalType()), - false, - TimestampFormat.ISO_8601); - open(deserializationSchema); - - final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - assertThat(collector.list).hasSize(1); - assertThat(collector.list.get(0)) - .satisfies( - row -> { - assertThat(row.getInt(0)).isEqualTo(101); - assertThat(row.getString(1).toString()).isEqualTo("scooter"); - assertThat(row.getString(2).toString()) - .isEqualTo("Small 2-wheel scooter"); - assertThat(row.getFloat(3)) - .isCloseTo( - 3.140000104904175f, Percentage.withPercentage(1e-15)); - assertThat(row.getString(4).toString()).isEqualTo("OGG.TBL_TEST"); - assertThat(row.getArray(5).getString(0).toString()).isEqualTo("id"); - assertThat(row.getTimestamp(6, 6).getMillisecond()) - .isEqualTo(1589377175766L); - assertThat(row.getTimestamp(7, 6).getMillisecond()) - .isEqualTo(1589384406000L); - }); - } - - private void testSerializationDeserialization(String resourceFile) throws Exception { - List lines = readLines(resourceFile); - OggJsonDeserializationSchema deserializationSchema = - new OggJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - false, - TimestampFormat.ISO_8601); - open(deserializationSchema); - - SimpleCollector collector = new SimpleCollector(); - for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); - } - - // Ogg captures change data (`ogg-data.txt`) on the `product` - // table: - // - // CREATE TABLE product ( - // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - // name VARCHAR(255), - // description VARCHAR(512), - // weight FLOAT - // ); - // ALTER TABLE product AUTO_INCREMENT = 101; - // - // INSERT INTO product - // VALUES (default,"scooter","Small 2-wheel scooter",3.14), - // (default,"car battery","12V car battery",8.1), - // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 - // to #3",0.8), - // (default,"hammer","12oz carpenter's hammer",0.75), - // (default,"hammer","14oz carpenter's hammer",0.875), - // (default,"hammer","16oz carpenter's hammer",1.0), - // (default,"rocks","box of assorted rocks",5.3), - // (default,"jacket","water resistent black wind breaker",0.1), - // (default,"spare tire","24 inch spare tire",22.2); - // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; - // UPDATE product SET weight='5.1' WHERE id=107; - // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); - // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); - // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' - // WHERE id=110; - // UPDATE product SET weight='5.17' WHERE id=111; - // DELETE FROM product WHERE id=111; - List expected = - Arrays.asList( - "+I(101,scooter,Small 2-wheel scooter,3.14)", - "+I(102,car battery,12V car battery,8.1)", - "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", - "+I(104,hammer,12oz carpenter's hammer,0.75)", - "+I(105,hammer,14oz carpenter's hammer,0.875)", - "+I(106,hammer,16oz carpenter's hammer,1.0)", - "+I(107,rocks,box of assorted rocks,5.3)", - "+I(108,jacket,water resistent black wind breaker,0.1)", - "+I(109,spare tire,24 inch spare tire,22.2)", - "-U(106,hammer,16oz carpenter's hammer,1.0)", - "+U(106,hammer,18oz carpenter hammer,1.0)", - "-U(107,rocks,box of assorted rocks,5.3)", - "+U(107,rocks,box of assorted rocks,5.1)", - "+I(110,jacket,water resistent white wind breaker,0.2)", - "+I(111,scooter,Big 2-wheel scooter ,5.18)", - "-U(110,jacket,water resistent white wind breaker,0.2)", - "+U(110,jacket,new water resistent white wind breaker,0.5)", - "-U(111,scooter,Big 2-wheel scooter ,5.18)", - "+U(111,scooter,Big 2-wheel scooter ,5.17)", - "-D(111,scooter,Big 2-wheel scooter ,5.17)"); - List actual = - collector.list.stream().map(Object::toString).collect(Collectors.toList()); - assertThat(expected).containsExactlyElementsOf(actual); - - OggJsonSerializationSchema serializationSchema = - new OggJsonSerializationSchema( - (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), - TimestampFormat.SQL, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - - open(serializationSchema); - actual = new ArrayList<>(); - for (RowData rowData : collector.list) { - actual.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); - } - - expected = - Arrays.asList( - "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op_type\":\"I\"}", - "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op_type\":\"I\"}", - "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op_type\":\"I\"}", - "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op_type\":\"I\"}", - "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op_type\":\"I\"}", - "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op_type\":\"I\"}", - "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op_type\":\"I\"}", - "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op_type\":\"I\"}", - "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op_type\":\"I\"}", - "{\"before\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"after\":null,\"op_type\":\"D\"}", - "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op_type\":\"I\"}", - "{\"before\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"after\":null,\"op_type\":\"D\"}", - "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op_type\":\"I\"}", - "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op_type\":\"I\"}", - "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op_type\":\"I\"}", - "{\"before\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"after\":null,\"op_type\":\"D\"}", - "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op_type\":\"I\"}", - "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":null,\"op_type\":\"D\"}", - "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op_type\":\"I\"}", - "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op_type\":\"D\"}"); - assertThat(expected).containsExactlyElementsOf(actual); - } - - private static class SimpleCollector implements Collector { - - private final List list = new ArrayList<>(); - - @Override - public void collect(RowData record) { - list.add(record); - } - - @Override - public void close() { - // do nothing - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java deleted file mode 100644 index c6d9e6229..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/DeserializationSchemaMatcher.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.utils; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.types.Row; - -import org.hamcrest.Description; -import org.hamcrest.Matcher; -import org.hamcrest.TypeSafeMatcher; - -import java.io.IOException; -import java.util.Objects; - -import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; -import static org.apache.flink.util.InstantiationUtil.deserializeObject; -import static org.apache.flink.util.InstantiationUtil.serializeObject; - -/** - * Matcher that provides a common way for asserting results of {@link DeserializationSchema}. It - * takes into account e.g. the fact that serialization schema during runtime might be used after - * serializing it over a wire. Usage: - * - *

    - *
  • when asserting for result after deserializing a row - *
    {@code
    - * byte[] jsonBytes = ...
    - * Row expectedRow = ...
    - * final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema);
    - *
    - * assertThat(jsonBytes, whenDeserializedWith(deserializationSchema)
    - *     .matches(expectedRow));
    - * }
    - *
  • to check if an exception is thrown during serialization: - *
    {@code
    - * assertThat(serializedJson,
    - *     whenDeserializedWith(deserializationSchema)
    - *         .failsWithException(hasCause(instanceOf(IllegalStateException.class))));
    - * }
    - *
- */ -public abstract class DeserializationSchemaMatcher extends TypeSafeMatcher { - - final DeserializationSchema deserializationSchema; - - private DeserializationSchemaMatcher(DeserializationSchema deserializationSchema) { - this.deserializationSchema = deserializationSchema; - } - - public static DeserializationSchemaMatcherBuilder whenDeserializedWith( - DeserializationSchema deserializationSchema) { - return new DeserializationSchemaMatcherBuilder(deserializationSchema); - } - - private static class DeserializationSchemaResultMatcher extends DeserializationSchemaMatcher { - - private final Row expected; - - private DeserializationSchemaResultMatcher( - DeserializationSchema deserializationSchema, Row expected) { - super(deserializationSchema); - - this.expected = expected; - } - - @Override - protected boolean matchesSafely(byte[] item) { - try { - return Objects.deepEquals(deserializationSchema.deserialize(item), expected); - } catch (IOException e) { - throw new AssertionError("Could not deserialize", e); - } - } - - @Override - public void describeTo(Description description) { - description.appendValue(expected); - } - } - - private static class DeserializationSchemaExceptionMatcher - extends DeserializationSchemaMatcher { - - private final Matcher exceptionMatcher; - private Throwable thrownException = null; - - private DeserializationSchemaExceptionMatcher( - DeserializationSchema deserializationSchema, - Matcher exceptionMatcher) { - super(deserializationSchema); - this.exceptionMatcher = exceptionMatcher; - } - - @Override - protected boolean matchesSafely(byte[] item) { - try { - deserializationSchema.deserialize(item); - } catch (IOException e) { - thrownException = e; - } - return exceptionMatcher.matches(thrownException); - } - - @Override - public void describeTo(Description description) { - exceptionMatcher.describeTo(description); - } - - @Override - protected void describeMismatchSafely(byte[] item, Description mismatchDescription) { - exceptionMatcher.describeMismatch(thrownException, mismatchDescription); - } - } - - /** Builder for {@link DeserializationSchemaMatcher}. */ - public static class DeserializationSchemaMatcherBuilder { - - private DeserializationSchema deserializationSchema; - - private DeserializationSchemaMatcherBuilder( - DeserializationSchema deserializationSchema) { - try { - // we serialize and deserialize the schema to test runtime behavior - // when the schema is shipped to the cluster - this.deserializationSchema = - deserializeObject( - serializeObject(deserializationSchema), - this.getClass().getClassLoader()); - open(this.deserializationSchema); - } catch (IOException | ClassNotFoundException e) { - throw new RuntimeException(e); - } - } - - public DeserializationSchemaMatcher equalsTo(Row row) { - return new DeserializationSchemaResultMatcher(deserializationSchema, row); - } - - public DeserializationSchemaMatcher failsWithException( - Matcher exceptionMatcher) { - return new DeserializationSchemaExceptionMatcher( - deserializationSchema, exceptionMatcher); - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java deleted file mode 100644 index 1cf6f6d20..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/utils/SerializationSchemaMatcher.java +++ /dev/null @@ -1,191 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.utils; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.types.Row; - -import org.hamcrest.Description; -import org.hamcrest.Matcher; -import org.hamcrest.TypeSafeMatcher; - -import java.io.IOException; -import java.util.Objects; - -import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; -import static org.apache.flink.util.InstantiationUtil.deserializeObject; -import static org.apache.flink.util.InstantiationUtil.serializeObject; - -/** - * Matcher that provides a common way for asserting results of {@link SerializationSchema}. It takes - * into account e.g. the fact that serialization schema during runtime might be used after - * serializing and deserializing it over a wire. Usage: - * - *
    - *
  • when asserting for result after serializing and deserializing a row - *
    {@code
    - * final JsonRowSerializationSchema serializationSchema = new JsonRowSerializationSchema(rowSchema);
    - * final JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema(rowSchema);
    - *
    - * assertThat(row, whenSerializedWith(serializationSchema)
    - *     .andDeserializedWith(deserializationSchema)
    - *     .matches(row));
    - * }
    - *
  • to check if an exception is thrown during serialization: - *
    {@code
    - * assertThat(row, whenSerializedWith(serializationSchema).failsWithException(instanceOf(RuntimeException.class)));
    - * }
    - *
- */ -public abstract class SerializationSchemaMatcher extends TypeSafeMatcher { - - final SerializationSchema serializationSchema; - - private SerializationSchemaMatcher(SerializationSchema serializationSchema) { - this.serializationSchema = serializationSchema; - } - - public static SerializationSchemaMatcherBuilder whenSerializedWith( - SerializationSchema serializationSchema) { - return new SerializationSchemaMatcherBuilder(serializationSchema); - } - - private static class SerializationSchemaResultMatcher extends SerializationSchemaMatcher { - - private final Row expected; - private final DeserializationSchema deserializationSchema; - - private SerializationSchemaResultMatcher( - SerializationSchema serializationSchema, - DeserializationSchema deserializationSchema, - Row expected) { - super(serializationSchema); - - this.expected = expected; - this.deserializationSchema = deserializationSchema; - } - - @Override - protected boolean matchesSafely(Row item) { - try { - return Objects.deepEquals( - deserializationSchema.deserialize(serializationSchema.serialize(item)), - expected); - } catch (IOException e) { - throw new AssertionError("Could not deserialize", e); - } - } - - @Override - public void describeTo(Description description) { - description.appendValue(expected); - } - } - - private static class SerializationSchemaExceptionMatcher extends SerializationSchemaMatcher { - - private final Matcher exceptionMatcher; - private Throwable thrownException = null; - - private SerializationSchemaExceptionMatcher( - SerializationSchema serializationSchema, - Matcher exceptionMatcher) { - super(serializationSchema); - this.exceptionMatcher = exceptionMatcher; - } - - @Override - protected boolean matchesSafely(Row item) { - try { - serializationSchema.serialize(item); - } catch (Exception e) { - thrownException = e; - } - return exceptionMatcher.matches(thrownException); - } - - @Override - public void describeTo(Description description) { - exceptionMatcher.describeTo(description); - } - - @Override - protected void describeMismatchSafely(Row item, Description mismatchDescription) { - exceptionMatcher.describeMismatch(thrownException, mismatchDescription); - } - } - - /** - * Builder for {@link SerializationSchemaMatcher} that can assert results after serialize and - * deserialize. - */ - public static class SerializationWithDeserializationSchemaMatcherBuilder { - - private SerializationSchema serializationSchema; - private DeserializationSchema deserializationSchema; - - private SerializationWithDeserializationSchemaMatcherBuilder( - SerializationSchema serializationSchema, - DeserializationSchema deserializationSchema) { - try { - // we serialize and deserialize the schema to test runtime behavior - // when the schema is shipped to the cluster - this.serializationSchema = - deserializeObject( - serializeObject(serializationSchema), - this.getClass().getClassLoader()); - open(this.serializationSchema); - this.deserializationSchema = - deserializeObject( - serializeObject(deserializationSchema), - this.getClass().getClassLoader()); - open(this.deserializationSchema); - } catch (IOException | ClassNotFoundException e) { - throw new RuntimeException(e); - } - } - - public SerializationSchemaMatcher equalsTo(Row expected) { - return new SerializationSchemaResultMatcher( - serializationSchema, deserializationSchema, expected); - } - } - - /** Builder for {@link SerializationSchemaMatcher}. */ - public static class SerializationSchemaMatcherBuilder { - - private SerializationSchema serializationSchema; - - private SerializationSchemaMatcherBuilder(SerializationSchema serializationSchema) { - this.serializationSchema = serializationSchema; - } - - public SerializationWithDeserializationSchemaMatcherBuilder andDeserializedWith( - DeserializationSchema deserializationSchema) { - return new SerializationWithDeserializationSchemaMatcherBuilder( - serializationSchema, deserializationSchema); - } - - public SerializationSchemaMatcher failsWithException( - Matcher exceptionMatcher) { - return new SerializationSchemaExceptionMatcher(serializationSchema, exceptionMatcher); - } - } -} diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index 10789e462..9ed0c2119 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -24,8 +24,8 @@ under the License. org.apache.flink - flink-formats - 1.18-SNAPSHOT + flink-formats-kafka + 4.1-SNAPSHOT flink-sql-avro-confluent-registry diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 6e85cfe9c..474ed028c 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -15,7 +15,7 @@ This project bundles the following dependencies under the Apache Software Licens - io.confluent:kafka-schema-registry-client:7.2.2 - org.apache.avro:avro:1.11.1 - org.apache.commons:commons-compress:1.21 -- org.apache.kafka:kafka-clients:7.2.2-ccs +- org.apache.kafka:kafka-clients:3.4.0 - org.glassfish.jersey.core:jersey-common:2.30 - org.xerial.snappy:snappy-java:1.1.8.3 diff --git a/flink-formats-kafka/pom.xml b/flink-formats-kafka/pom.xml new file mode 100644 index 000000000..dbe67f52d --- /dev/null +++ b/flink-formats-kafka/pom.xml @@ -0,0 +1,42 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connector-kafka-parent + 4.1-SNAPSHOT + + + pom + + flink-formats-kafka + Flink : Formats : Kafka + + + flink-avro-confluent-registry + flink-json-debezium + flink-sql-avro-confluent-registry + + + diff --git a/pom.xml b/pom.xml index 2557a589d..68a20c913 100644 --- a/pom.xml +++ b/pom.xml @@ -46,6 +46,8 @@ under the License. flink-connector-kafka flink-sql-connector-kafka flink-connector-kafka-e2e-tests + flink-formats-kafka + flink-confluent-schema-registry-e2e-tests @@ -53,6 +55,7 @@ under the License. 16.1 3.4.0 3.5.9 + 7.2.2 2.13.4.20221013 4.13.2 @@ -60,9 +63,11 @@ under the License. 3.23.1 1.17.2 3.4.6 + 2.0.9 1.3 1.12.10 1.5.0 + 2.12 2.12.7 2.12.7 1.1.8.3 From 96780b965a7e662f32c1b91bfe518d0a91d48d4e Mon Sep 17 00:00:00 2001 From: mas-chen Date: Thu, 6 Apr 2023 15:54:25 -0700 Subject: [PATCH 211/322] [FLINK-31747] remove debezium json for now This closes #16. --- .../24119078-1071-4906-b2ac-ed57c8154eaa | 0 .../62c5e4e5-2b0e-41ed-a268-ee33d5edd162 | 0 .../archunit-violations/stored.rules | 4 - .../flink-json-debezium/pom.xml | 141 ------- .../debezium/DebeziumJsonDecodingFormat.java | 286 --------------- .../DebeziumJsonDeserializationSchema.java | 344 ------------------ .../debezium/DebeziumJsonFormatFactory.java | 161 -------- .../debezium/DebeziumJsonFormatOptions.java | 52 --- .../DebeziumJsonSerializationSchema.java | 129 ------- .../org.apache.flink.table.factories.Factory | 16 - .../TestCodeArchitectureTest.java | 40 -- .../DebeziumJsonFileSystemITCase.java | 159 -------- .../DebeziumJsonFormatFactoryTest.java | 205 ----------- .../debezium/DebeziumJsonSerDeSchemaTest.java | 342 ----------------- .../org.junit.jupiter.api.extension.Extension | 16 - .../src/test/resources/archunit.properties | 31 -- .../resources/canal-data-filter-table.txt | 16 - .../src/test/resources/canal-data.txt | 11 - .../src/test/resources/complex-schema.json | 114 ------ .../debezium-data-schema-exclude.txt | 16 - .../debezium-data-schema-include.txt | 16 - ...ebezium-postgres-data-replica-identity.txt | 16 - .../debezium-postgres-data-schema-exclude.txt | 16 - .../debezium-postgres-data-schema-include.txt | 16 - .../src/test/resources/maxwell-data.txt | 20 - .../src/test/resources/ogg-data.txt | 16 - .../src/test/resources/reference-schema.json | 61 ---- flink-formats-kafka/pom.xml | 1 - 28 files changed, 2245 deletions(-) delete mode 100644 flink-formats-kafka/flink-json-debezium/archunit-violations/24119078-1071-4906-b2ac-ed57c8154eaa delete mode 100644 flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 delete mode 100644 flink-formats-kafka/flink-json-debezium/archunit-violations/stored.rules delete mode 100644 flink-formats-kafka/flink-json-debezium/pom.xml delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatOptions.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/archunit.properties delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/complex-schema.json delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-exclude.txt delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-include.txt delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-replica-identity.txt delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-exclude.txt delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-include.txt delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt delete mode 100644 flink-formats-kafka/flink-json-debezium/src/test/resources/reference-schema.json diff --git a/flink-formats-kafka/flink-json-debezium/archunit-violations/24119078-1071-4906-b2ac-ed57c8154eaa b/flink-formats-kafka/flink-json-debezium/archunit-violations/24119078-1071-4906-b2ac-ed57c8154eaa deleted file mode 100644 index e69de29bb..000000000 diff --git a/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 b/flink-formats-kafka/flink-json-debezium/archunit-violations/62c5e4e5-2b0e-41ed-a268-ee33d5edd162 deleted file mode 100644 index e69de29bb..000000000 diff --git a/flink-formats-kafka/flink-json-debezium/archunit-violations/stored.rules b/flink-formats-kafka/flink-json-debezium/archunit-violations/stored.rules deleted file mode 100644 index d43a1445d..000000000 --- a/flink-formats-kafka/flink-json-debezium/archunit-violations/stored.rules +++ /dev/null @@ -1,4 +0,0 @@ -# -#Mon Apr 04 17:11:32 CEST 2022 -Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=24119078-1071-4906-b2ac-ed57c8154eaa -ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=62c5e4e5-2b0e-41ed-a268-ee33d5edd162 diff --git a/flink-formats-kafka/flink-json-debezium/pom.xml b/flink-formats-kafka/flink-json-debezium/pom.xml deleted file mode 100644 index 147cf0e30..000000000 --- a/flink-formats-kafka/flink-json-debezium/pom.xml +++ /dev/null @@ -1,141 +0,0 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-formats-kafka - 4.1-SNAPSHOT - - - flink-json-debezium - Flink : Formats : Json Debezium - - jar - - - - org.apache.flink - flink-json - ${flink.version} - - - - org.apache.flink - flink-table-common - ${flink.version} - provided - true - - - - - - - org.apache.flink - flink-connector-files - ${flink.version} - test - - - - - org.apache.flink - flink-table-common - ${flink.version} - test - test-jar - - - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${flink.version} - test - - - - - org.apache.flink - flink-table-api-scala-bridge_${scala.binary.version} - ${flink.version} - test - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${flink.version} - test - test-jar - - - - - org.apache.flink - flink-connector-test-utils - ${flink.version} - test - - - org.apache.flink - flink-test-utils - ${flink.version} - test - - - - - - org.apache.flink - flink-architecture-tests-test - test - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-format-common - - - - - - - - - - diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java deleted file mode 100644 index 706d01a81..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDecodingFormat.java +++ /dev/null @@ -1,286 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.debezium; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.debezium.DebeziumJsonDeserializationSchema.MetadataConverter; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.Projection; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.ProjectableDecodingFormat; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.types.RowKind; - -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -/** {@link DecodingFormat} for Debezium using JSON encoding. */ -public class DebeziumJsonDecodingFormat - implements ProjectableDecodingFormat> { - - // -------------------------------------------------------------------------------------------- - // Mutable attributes - // -------------------------------------------------------------------------------------------- - - private List metadataKeys; - - // -------------------------------------------------------------------------------------------- - // Debezium-specific attributes - // -------------------------------------------------------------------------------------------- - - private final boolean schemaInclude; - - private final boolean ignoreParseErrors; - - private final TimestampFormat timestampFormat; - - public DebeziumJsonDecodingFormat( - boolean schemaInclude, boolean ignoreParseErrors, TimestampFormat timestampFormat) { - this.schemaInclude = schemaInclude; - this.ignoreParseErrors = ignoreParseErrors; - this.timestampFormat = timestampFormat; - this.metadataKeys = Collections.emptyList(); - } - - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) { - physicalDataType = Projection.of(projections).project(physicalDataType); - - final List readableMetadata = - metadataKeys.stream() - .map( - k -> - Stream.of(ReadableMetadata.values()) - .filter(rm -> rm.key.equals(k)) - .findFirst() - .orElseThrow(IllegalStateException::new)) - .collect(Collectors.toList()); - - final List metadataFields = - readableMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList()); - - final DataType producedDataType = - DataTypeUtils.appendRowFields(physicalDataType, metadataFields); - - final TypeInformation producedTypeInfo = - context.createTypeInformation(producedDataType); - - return new DebeziumJsonDeserializationSchema( - physicalDataType, - readableMetadata, - producedTypeInfo, - schemaInclude, - ignoreParseErrors, - timestampFormat); - } - - @Override - public Map listReadableMetadata() { - final Map metadataMap = new LinkedHashMap<>(); - Stream.of(ReadableMetadata.values()) - .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); - return metadataMap; - } - - @Override - public void applyReadableMetadata(List metadataKeys) { - this.metadataKeys = metadataKeys; - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - // -------------------------------------------------------------------------------------------- - // Metadata handling - // -------------------------------------------------------------------------------------------- - - /** List of metadata that can be read with this format. */ - enum ReadableMetadata { - SCHEMA( - "schema", - DataTypes.STRING().nullable(), - false, - DataTypes.FIELD("schema", DataTypes.STRING()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getString(pos); - } - }), - - INGESTION_TIMESTAMP( - "ingestion-timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), - true, - DataTypes.FIELD("ts_ms", DataTypes.BIGINT()), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - if (row.isNullAt(pos)) { - return null; - } - return TimestampData.fromEpochMillis(row.getLong(pos)); - } - }), - - SOURCE_TIMESTAMP( - "source.timestamp", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - final StringData timestamp = - (StringData) readProperty(row, pos, KEY_SOURCE_TIMESTAMP); - if (timestamp == null) { - return null; - } - return TimestampData.fromEpochMillis(Long.parseLong(timestamp.toString())); - } - }), - - SOURCE_DATABASE( - "source.database", - DataTypes.STRING().nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return readProperty(row, pos, KEY_SOURCE_DATABASE); - } - }), - - SOURCE_SCHEMA( - "source.schema", - DataTypes.STRING().nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return readProperty(row, pos, KEY_SOURCE_SCHEMA); - } - }), - - SOURCE_TABLE( - "source.table", - DataTypes.STRING().nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return readProperty(row, pos, KEY_SOURCE_TABLE); - } - }), - - SOURCE_PROPERTIES( - "source.properties", - // key and value of the map are nullable to make handling easier in queries - DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) - .nullable(), - true, - DataTypes.FIELD("source", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData row, int pos) { - return row.getMap(pos); - } - }); - - final String key; - - final DataType dataType; - - final boolean isJsonPayload; - - final DataTypes.Field requiredJsonField; - - final MetadataConverter converter; - - ReadableMetadata( - String key, - DataType dataType, - boolean isJsonPayload, - DataTypes.Field requiredJsonField, - MetadataConverter converter) { - this.key = key; - this.dataType = dataType; - this.isJsonPayload = isJsonPayload; - this.requiredJsonField = requiredJsonField; - this.converter = converter; - } - } - - private static final StringData KEY_SOURCE_TIMESTAMP = StringData.fromString("ts_ms"); - - private static final StringData KEY_SOURCE_DATABASE = StringData.fromString("db"); - - private static final StringData KEY_SOURCE_SCHEMA = StringData.fromString("schema"); - - private static final StringData KEY_SOURCE_TABLE = StringData.fromString("table"); - - private static Object readProperty(GenericRowData row, int pos, StringData key) { - final GenericMapData map = (GenericMapData) row.getMap(pos); - if (map == null) { - return null; - } - return map.get(key); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java deleted file mode 100644 index cba336d4f..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonDeserializationSchema.java +++ /dev/null @@ -1,344 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.debezium; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; -import org.apache.flink.formats.json.debezium.DebeziumJsonDecodingFormat.ReadableMetadata; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Collector; - -import java.io.IOException; -import java.io.Serializable; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; - -import static java.lang.String.format; - -/** - * Deserialization schema from Debezium JSON to Flink Table/SQL internal data structure {@link - * RowData}. The deserialization schema knows Debezium's schema definition and can extract the - * database data and convert into {@link RowData} with {@link RowKind}. - * - *

Deserializes a byte[] message as a JSON object and reads the specified fields. - * - *

Failures during deserialization are forwarded as wrapped IOExceptions. - * - * @see Debezium - */ -@Internal -public final class DebeziumJsonDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - - private static final String OP_READ = "r"; // snapshot read - private static final String OP_CREATE = "c"; // insert - private static final String OP_UPDATE = "u"; // update - private static final String OP_DELETE = "d"; // delete - - private static final String REPLICA_IDENTITY_EXCEPTION = - "The \"before\" field of %s message is null, " - + "if you are using Debezium Postgres Connector, " - + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; - - /** The deserializer to deserialize Debezium JSON data. */ - private final JsonRowDataDeserializationSchema jsonDeserializer; - - /** Flag that indicates that an additional projection is required for metadata. */ - private final boolean hasMetadata; - - /** Metadata to be extracted for every record. */ - private final MetadataConverter[] metadataConverters; - - /** {@link TypeInformation} of the produced {@link RowData} (physical + meta data). */ - private final TypeInformation producedTypeInfo; - - /** - * Flag indicating whether the Debezium JSON data contains schema part or not. When Debezium - * Kafka Connect enables "value.converter.schemas.enable", the JSON will contain "schema" - * information, but we just ignore "schema" and extract data from "payload". - */ - private final boolean schemaInclude; - - /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ - private final boolean ignoreParseErrors; - - public DebeziumJsonDeserializationSchema( - DataType physicalDataType, - List requestedMetadata, - TypeInformation producedTypeInfo, - boolean schemaInclude, - boolean ignoreParseErrors, - TimestampFormat timestampFormat) { - final RowType jsonRowType = - createJsonRowType(physicalDataType, requestedMetadata, schemaInclude); - this.jsonDeserializer = - new JsonRowDataDeserializationSchema( - jsonRowType, - // the result type is never used, so it's fine to pass in the produced type - // info - producedTypeInfo, - false, // ignoreParseErrors already contains the functionality of - // failOnMissingField - ignoreParseErrors, - timestampFormat); - this.hasMetadata = requestedMetadata.size() > 0; - this.metadataConverters = - createMetadataConverters(jsonRowType, requestedMetadata, schemaInclude); - this.producedTypeInfo = producedTypeInfo; - this.schemaInclude = schemaInclude; - this.ignoreParseErrors = ignoreParseErrors; - } - - @Override - public void open(InitializationContext context) throws Exception { - jsonDeserializer.open(context); - } - - @Override - public RowData deserialize(byte[] message) { - throw new RuntimeException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); - } - - @Override - public void deserialize(byte[] message, Collector out) throws IOException { - if (message == null || message.length == 0) { - // skip tombstone messages - return; - } - try { - GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); - GenericRowData payload; - if (schemaInclude) { - payload = (GenericRowData) row.getField(0); - } else { - payload = row; - } - - GenericRowData before = (GenericRowData) payload.getField(0); - GenericRowData after = (GenericRowData) payload.getField(1); - String op = payload.getField(2).toString(); - if (OP_CREATE.equals(op) || OP_READ.equals(op)) { - after.setRowKind(RowKind.INSERT); - emitRow(row, after, out); - } else if (OP_UPDATE.equals(op)) { - if (before == null) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - before.setRowKind(RowKind.UPDATE_BEFORE); - after.setRowKind(RowKind.UPDATE_AFTER); - emitRow(row, before, out); - emitRow(row, after, out); - } else if (OP_DELETE.equals(op)) { - if (before == null) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); - } - before.setRowKind(RowKind.DELETE); - emitRow(row, before, out); - } else { - if (!ignoreParseErrors) { - throw new IOException( - format( - "Unknown \"op\" value \"%s\". The Debezium JSON message is '%s'", - op, new String(message))); - } - } - } catch (Throwable t) { - // a big try catch to protect the processing. - if (!ignoreParseErrors) { - throw new IOException( - format("Corrupt Debezium JSON message '%s'.", new String(message)), t); - } - } - } - - private void emitRow( - GenericRowData rootRow, GenericRowData physicalRow, Collector out) { - // shortcut in case no output projection is required - if (!hasMetadata) { - out.collect(physicalRow); - return; - } - - final int physicalArity = physicalRow.getArity(); - final int metadataArity = metadataConverters.length; - - final GenericRowData producedRow = - new GenericRowData(physicalRow.getRowKind(), physicalArity + metadataArity); - - for (int physicalPos = 0; physicalPos < physicalArity; physicalPos++) { - producedRow.setField(physicalPos, physicalRow.getField(physicalPos)); - } - - for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { - producedRow.setField( - physicalArity + metadataPos, metadataConverters[metadataPos].convert(rootRow)); - } - - out.collect(producedRow); - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return producedTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DebeziumJsonDeserializationSchema that = (DebeziumJsonDeserializationSchema) o; - return Objects.equals(jsonDeserializer, that.jsonDeserializer) - && hasMetadata == that.hasMetadata - && Objects.equals(producedTypeInfo, that.producedTypeInfo) - && schemaInclude == that.schemaInclude - && ignoreParseErrors == that.ignoreParseErrors; - } - - @Override - public int hashCode() { - return Objects.hash( - jsonDeserializer, hasMetadata, producedTypeInfo, schemaInclude, ignoreParseErrors); - } - - // -------------------------------------------------------------------------------------------- - - private static RowType createJsonRowType( - DataType physicalDataType, - List readableMetadata, - boolean schemaInclude) { - DataType payload = - DataTypes.ROW( - DataTypes.FIELD("before", physicalDataType), - DataTypes.FIELD("after", physicalDataType), - DataTypes.FIELD("op", DataTypes.STRING())); - - // append fields that are required for reading metadata in the payload - final List payloadMetadataFields = - readableMetadata.stream() - .filter(m -> m.isJsonPayload) - .map(m -> m.requiredJsonField) - .distinct() - .collect(Collectors.toList()); - payload = DataTypeUtils.appendRowFields(payload, payloadMetadataFields); - - DataType root = payload; - if (schemaInclude) { - // when Debezium Kafka connect enables "value.converter.schemas.enable", - // the JSON will contain "schema" information and we need to extract data from - // "payload". - root = DataTypes.ROW(DataTypes.FIELD("payload", payload)); - } - - // append fields that are required for reading metadata in the root - final List rootMetadataFields = - readableMetadata.stream() - .filter(m -> !m.isJsonPayload) - .map(m -> m.requiredJsonField) - .distinct() - .collect(Collectors.toList()); - root = DataTypeUtils.appendRowFields(root, rootMetadataFields); - - return (RowType) root.getLogicalType(); - } - - private static MetadataConverter[] createMetadataConverters( - RowType jsonRowType, List requestedMetadata, boolean schemaInclude) { - return requestedMetadata.stream() - .map( - m -> { - if (m.isJsonPayload) { - return convertInPayload(jsonRowType, m, schemaInclude); - } else { - return convertInRoot(jsonRowType, m); - } - }) - .toArray(MetadataConverter[]::new); - } - - private static MetadataConverter convertInRoot(RowType jsonRowType, ReadableMetadata metadata) { - final int pos = findFieldPos(metadata, jsonRowType); - return new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData root, int unused) { - return metadata.converter.convert(root, pos); - } - }; - } - - private static MetadataConverter convertInPayload( - RowType jsonRowType, ReadableMetadata metadata, boolean schemaInclude) { - if (schemaInclude) { - final int pos = findFieldPos(metadata, (RowType) jsonRowType.getChildren().get(0)); - return new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(GenericRowData root, int unused) { - final GenericRowData payload = (GenericRowData) root.getField(0); - return metadata.converter.convert(payload, pos); - } - }; - } - return convertInRoot(jsonRowType, metadata); - } - - private static int findFieldPos(ReadableMetadata metadata, RowType jsonRowType) { - return jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Converter that extracts a metadata field from the row (root or payload) that comes out of the - * JSON schema and converts it to the desired data type. - */ - interface MetadataConverter extends Serializable { - - // Method for top-level access. - default Object convert(GenericRowData row) { - return convert(row, -1); - } - - Object convert(GenericRowData row, int pos); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java deleted file mode 100644 index d72fcd23d..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.debezium; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.JsonFormatOptionsUtil; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DeserializationFormatFactory; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; - -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - -import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; -import static org.apache.flink.formats.json.debezium.DebeziumJsonFormatOptions.IGNORE_PARSE_ERRORS; -import static org.apache.flink.formats.json.debezium.DebeziumJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL; -import static org.apache.flink.formats.json.debezium.DebeziumJsonFormatOptions.JSON_MAP_NULL_KEY_MODE; -import static org.apache.flink.formats.json.debezium.DebeziumJsonFormatOptions.SCHEMA_INCLUDE; -import static org.apache.flink.formats.json.debezium.DebeziumJsonFormatOptions.TIMESTAMP_FORMAT; - -/** - * Format factory for providing configured instances of Debezium JSON to RowData {@link - * DeserializationSchema}. - */ -@Internal -public class DebeziumJsonFormatFactory - implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "debezium-json"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateDecodingFormatOptions(formatOptions); - - final boolean schemaInclude = formatOptions.get(SCHEMA_INCLUDE); - - final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS); - - final TimestampFormat timestampFormat = - JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - - return new DebeziumJsonDecodingFormat(schemaInclude, ignoreParseErrors, timestampFormat); - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - validateEncodingFormatOptions(formatOptions); - - TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); - JsonFormatOptions.MapNullKeyMode mapNullKeyMode = - JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); - String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); - - final boolean encodeDecimalAsPlainNumber = - formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - - return new EncodingFormat>() { - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new DebeziumJsonSerializationSchema( - rowType, - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return Collections.emptySet(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(SCHEMA_INCLUDE); - options.add(IGNORE_PARSE_ERRORS); - options.add(TIMESTAMP_FORMAT); - options.add(JSON_MAP_NULL_KEY_MODE); - options.add(JSON_MAP_NULL_KEY_LITERAL); - options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER); - return options; - } - - /** Validator for debezium decoding format. */ - private static void validateDecodingFormatOptions(ReadableConfig tableOptions) { - JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions); - } - - /** Validator for debezium encoding format. */ - private static void validateEncodingFormatOptions(ReadableConfig tableOptions) { - JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions); - - // validator for {@link SCHEMA_INCLUDE} - if (tableOptions.get(SCHEMA_INCLUDE)) { - throw new ValidationException( - String.format( - "Debezium JSON serialization doesn't support '%s.%s' option been set to true.", - IDENTIFIER, SCHEMA_INCLUDE.key())); - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatOptions.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatOptions.java deleted file mode 100644 index bf338a9f9..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatOptions.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.debezium; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.formats.json.JsonFormatOptions; - -/** Option utils for debezium-json format. */ -@PublicEvolving -public class DebeziumJsonFormatOptions { - - public static final ConfigOption SCHEMA_INCLUDE = - ConfigOptions.key("schema-include") - .booleanType() - .defaultValue(false) - .withDescription( - "When setting up a Debezium Kafka Connect, users can enable " - + "a Kafka configuration 'value.converter.schemas.enable' to include schema in the message. " - + "This option indicates the Debezium JSON data include the schema in the message or not. " - + "Default is false."); - - public static final ConfigOption IGNORE_PARSE_ERRORS = - JsonFormatOptions.IGNORE_PARSE_ERRORS; - - public static final ConfigOption TIMESTAMP_FORMAT = JsonFormatOptions.TIMESTAMP_FORMAT; - - public static final ConfigOption JSON_MAP_NULL_KEY_MODE = - JsonFormatOptions.MAP_NULL_KEY_MODE; - - public static final ConfigOption JSON_MAP_NULL_KEY_LITERAL = - JsonFormatOptions.MAP_NULL_KEY_LITERAL; - - private DebeziumJsonFormatOptions() {} -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java b/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java deleted file mode 100644 index 0dc9a96b0..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.debezium; - -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.JsonRowDataSerializationSchema; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; - -import java.util.Objects; - -import static java.lang.String.format; -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; - -/** - * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium - * JSON. - * - * @see Debezium - */ -public class DebeziumJsonSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = 1L; - - private static final StringData OP_INSERT = StringData.fromString("c"); // insert - private static final StringData OP_DELETE = StringData.fromString("d"); // delete - - /** The serializer to serialize Debezium JSON data. * */ - private final JsonRowDataSerializationSchema jsonSerializer; - - private transient GenericRowData genericRowData; - - public DebeziumJsonSerializationSchema( - RowType rowType, - TimestampFormat timestampFormat, - JsonFormatOptions.MapNullKeyMode mapNullKeyMode, - String mapNullKeyLiteral, - boolean encodeDecimalAsPlainNumber) { - jsonSerializer = - new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - } - - @Override - public void open(InitializationContext context) throws Exception { - jsonSerializer.open(context); - genericRowData = new GenericRowData(3); - } - - @Override - public byte[] serialize(RowData rowData) { - try { - switch (rowData.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - genericRowData.setField(0, null); - genericRowData.setField(1, rowData); - genericRowData.setField(2, OP_INSERT); - return jsonSerializer.serialize(genericRowData); - case UPDATE_BEFORE: - case DELETE: - genericRowData.setField(0, rowData); - genericRowData.setField(1, null); - genericRowData.setField(2, OP_DELETE); - return jsonSerializer.serialize(genericRowData); - default: - throw new UnsupportedOperationException( - format( - "Unsupported operation '%s' for row kind.", - rowData.getRowKind())); - } - } catch (Throwable t) { - throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DebeziumJsonSerializationSchema that = (DebeziumJsonSerializationSchema) o; - return Objects.equals(jsonSerializer, that.jsonSerializer); - } - - @Override - public int hashCode() { - return Objects.hash(jsonSerializer); - } - - private static RowType createJsonRowType(DataType databaseSchema) { - // Debezium JSON contains some other information, e.g. "source", "ts_ms" - // but we don't need them. - return (RowType) - DataTypes.ROW( - DataTypes.FIELD("before", databaseSchema), - DataTypes.FIELD("after", databaseSchema), - DataTypes.FIELD("op", DataTypes.STRING())) - .getLogicalType(); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory deleted file mode 100644 index 3b83658fc..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -org.apache.flink.formats.json.debezium.DebeziumJsonFormatFactory diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java deleted file mode 100644 index ad61f218b..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.architecture; - -import org.apache.flink.architecture.common.ImportOptions; - -import com.tngtech.archunit.core.importer.ImportOption; -import com.tngtech.archunit.junit.AnalyzeClasses; -import com.tngtech.archunit.junit.ArchTest; -import com.tngtech.archunit.junit.ArchTests; - -/** Architecture tests for test code. */ -@AnalyzeClasses( - packages = {"org.apache.flink.formats.json"}, - importOptions = { - ImportOption.OnlyIncludeTests.class, - ImportOptions.ExcludeScalaImportOption.class, - ImportOptions.ExcludeShadedImportOption.class - }) -public class TestCodeArchitectureTest { - - @ArchTest - public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java deleted file mode 100644 index 0c84351f0..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFileSystemITCase.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.debezium; - -import org.apache.flink.table.planner.runtime.utils.StreamingTestBase; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.CollectionUtil; - -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; - -import static java.lang.String.format; -import static org.assertj.core.api.Assertions.assertThat; - -/** Test Filesystem connector with DebeziumJson. */ -class DebeziumJsonFileSystemITCase extends StreamingTestBase { - - private static final List EXPECTED = - Arrays.asList( - "+I[101, SCOOTER, Small 2-wheel scooter, 3.14]", - "+I[102, CAR BATTERY, 12V car battery, 8.1]", - "+I[103, 12-PACK DRILL BITS, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+I[104, HAMMER, 12oz carpenter's hammer, 0.75]", - "+I[105, HAMMER, 14oz carpenter's hammer, 0.875]", - "+I[106, HAMMER, 16oz carpenter's hammer, 1.0]", - "+I[107, ROCKS, box of assorted rocks, 5.3]", - "+I[108, JACKET, water resistent black wind breaker, 0.1]", - "+I[109, SPARE TIRE, 24 inch spare tire, 22.2]", - "-D[106, HAMMER, 16oz carpenter's hammer, 1.0]", // -U - "+I[106, HAMMER, 18oz carpenter hammer, 1.0]", // +U - "-D[107, ROCKS, box of assorted rocks, 5.3]", // -U - "+I[107, ROCKS, box of assorted rocks, 5.1]", // +U - "+I[110, JACKET, water resistent white wind breaker, 0.2]", - "+I[111, SCOOTER, Big 2-wheel scooter , 5.18]", - "-D[110, JACKET, water resistent white wind breaker, 0.2]", // -U - "+I[110, JACKET, new water resistent white wind breaker, 0.5]", // +U - "-D[111, SCOOTER, Big 2-wheel scooter , 5.18]", // -U - "+I[111, SCOOTER, Big 2-wheel scooter , 5.17]", // +U - "-D[111, SCOOTER, Big 2-wheel scooter , 5.17]"); - - private File source; - private File sink; - - private void prepareTables(boolean isPartition, Path tempSourceDir, Path tempSinkDir) - throws IOException { - byte[] bytes = readBytes("debezium-data-schema-exclude.txt"); - source = tempSourceDir.toFile(); - File file; - if (isPartition) { - File partition = new File(source, "p=1"); - partition.mkdirs(); - file = new File(partition, "my_file"); - } else { - file = new File(source, "my_file"); - } - file.createNewFile(); - Files.write(file.toPath(), bytes); - - sink = tempSinkDir.toFile(); - - env().setParallelism(1); - } - - private void createTable(boolean isSink, String path, boolean isPartition) { - tEnv().executeSql( - format("create table %s (", isSink ? "sink" : "source") - + "id int, name string," - + (isSink ? "upper_name string," : "") - + " description string, weight float" - + (isPartition ? ", p int) partitioned by (p) " : ")") - + " with (" - + "'connector'='filesystem'," - + "'format'='debezium-json'," - + format("'path'='%s'", path) - + ")"); - } - - @Test - void testNonPartition(@TempDir Path tempSourceDir, @TempDir Path tempSinkDir) throws Exception { - prepareTables(false, tempSourceDir, tempSinkDir); - createTable(false, source.toURI().toString(), false); - createTable(true, sink.toURI().toString(), false); - - tEnv().executeSql( - "insert into sink select id,name,UPPER(name),description,weight from source") - .await(); - CloseableIterator iter = - tEnv().executeSql("select id,upper_name,description,weight from sink").collect(); - - List results = - CollectionUtil.iteratorToList(iter).stream() - .map(Row::toString) - .collect(Collectors.toList()); - iter.close(); - - assertThat(results).isEqualTo(EXPECTED); - } - - @Test - void testPartition(@TempDir Path tempSourceDir, @TempDir Path tempSinkDir) throws Exception { - prepareTables(true, tempSourceDir, tempSinkDir); - createTable(false, source.toURI().toString(), true); - createTable(true, sink.toURI().toString(), true); - - tEnv().executeSql( - "insert into sink select id,name,UPPER(name),description,weight,p from source") - .await(); - CloseableIterator iter = - tEnv().executeSql("select id,upper_name,description,weight,p from sink").collect(); - List list = CollectionUtil.iteratorToList(iter); - iter.close(); - - List results = - list.stream() - .map(row -> Row.project(row, new int[] {0, 1, 2, 3})) - .map(Row::toString) - .collect(Collectors.toList()); - - assertThat(results).isEqualTo(EXPECTED); - - // check partition value - for (Row row : list) { - assertThat(row.getField(4)).isEqualTo(1); - } - } - - private static byte[] readBytes(String resource) throws IOException { - final URL url = DebeziumJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return Files.readAllBytes(path); - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java deleted file mode 100644 index d000877b2..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactoryTest.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.debezium; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.TestDynamicTableFactory; -import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; -import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.logical.RowType; - -import org.junit.jupiter.api.Test; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.function.Consumer; - -import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; -import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_DATA_TYPE; -import static org.apache.flink.table.factories.utils.FactoryMocks.PHYSICAL_TYPE; -import static org.apache.flink.table.factories.utils.FactoryMocks.SCHEMA; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link DebeziumJsonFormatFactory}. */ -class DebeziumJsonFormatFactoryTest { - - @Test - void testSeDeSchema() { - final DebeziumJsonDeserializationSchema expectedDeser = - new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_TYPE), - false, - true, - TimestampFormat.ISO_8601); - - final Map options = getAllOptions(); - - final DynamicTableSource actualSource = createTableSource(SCHEMA, options); - assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock; - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - DeserializationSchema actualDeser = - scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); - - assertThat(actualDeser).isEqualTo(expectedDeser); - - final DebeziumJsonSerializationSchema expectedSer = - new DebeziumJsonSerializationSchema( - (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), - TimestampFormat.ISO_8601, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - - final DynamicTableSink actualSink = createTableSink(SCHEMA, options); - assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock; - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = - sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); - - assertThat(actualSer).isEqualTo(expectedSer); - } - - @Test - void testInvalidIgnoreParseError() { - final Map options = - getModifiedOptions(opts -> opts.put("debezium-json.ignore-parse-errors", "abc")); - - assertThatThrownBy(() -> createTableSource(SCHEMA, options)) - .satisfies( - anyCauseMatches( - IllegalArgumentException.class, - "Unrecognized option for boolean: abc. " - + "Expected either true or false(case insensitive)")); - } - - @Test - void testSchemaIncludeOption() { - Map options = getAllOptions(); - options.put("debezium-json.schema-include", "true"); - - final DebeziumJsonDeserializationSchema expectedDeser = - new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - true, - true, - TimestampFormat.ISO_8601); - final DynamicTableSource actualSource = createTableSource(SCHEMA, options); - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - DeserializationSchema actualDeser = - scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, PHYSICAL_DATA_TYPE); - assertThat(actualDeser).isEqualTo(expectedDeser); - - assertThatThrownBy( - () -> { - final DynamicTableSink actualSink = createTableSink(SCHEMA, options); - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), PHYSICAL_DATA_TYPE); - }) - .satisfies( - anyCauseMatches( - RuntimeException.class, - "Debezium JSON serialization doesn't support " - + "'debezium-json.schema-include' option been set to true.")); - } - - @Test - void testInvalidOptionForTimestampFormat() { - final Map tableOptions = - getModifiedOptions( - opts -> opts.put("debezium-json.timestamp-format.standard", "test")); - - assertThatThrownBy(() -> createTableSource(SCHEMA, tableOptions)) - .isInstanceOf(ValidationException.class) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'test' for timestamp-format.standard. " - + "Supported values are [SQL, ISO-8601].")); - } - - @Test - void testInvalidOptionForMapNullKeyMode() { - final Map tableOptions = - getModifiedOptions(opts -> opts.put("debezium-json.map-null-key.mode", "invalid")); - - assertThatThrownBy(() -> createTableSink(SCHEMA, tableOptions)) - .isInstanceOf(ValidationException.class) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Unsupported value 'invalid' for option map-null-key.mode. " - + "Supported values are [LITERAL, FAIL, DROP].")); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifiedOptions(Consumer> optionModifier) { - Map options = getAllOptions(); - optionModifier.accept(options); - return options; - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", "debezium-json"); - options.put("debezium-json.ignore-parse-errors", "true"); - options.put("debezium-json.timestamp-format.standard", "ISO-8601"); - options.put("debezium-json.map-null-key.mode", "LITERAL"); - options.put("debezium-json.map-null-key.literal", "null"); - options.put("debezium-json.encode.decimal-as-plain-number", "true"); - return options; - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java b/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java deleted file mode 100644 index 3b9151f33..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ /dev/null @@ -1,342 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.json.debezium; - -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonFormatOptions; -import org.apache.flink.formats.json.debezium.DebeziumJsonDecodingFormat.ReadableMetadata; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.function.Consumer; -import java.util.stream.Collectors; - -import static org.apache.flink.connector.testutils.formats.SchemaTestUtils.open; -import static org.apache.flink.table.api.DataTypes.FIELD; -import static org.apache.flink.table.api.DataTypes.FLOAT; -import static org.apache.flink.table.api.DataTypes.INT; -import static org.apache.flink.table.api.DataTypes.ROW; -import static org.apache.flink.table.api.DataTypes.STRING; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** - * Tests for {@link DebeziumJsonSerializationSchema} and {@link DebeziumJsonDeserializationSchema}. - */ -class DebeziumJsonSerDeSchemaTest { - - private static final DataType PHYSICAL_DATA_TYPE = - ROW( - FIELD("id", INT().notNull()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", FLOAT())); - - @Test - void testSerializationAndSchemaIncludeDeserialization() throws Exception { - testSerializationDeserialization("debezium-data-schema-include.txt", true); - } - - @Test - void testSerializationAndSchemaExcludeDeserialization() throws Exception { - testSerializationDeserialization("debezium-data-schema-exclude.txt", false); - } - - @Test - void testSerializationAndPostgresSchemaIncludeDeserialization() throws Exception { - testSerializationDeserialization("debezium-postgres-data-schema-include.txt", true); - } - - @Test - void testSerializationAndPostgresSchemaExcludeDeserialization() throws Exception { - testSerializationDeserialization("debezium-postgres-data-schema-exclude.txt", false); - } - - @Test - void testPostgresDefaultReplicaIdentify() { - assertThatThrownBy( - () -> - testSerializationDeserialization( - "debezium-postgres-data-replica-identity.txt", false)) - .as( - "The \"before\" field of UPDATE message is null, if you are using Debezium Postgres Connector, " - + "please check the Postgres table has been set REPLICA IDENTITY to FULL level.") - .isInstanceOf(Exception.class); - } - - @Test - void testTombstoneMessages() throws Exception { - DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - false, - false, - TimestampFormat.ISO_8601); - SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(null, collector); - deserializationSchema.deserialize(new byte[] {}, collector); - assertThat(collector.list).isEmpty(); - } - - @Test - void testDeserializationWithMetadata() throws Exception { - testDeserializationWithMetadata( - "debezium-data-schema-include.txt", - true, - row -> { - assertThat(row.getInt(0)).isEqualTo(101); - assertThat(row.getString(1).toString()).isEqualTo("scooter"); - assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter"); - assertThat(row.getFloat(3)).isEqualTo(3.14f); - assertThat(row.getString(4).toString()) - .startsWith( - "{\"type\":\"struct\",\"fields\":[{\"type\":\"struct\",\"fields\":[{\"type\":\"int32\",\"optional\":false,\"field\":\"id\"},"); - assertThat(row.getTimestamp(5, 3).getMillisecond()).isEqualTo(1589355606100L); - assertThat(row.getTimestamp(6, 3).getMillisecond()).isEqualTo(0L); - assertThat(row.getString(7).toString()).isEqualTo("inventory"); - assertThat(row.isNullAt(8)).isEqualTo(true); - assertThat(row.getString(9).toString()).isEqualTo("products"); - assertThat(row.getMap(10).size()).isEqualTo(14); - }); - - testDeserializationWithMetadata( - "debezium-data-schema-exclude.txt", - false, - row -> { - assertThat(row.getInt(0)).isEqualTo(101); - assertThat(row.getString(1).toString()).isEqualTo("scooter"); - assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter"); - assertThat(row.getFloat(3)).isEqualTo(3.14f); - assertThat(row.isNullAt(4)).isEqualTo(true); - assertThat(row.getTimestamp(5, 3).getMillisecond()).isEqualTo(1589355606100L); - assertThat(row.getTimestamp(6, 3).getMillisecond()).isEqualTo(0L); - assertThat(row.getString(7).toString()).isEqualTo("inventory"); - assertThat(row.isNullAt(8)).isEqualTo(true); - assertThat(row.getString(9).toString()).isEqualTo("products"); - assertThat(row.getMap(10).size()).isEqualTo(14); - }); - - testDeserializationWithMetadata( - "debezium-postgres-data-schema-exclude.txt", - false, - row -> { - assertThat(row.getInt(0)).isEqualTo(101); - assertThat(row.getString(1).toString()).isEqualTo("scooter"); - assertThat(row.getString(2).toString()).isEqualTo("Small 2-wheel scooter"); - assertThat(row.getFloat(3)).isEqualTo(3.14f); - assertThat(row.isNullAt(4)).isEqualTo(true); - assertThat(row.getTimestamp(5, 3).getMillisecond()).isEqualTo(1596001099434L); - assertThat(row.getTimestamp(6, 3).getMillisecond()).isEqualTo(1596001099434L); - assertThat(row.getString(7).toString()).isEqualTo("postgres"); - assertThat(row.getString(8).toString()).isEqualTo("inventory"); - assertThat(row.getString(9).toString()).isEqualTo("products"); - assertThat(row.getMap(10).size()).isEqualTo(11); - }); - } - - private void testSerializationDeserialization(String resourceFile, boolean schemaInclude) - throws Exception { - List lines = readLines(resourceFile); - DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - Collections.emptyList(), - InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType()), - schemaInclude, - false, - TimestampFormat.ISO_8601); - open(deserializationSchema); - - SimpleCollector collector = new SimpleCollector(); - for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); - } - - // Debezium captures change data (`debezium-data-schema-include.txt`) on the `product` - // table: - // - // CREATE TABLE product ( - // id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - // name VARCHAR(255), - // description VARCHAR(512), - // weight FLOAT - // ); - // ALTER TABLE product AUTO_INCREMENT = 101; - // - // INSERT INTO product - // VALUES (default,"scooter","Small 2-wheel scooter",3.14), - // (default,"car battery","12V car battery",8.1), - // (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 - // to #3",0.8), - // (default,"hammer","12oz carpenter's hammer",0.75), - // (default,"hammer","14oz carpenter's hammer",0.875), - // (default,"hammer","16oz carpenter's hammer",1.0), - // (default,"rocks","box of assorted rocks",5.3), - // (default,"jacket","water resistent black wind breaker",0.1), - // (default,"spare tire","24 inch spare tire",22.2); - // UPDATE product SET description='18oz carpenter hammer' WHERE id=106; - // UPDATE product SET weight='5.1' WHERE id=107; - // INSERT INTO product VALUES (default,"jacket","water resistent white wind breaker",0.2); - // INSERT INTO product VALUES (default,"scooter","Big 2-wheel scooter ",5.18); - // UPDATE product SET description='new water resistent white wind breaker', weight='0.5' - // WHERE id=110; - // UPDATE product SET weight='5.17' WHERE id=111; - // DELETE FROM product WHERE id=111; - List expected = - Arrays.asList( - "+I(101,scooter,Small 2-wheel scooter,3.14)", - "+I(102,car battery,12V car battery,8.1)", - "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8)", - "+I(104,hammer,12oz carpenter's hammer,0.75)", - "+I(105,hammer,14oz carpenter's hammer,0.875)", - "+I(106,hammer,16oz carpenter's hammer,1.0)", - "+I(107,rocks,box of assorted rocks,5.3)", - "+I(108,jacket,water resistent black wind breaker,0.1)", - "+I(109,spare tire,24 inch spare tire,22.2)", - "-U(106,hammer,16oz carpenter's hammer,1.0)", - "+U(106,hammer,18oz carpenter hammer,1.0)", - "-U(107,rocks,box of assorted rocks,5.3)", - "+U(107,rocks,box of assorted rocks,5.1)", - "+I(110,jacket,water resistent white wind breaker,0.2)", - "+I(111,scooter,Big 2-wheel scooter ,5.18)", - "-U(110,jacket,water resistent white wind breaker,0.2)", - "+U(110,jacket,new water resistent white wind breaker,0.5)", - "-U(111,scooter,Big 2-wheel scooter ,5.18)", - "+U(111,scooter,Big 2-wheel scooter ,5.17)", - "-D(111,scooter,Big 2-wheel scooter ,5.17)"); - List actual = - collector.list.stream().map(Object::toString).collect(Collectors.toList()); - assertThat(actual).isEqualTo(expected); - - DebeziumJsonSerializationSchema serializationSchema = - new DebeziumJsonSerializationSchema( - (RowType) PHYSICAL_DATA_TYPE.getLogicalType(), - TimestampFormat.SQL, - JsonFormatOptions.MapNullKeyMode.LITERAL, - "null", - true); - - open(serializationSchema); - actual = new ArrayList<>(); - for (RowData rowData : collector.list) { - actual.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); - } - - expected = - Arrays.asList( - "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op\":\"c\"}", - "{\"before\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"after\":null,\"op\":\"d\"}", - "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op\":\"c\"}", - "{\"before\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"after\":null,\"op\":\"d\"}", - "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op\":\"c\"}", - "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op\":\"c\"}", - "{\"before\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"after\":null,\"op\":\"d\"}", - "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op\":\"c\"}", - "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":null,\"op\":\"d\"}", - "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op\":\"c\"}", - "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op\":\"d\"}"); - assertThat(actual).isEqualTo(expected); - } - - private void testDeserializationWithMetadata( - String resourceFile, boolean schemaInclude, Consumer testConsumer) - throws Exception { - // we only read the first line for keeping the test simple - final String firstLine = readLines(resourceFile).get(0); - - final List requestedMetadata = Arrays.asList(ReadableMetadata.values()); - - final DataType producedDataType = - DataTypeUtils.appendRowFields( - PHYSICAL_DATA_TYPE, - requestedMetadata.stream() - .map(m -> DataTypes.FIELD(m.key, m.dataType)) - .collect(Collectors.toList())); - - final DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, - requestedMetadata, - InternalTypeInfo.of(producedDataType.getLogicalType()), - schemaInclude, - false, - TimestampFormat.ISO_8601); - open(deserializationSchema); - - final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(firstLine.getBytes(StandardCharsets.UTF_8), collector); - - assertThat(collector.list).hasSize(1); - assertThat(collector.list.get(0)).satisfies(testConsumer); - } - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - private static List readLines(String resource) throws IOException { - final URL url = DebeziumJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); - assert url != null; - Path path = new File(url.getFile()).toPath(); - return Files.readAllLines(path); - } - - private static class SimpleCollector implements Collector { - - private List list = new ArrayList<>(); - - @Override - public void collect(RowData record) { - list.add(record); - } - - @Override - public void close() { - // do nothing - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/flink-formats-kafka/flink-json-debezium/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension deleted file mode 100644 index 28999133c..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -org.apache.flink.util.TestLoggerExtension \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/archunit.properties b/flink-formats-kafka/flink-json-debezium/src/test/resources/archunit.properties deleted file mode 100644 index 15be88c95..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/archunit.properties +++ /dev/null @@ -1,31 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# By default we allow removing existing violations, but fail when new violations are added. -freeze.store.default.allowStoreUpdate=true - -# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. -#freeze.store.default.allowStoreCreation=true - -# Enable this to add allow new violations to be recorded. -# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new -# violation, please try to avoid creating the violation. If the violation was created due to a -# shortcoming of the rule, file a JIRA issue so the rule can be improved. -#freeze.refreeze=true - -freeze.store.default.path=archunit-violations diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt deleted file mode 100644 index 68661d762..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data-filter-table.txt +++ /dev/null @@ -1,16 +0,0 @@ -{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"106","name":"hammer","description":null,"weight":"1.0"},{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"mydb","es":1598944132000,"id":1,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944146308,"type":"INSERT"} -{"data":[{"id":"106","name":"hammer","description":"18oz carpenter hammer","weight":"1.0"}],"database":"mydb","es":1598944202000,"id":2,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"description":null}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944202218,"type":"UPDATE"} -{"data":null,"database":"mydb","es":1598944271000,"id":3,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE TABLE orders (\n order_number INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,\n order_date DATE NOT NULL,\n purchaser INTEGER NOT NULL,\n quantity INTEGER NOT NULL,\n product_id INTEGER NOT NULL\n) AUTO_INCREMENT = 10001","sqlType":null,"table":"orders","ts":1598944271192,"type":"CREATE"} -{"data":[{"order_number":"10001","order_date":"2016-01-16","purchaser":"1001","quantity":"1","product_id":"102"},{"order_number":"10002","order_date":"2016-01-17","purchaser":"1002","quantity":"2","product_id":"105"},{"order_number":"10003","order_date":"2016-02-19","purchaser":"1002","quantity":"2","product_id":"106"},{"order_number":"10004","order_date":"2016-02-21","purchaser":"1003","quantity":"1","product_id":"107"}],"database":"mydb","es":1598944275000,"id":4,"isDdl":false,"mysqlType":{"order_number":"INTEGER","order_date":"DATE","purchaser":"INTEGER","quantity":"INTEGER","product_id":"INTEGER"},"old":null,"pkNames":["order_number"],"sql":"","sqlType":{"order_number":4,"order_date":91,"purchaser":4,"quantity":4,"product_id":4},"table":"orders","ts":1598944275018,"type":"INSERT"} -{"data":[{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.1"}],"database":"mydb","es":1598944279000,"id":5,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944279665,"type":"UPDATE"} -{"data":[{"id":"110","name":"jacket","description":"water resistent white wind breaker","weight":"0.2"}],"database":"mydb","es":1598944288000,"id":6,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944288394,"type":"INSERT"} -{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.18"}],"database":"mydb","es":1598944288000,"id":6,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944288394,"type":"INSERT"} -{"data":[{"id":"110","name":"jacket","description":"new water resistent white wind breaker","weight":"0.5"}],"database":"mydb","es":1598944288000,"id":7,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"description":"water resistent white wind breaker","weight":"0.2"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944288717,"type":"UPDATE"} -{"data":[{"order_number":"10001","order_date":"2016-01-16","purchaser":"1001","quantity":"3","product_id":"102"}],"database":"mydb","es":1598944331000,"id":8,"isDdl":false,"mysqlType":{"order_number":"INTEGER","order_date":"DATE","purchaser":"INTEGER","quantity":"INTEGER","product_id":"INTEGER"},"old":[{"quantity":"1"}],"pkNames":["order_number"],"sql":"","sqlType":{"order_number":4,"order_date":91,"purchaser":4,"quantity":4,"product_id":4},"table":"orders","ts":1598944331870,"type":"UPDATE"} -{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"mydb","es":1598944337000,"id":9,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"weight":"5.18"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944337341,"type":"UPDATE"} -{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"mydb","es":1598944337000,"id":9,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944337341,"type":"DELETE"} -{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"5.17"},{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"}],"database":"mydb","es":1598944337000,"id":10,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":[{"weight":"3.14"},{"weight":"8.1"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944337663,"type":"UPDATE"} -{"data":[{"order_number":"10002","order_date":"2016-01-17","purchaser":"1002","quantity":"2","product_id":"105"}],"database":"mydb","es":1598944374000,"id":11,"isDdl":false,"mysqlType":{"order_number":"INTEGER","order_date":"DATE","purchaser":"INTEGER","quantity":"INTEGER","product_id":"INTEGER"},"old":null,"pkNames":["order_number"],"sql":"","sqlType":{"order_number":4,"order_date":91,"purchaser":4,"quantity":4,"product_id":4},"table":"orders","ts":1598944374999,"type":"DELETE"} -{"data":[{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"}],"database":"mydb","es":1598944418000,"id":12,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"product","ts":1598944418418,"type":"DELETE"} -{"data":null,"database":"mydb","es":1598944271000,"id":13,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE TABLE project (\n id VARCHAR(255) NOT NULL,\n name VARCHAR(255) NOT NULL,\n description VARCHAR(255) NOT NULL,\n weight FLOAT NOT NULL\n)","sqlType":null,"table":"projects","ts":1598944271192,"type":"CREATE"} -{"data":[{"id":"A101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"A102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"A103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"A104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"A105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"A106","name":"hammer","description":"16oz carpenter's hammer","weight":"1.0"},{"id":"A107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"A108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"A109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"mydb","es":1598944132000,"id":14,"isDdl":false,"mysqlType":{"id":"int(11)","name":"varchar(255)","description":"varchar(512)","weight":"float"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"project","ts":1598944146308,"type":"INSERT"} \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt deleted file mode 100644 index a83b7da4f..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/canal-data.txt +++ /dev/null @@ -1,11 +0,0 @@ -{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"106","name":"hammer","description":null,"weight":"1.0"},{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"inventory","es":1589373515000,"id":3,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373515477,"type":"INSERT"} -{"data":[{"id":"106","name":"hammer","description":"18oz carpenter hammer","weight":"1.0"}],"database":"inventory","es":1589373546000,"id":4,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"description":null}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373546301,"type":"UPDATE"} -{"data":[{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.1"}],"database":"inventory","es":1589373549000,"id":5,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373549489,"type":"UPDATE"} -{"data":[{"id":"110","name":"jacket","description":"water resistent white wind breaker","weight":"0.2"}],"database":"inventory","es":1589373552000,"id":6,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373552882,"type":"INSERT"} -{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.18"}],"database":"inventory","es":1589373555000,"id":7,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373555457,"type":"INSERT"} -{"data":[{"id":"110","name":"jacket","description":"new water resistent white wind breaker","weight":"0.5"}],"database":"inventory","es":1589373558000,"id":8,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"description":"water resistent white wind breaker","weight":"0.2"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373558230,"type":"UPDATE"} -{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"inventory","es":1589373560000,"id":9,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"5.18"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373560798,"type":"UPDATE"} -{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"inventory","es":1589373563000,"id":10,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373563798,"type":"DELETE"} -{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"5.17"},{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"}],"database":"inventory","es":1589373753000,"id":11,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"3.14"},{"weight":"8.1"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373753939,"type":"UPDATE"} -{"data":null,"database":"inventory","es":1589373566000,"id":13,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE TABLE `xj_`.`user02` (`uid` int(0) NOT NULL,`uname` varchar(255) NULL, PRIMARY KEY (`uid`))","sqlType":null,"table":"user02","ts":1589373566000,"type":"CREATE"} -{"data":[{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"}],"database":"inventory","es":1589374013000,"id":12,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589374013680,"type":"DELETE"} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/complex-schema.json b/flink-formats-kafka/flink-json-debezium/src/test/resources/complex-schema.json deleted file mode 100644 index 86664284e..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/complex-schema.json +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -{ - "$schema": "http://json-schema.org/draft-06/schema#", - "description": "A representation of a person, company, organization, or place", - "type": "object", - "required": [ - "familyName", - "givenName" - ], - "properties": { - "fn": { - "description": "Formatted Name", - "type": "string" - }, - "familyName": { - "type": "string" - }, - "additionalName": { - "type": "boolean" - }, - "tuples": { - "type": "array", - "items": [ - { - "type": "number" - }, - { - "type": "string" - }, - { - "type": "string", - "enum": [ - "Street", - "Avenue", - "Boulevard" - ] - }, - { - "type": "string", - "enum": [ - "NW", - "NE", - "SW", - "SE" - ] - } - ], - "additionalItems": false - }, - "honorificPrefix": { - "type": "array", - "items": { - "type": "string" - } - }, - "url": { - "type": "string", - "format": "uri" - }, - "email": { - "type": "object", - "properties": { - "type": { - "type": "string" - }, - "value": { - "type": "string", - "format": "email" - } - } - }, - tel: { - "type": "object", - "properties": { - "type": { - "type": "integer" - }, - "value": { - "type": "string", - "format": "phone" - } - } - }, - "sound": { - "type": "null" - }, - "org": { - "type": "object", - "properties": { - "organizationUnit": { - "type": "object", - "properties": {} - } - } - } - } -} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-exclude.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-exclude.txt deleted file mode 100644 index 3763369e4..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-exclude.txt +++ /dev/null @@ -1,16 +0,0 @@ -{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606100,"transaction":null} -{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} -{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} -{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} -{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} -{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} -{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} -{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} -{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} -{"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589361987000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":362,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589361987936,"transaction":null} -{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362099000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362099505,"transaction":null} -{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362210230,"transaction":null} -{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362243000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1394,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362243428,"transaction":null} -{"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362293000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1707,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362293539,"transaction":null} -{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362330000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2090,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362330904,"transaction":null} -{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362344000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"d","ts_ms":1589362344455,"transaction":null} \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-include.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-include.txt deleted file mode 100644 index b3e0f7d5b..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-data-schema-include.txt +++ /dev/null @@ -1,16 +0,0 @@ -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606100,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589361987000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":362,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589361987936,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362099000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362099505,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362210230,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362243000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1394,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362243428,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362293000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1707,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362293539,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362330000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2090,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362330904,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362344000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"d","ts_ms":1589362344455,"transaction":null}} \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-replica-identity.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-replica-identity.txt deleted file mode 100644 index d4fcb8898..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-replica-identity.txt +++ /dev/null @@ -1,16 +0,0 @@ -{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099434,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099434,"transaction":null} -{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null} -{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null} -{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} -{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} -{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} -{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null} -{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null} -{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099438,"transaction":null} -{"before":null,"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010889629,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":602,"lsn":34131104,"xmin":null},"op":"u","ts_ms":1596010890411,"transaction":null} -{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010930407,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":603,"lsn":34132200,"xmin":null},"op":"u","ts_ms":1596010930623,"transaction":null} -{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010946488,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":604,"lsn":34132560,"xmin":null},"op":"c","ts_ms":1596010946870,"transaction":null} -{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010976756,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":605,"lsn":34133072,"xmin":null},"op":"c","ts_ms":1596010976880,"transaction":null} -{"before":null,"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010982228,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":606,"lsn":34133344,"xmin":null},"op":"u","ts_ms":1596010982481,"transaction":null} -{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010985627,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":607,"lsn":34133584,"xmin":null},"op":"u","ts_ms":1596010986047,"transaction":null} -{"before":null,"after":null,"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010988168,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":608,"lsn":34133800,"xmin":null},"op":"d","ts_ms":1596010988596,"transaction":null} \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-exclude.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-exclude.txt deleted file mode 100644 index 993f5f489..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-exclude.txt +++ /dev/null @@ -1,16 +0,0 @@ -{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099434,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099434,"transaction":null} -{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null} -{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null} -{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} -{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} -{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null} -{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null} -{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null} -{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099438,"transaction":null} -{"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010889629,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":602,"lsn":34131104,"xmin":null},"op":"u","ts_ms":1596010890411,"transaction":null} -{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010930407,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":603,"lsn":34132200,"xmin":null},"op":"u","ts_ms":1596010930623,"transaction":null} -{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010946488,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":604,"lsn":34132560,"xmin":null},"op":"c","ts_ms":1596010946870,"transaction":null} -{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010976756,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":605,"lsn":34133072,"xmin":null},"op":"c","ts_ms":1596010976880,"transaction":null} -{"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010982228,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":606,"lsn":34133344,"xmin":null},"op":"u","ts_ms":1596010982481,"transaction":null} -{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010985627,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":607,"lsn":34133584,"xmin":null},"op":"u","ts_ms":1596010986047,"transaction":null} -{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"after":null,"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010988168,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":608,"lsn":34133800,"xmin":null},"op":"d","ts_ms":1596010988596,"transaction":null} \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-include.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-include.txt deleted file mode 100644 index 8301935f3..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/debezium-postgres-data-schema-include.txt +++ /dev/null @@ -1,16 +0,0 @@ -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099434,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099434,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099435,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099435,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099436,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099436,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099437,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596001099437,"snapshot":"true","db":"postgres","schema":"inventory","table":"products","txId":601,"lsn":34078720,"xmin":null},"op":"r","ts_ms":1596001099438,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010889629,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":602,"lsn":34131104,"xmin":null},"op":"u","ts_ms":1596010890411,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010930407,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":603,"lsn":34132200,"xmin":null},"op":"u","ts_ms":1596010930623,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010946488,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":604,"lsn":34132560,"xmin":null},"op":"c","ts_ms":1596010946870,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010976756,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":605,"lsn":34133072,"xmin":null},"op":"c","ts_ms":1596010976880,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010982228,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":606,"lsn":34133344,"xmin":null},"op":"u","ts_ms":1596010982481,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010985627,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":607,"lsn":34133584,"xmin":null},"op":"u","ts_ms":1596010986047,"transaction":null}} -{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"fullfillment.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"schema"},{"type":"string","optional":false,"field":"table"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"int64","optional":true,"field":"xmin"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"fullfillment.inventory.products.Envelope"},"payload":{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"after":null,"source":{"version":"1.2.1.Final","connector":"postgresql","name":"fullfillment","ts_ms":1596010988168,"snapshot":"false","db":"postgres","schema":"inventory","table":"products","txId":608,"lsn":34133800,"xmin":null},"op":"d","ts_ms":1596010988596,"transaction":null}} \ No newline at end of file diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt deleted file mode 100644 index 2d33ff7d7..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/maxwell-data.txt +++ /dev/null @@ -1,20 +0,0 @@ -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":0,"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":1,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":2,"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":3,"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":4,"data":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":5,"data":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":6,"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":7,"data":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"commit":true,"data":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"update","ts":1596684893,"xid":7152,"commit":true,"data":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"old":{"description":"16oz carpenter's hammer"},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"update","ts":1596684897,"xid":7169,"commit":true,"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"old":{"weight":5.3},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"insert","ts":1596684900,"xid":7186,"commit":true,"data":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"insert","ts":1596684904,"xid":7201,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"update","ts":1596684906,"xid":7216,"commit":true,"data":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"old":{"description":"water resistent white wind breaker","weight":0.2},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"update","ts":1596684912,"xid":7235,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"old":{"weight":5.18},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"delete","ts":1596684914,"xid":7250,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"update","ts":1596684928,"xid":7291,"xoffset":0,"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":5.17},"old":{"weight":3.14},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"update","ts":1596684928,"xid":7291,"commit":true,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":5.17},"old":{"weight":8.1},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"delete","ts":1596684938,"xid":7322,"xoffset":0,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":5.17},"primary_key_columns": ["id"]} -{"database":"test","table":"product","type":"delete","ts":1596684938,"xid":7322,"commit":true,"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"primary_key_columns": ["id"]} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt b/flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt deleted file mode 100644 index d29592532..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/ogg-data.txt +++ /dev/null @@ -1,16 +0,0 @@ -{"table":"OGG.TBL_TEST","pos":"00000000000000000000143","primary_keys":["id"],"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"op_type":"I", "current_ts":"2020-05-13T13:39:35.766000", "op_ts":"2020-05-13 15:40:06.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000144","primary_keys":["id"],"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000145","primary_keys":["id"],"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000146","primary_keys":["id"],"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000147","primary_keys":["id"],"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000148","primary_keys":["id"],"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000149","primary_keys":["id"],"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000150","primary_keys":["id"],"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000151","primary_keys":["id"],"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"op_type":"I","op_ts":"2020-05-13 15:40:07.000000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000152","primary_keys":["id"],"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1},"op_type":"U","op_ts":"2020-05-13 17:26:27.936000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000153","primary_keys":["id"],"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"op_type":"U","op_ts":"2020-05-13 17:28:19.505000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000154","primary_keys":["id"],"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","op_ts":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op_type":"I","op_ts":"2020-05-13 17:30:10.230000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000155","primary_keys":["id"],"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"op_type":"I","op_ts":"2020-05-13 17:30:43.428000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000156","primary_keys":["id"],"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"op_type":"U","op_ts":"2020-05-13 17:32:20.327000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000157","primary_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"op_type":"U","op_ts":"2020-05-13 17:32:10.904000"} -{"table":"OGG.TBL_TEST","pos":"00000000000000000000158","primary_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"op_type":"D","op_ts":"2020-05-13 17:32:24.455000"} diff --git a/flink-formats-kafka/flink-json-debezium/src/test/resources/reference-schema.json b/flink-formats-kafka/flink-json-debezium/src/test/resources/reference-schema.json deleted file mode 100644 index 99e0e79dc..000000000 --- a/flink-formats-kafka/flink-json-debezium/src/test/resources/reference-schema.json +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -{ - "$schema": "http://json-schema.org/draft-04/schema#", - "definitions": { - "address": { - "type": "object", - "properties": { - "street_address": { - "type": "string" - }, - "city": { - "type": "string" - }, - "state": { - "type": "string" - } - }, - "required": [ - "street_address", - "city", - "state" - ] - } - }, - "type": "object", - "properties": { - "billing_address": { - "$ref": "#/definitions/address" - }, - "shipping_address": { - "$ref": "#/definitions/address" - }, - "optional_address": { - "oneOf": [ - { - "type": "null" - }, - { - "$ref": "#/definitions/address" - } - ] - } - } -} diff --git a/flink-formats-kafka/pom.xml b/flink-formats-kafka/pom.xml index dbe67f52d..8ab527efe 100644 --- a/flink-formats-kafka/pom.xml +++ b/flink-formats-kafka/pom.xml @@ -35,7 +35,6 @@ under the License. flink-avro-confluent-registry - flink-json-debezium flink-sql-avro-confluent-registry From a7785630e714af303b224c38d9a6caa89a551265 Mon Sep 17 00:00:00 2001 From: Alex Gout Date: Mon, 27 Mar 2023 10:07:33 -0400 Subject: [PATCH 212/322] [FLINK-31049] [flink-connector-kafka] Add support for Kafka record headers to KafkaSink Co-Authored-By: Tzu-Li (Gordon) Tai This closes #18. --- .../connector/kafka/sink/HeaderProvider.java | 32 +++++++++++++ ...KafkaRecordSerializationSchemaBuilder.java | 48 +++++++++++++++---- ...aRecordSerializationSchemaBuilderTest.java | 24 ++++++++++ 3 files changed, 96 insertions(+), 8 deletions(-) create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/HeaderProvider.java diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/HeaderProvider.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/HeaderProvider.java new file mode 100644 index 000000000..2c0c080b8 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/HeaderProvider.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.annotation.PublicEvolving; + +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; + +import java.io.Serializable; + +/** Creates an {@link Iterable} of {@link Header}s from the input element. */ +@PublicEvolving +public interface HeaderProvider extends Serializable { + Headers getHeaders(IN input); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java index 59864a373..1cc92201c 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java @@ -84,6 +84,7 @@ public class KafkaRecordSerializationSchemaBuilder { @Nullable private SerializationSchema valueSerializationSchema; @Nullable private FlinkKafkaPartitioner partitioner; @Nullable private SerializationSchema keySerializationSchema; + @Nullable private HeaderProvider headerProvider; /** * Sets a custom partitioner determining the target partition of the target topic. @@ -190,6 +191,20 @@ public KafkaRecordSerializationSchemaBuilder setValueSerializa return self; } + /** + * Sets a {@link HeaderProvider} which is used to add headers to the {@link ProducerRecord} for + * the current element. + * + * @param headerProvider + * @return {@code this} + */ + public KafkaRecordSerializationSchemaBuilder setHeaderProvider( + HeaderProvider headerProvider) { + KafkaRecordSerializationSchemaBuilder self = self(); + self.headerProvider = checkNotNull(headerProvider); + return self; + } + @SuppressWarnings("unchecked") private KafkaRecordSerializationSchemaBuilder self() { return (KafkaRecordSerializationSchemaBuilder) this; @@ -239,7 +254,11 @@ public KafkaRecordSerializationSchema build() { checkState(valueSerializationSchema != null, "No value serializer is configured."); checkState(topicSelector != null, "No topic selector is configured."); return new KafkaRecordSerializationSchemaWrapper<>( - topicSelector, valueSerializationSchema, keySerializationSchema, partitioner); + topicSelector, + valueSerializationSchema, + keySerializationSchema, + partitioner, + headerProvider); } private void checkValueSerializerNotSet() { @@ -278,16 +297,19 @@ private static class KafkaRecordSerializationSchemaWrapper private final Function topicSelector; private final FlinkKafkaPartitioner partitioner; private final SerializationSchema keySerializationSchema; + private final HeaderProvider headerProvider; KafkaRecordSerializationSchemaWrapper( Function topicSelector, SerializationSchema valueSerializationSchema, @Nullable SerializationSchema keySerializationSchema, - @Nullable FlinkKafkaPartitioner partitioner) { + @Nullable FlinkKafkaPartitioner partitioner, + @Nullable HeaderProvider headerProvider) { this.topicSelector = checkNotNull(topicSelector); this.valueSerializationSchema = checkNotNull(valueSerializationSchema); this.partitioner = partitioner; this.keySerializationSchema = keySerializationSchema; + this.headerProvider = headerProvider; } @Override @@ -325,12 +347,22 @@ public ProducerRecord serialize( context.getPartitionsForTopic(targetTopic))) : OptionalInt.empty(); - return new ProducerRecord<>( - targetTopic, - partition.isPresent() ? partition.getAsInt() : null, - timestamp == null || timestamp < 0L ? null : timestamp, - key, - value); + if (headerProvider != null) { + return new ProducerRecord<>( + targetTopic, + partition.isPresent() ? partition.getAsInt() : null, + timestamp == null || timestamp < 0L ? null : timestamp, + key, + value, + headerProvider.getHeaders(element)); + } else { + return new ProducerRecord<>( + targetTopic, + partition.isPresent() ? partition.getAsInt() : null, + timestamp == null || timestamp < 0L ? null : timestamp, + key, + value); + } } } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java index 614624ea3..6dd5baed3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java @@ -28,12 +28,16 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; import org.junit.Before; import org.junit.Test; +import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -145,6 +149,26 @@ public void testSerializeRecordWithPartitioner() throws Exception { assertThat(opened.get()).isTrue(); } + @Test + public void testSerializeRecordWithHeaderProvider() throws Exception { + final HeaderProvider headerProvider = + (ignored) -> + new RecordHeaders(ImmutableList.of(new RecordHeader("a", "a".getBytes()))); + + final KafkaRecordSerializationSchema schema = + KafkaRecordSerializationSchema.builder() + .setTopic(DEFAULT_TOPIC) + .setValueSerializationSchema(new SimpleStringSchema()) + .setHeaderProvider(headerProvider) + .build(); + final ProducerRecord record = schema.serialize("a", null, null); + assertThat(record).isNotNull(); + assertThat(record.headers()) + .singleElement() + .extracting(Header::key, Header::value) + .containsExactly("a", "a".getBytes(StandardCharsets.UTF_8)); + } + @Test public void testSerializeRecordWithKey() { final SerializationSchema serializationSchema = new SimpleStringSchema(); From c9f8aa960aae4aa157152a9f69a828f7ac7690dd Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 12 Apr 2023 07:51:21 -0700 Subject: [PATCH 213/322] [hotfix] [flink-connector-kafka] Simplify ProducerRecord instantiation w.r.t. headers --- ...KafkaRecordSerializationSchemaBuilder.java | 23 ++++++------------- 1 file changed, 7 insertions(+), 16 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java index 1cc92201c..34cf6ef00 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java @@ -347,22 +347,13 @@ public ProducerRecord serialize( context.getPartitionsForTopic(targetTopic))) : OptionalInt.empty(); - if (headerProvider != null) { - return new ProducerRecord<>( - targetTopic, - partition.isPresent() ? partition.getAsInt() : null, - timestamp == null || timestamp < 0L ? null : timestamp, - key, - value, - headerProvider.getHeaders(element)); - } else { - return new ProducerRecord<>( - targetTopic, - partition.isPresent() ? partition.getAsInt() : null, - timestamp == null || timestamp < 0L ? null : timestamp, - key, - value); - } + return new ProducerRecord<>( + targetTopic, + partition.isPresent() ? partition.getAsInt() : null, + timestamp == null || timestamp < 0L ? null : timestamp, + key, + value, + headerProvider != null ? headerProvider.getHeaders(element) : null); } } } From fc526a8136f58a5a3c71f2c2605ba0e373114303 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 12 Apr 2023 09:24:57 -0700 Subject: [PATCH 214/322] Update version to 3.1-SNAPSHOT --- flink-confluent-schema-registry-e2e-tests/pom.xml | 2 +- .../flink-end-to-end-tests-common-kafka/pom.xml | 2 +- .../flink-streaming-kafka-test-base/pom.xml | 2 +- .../flink-streaming-kafka-test/pom.xml | 2 +- flink-connector-kafka-e2e-tests/pom.xml | 2 +- flink-connector-kafka/pom.xml | 2 +- flink-formats-kafka/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml | 2 +- flink-formats-kafka/pom.xml | 2 +- flink-sql-connector-kafka/pom.xml | 2 +- pom.xml | 2 +- 11 files changed, 11 insertions(+), 11 deletions(-) diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml index 75ff2cc0a..5305a7de3 100644 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ b/flink-confluent-schema-registry-e2e-tests/pom.xml @@ -22,7 +22,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.1-SNAPSHOT + 3.1-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index bd7c67a13..9c69af80a 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 4.1-SNAPSHOT + 3.1-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml index ef3d0b135..fa4e22d56 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 4.1-SNAPSHOT + 3.1-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml index b7080ea4f..2b5dc8cd3 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 4.1-SNAPSHOT + 3.1-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/pom.xml b/flink-connector-kafka-e2e-tests/pom.xml index d74e1a29e..c9208ed93 100644 --- a/flink-connector-kafka-e2e-tests/pom.xml +++ b/flink-connector-kafka-e2e-tests/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.1-SNAPSHOT + 3.1-SNAPSHOT pom diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index f754ae26c..0dd3e40a1 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.1-SNAPSHOT + 3.1-SNAPSHOT flink-connector-kafka diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml index a74fe8f6d..4cfc94b4d 100644 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-formats-kafka - 4.1-SNAPSHOT + 3.1-SNAPSHOT 4.0.0 diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml index 9ed0c2119..464affef8 100644 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats-kafka - 4.1-SNAPSHOT + 3.1-SNAPSHOT flink-sql-avro-confluent-registry diff --git a/flink-formats-kafka/pom.xml b/flink-formats-kafka/pom.xml index 8ab527efe..73fb90939 100644 --- a/flink-formats-kafka/pom.xml +++ b/flink-formats-kafka/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.1-SNAPSHOT + 3.1-SNAPSHOT pom diff --git a/flink-sql-connector-kafka/pom.xml b/flink-sql-connector-kafka/pom.xml index 0750e8bff..ca877f3eb 100644 --- a/flink-sql-connector-kafka/pom.xml +++ b/flink-sql-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.1-SNAPSHOT + 3.1-SNAPSHOT flink-sql-connector-kafka diff --git a/pom.xml b/pom.xml index 68a20c913..da9142c89 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.1-SNAPSHOT + 3.1-SNAPSHOT Flink : Connectors : Kafka : Parent pom 2022 From 224468804f4cfb4a293102b8b596a299463dc077 Mon Sep 17 00:00:00 2001 From: Weijie Guo Date: Thu, 25 May 2023 16:36:28 +0800 Subject: [PATCH 215/322] [FLINK-32172][kafka] KafkaExampleUtils incorrect check of the minimum number of parameters (#30) --- .../flink/streaming/kafka/test/base/KafkaExampleUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java index 2d05380b8..a6fa83e61 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java @@ -27,7 +27,7 @@ public class KafkaExampleUtil { public static StreamExecutionEnvironment prepareExecutionEnv(ParameterTool parameterTool) throws Exception { - if (parameterTool.getNumberOfParameters() < 5) { + if (parameterTool.getNumberOfParameters() < 4) { System.out.println( "Missing parameters!\n" + "Usage: Kafka --input-topic --output-topic " From 27e2bd30921a5e5c65bb25e4a5575a3945f5bd49 Mon Sep 17 00:00:00 2001 From: yuxiqian <34335406+yuxiqian@users.noreply.github.com> Date: Tue, 13 Jun 2023 22:06:27 +0800 Subject: [PATCH 216/322] [FLINK-32289][docs] Fix incorrect metadata column type in examples This closes #32. --- docs/content.zh/docs/connectors/table/kafka.md | 14 +++++++------- .../docs/connectors/table/upsert-kafka.md | 2 +- docs/content/docs/connectors/table/kafka.md | 14 +++++++------- docs/content/docs/connectors/table/upsert-kafka.md | 2 +- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/kafka.md b/docs/content.zh/docs/connectors/table/kafka.md index 408cb1a2f..f9ea66f86 100644 --- a/docs/content.zh/docs/connectors/table/kafka.md +++ b/docs/content.zh/docs/connectors/table/kafka.md @@ -48,7 +48,7 @@ CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', 'topic' = 'user_behavior', @@ -127,7 +127,7 @@ CREATE TABLE KafkaTable ( ```sql CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP(3) METADATA FROM 'timestamp', + `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `partition` BIGINT METADATA VIRTUAL, `offset` BIGINT METADATA VIRTUAL, `user_id` BIGINT, @@ -151,7 +151,7 @@ CREATE TABLE KafkaTable ( ```sql CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP(3) METADATA FROM 'value.source.timestamp' VIRTUAL, -- from Debezium format + `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'value.source.timestamp' VIRTUAL, -- from Debezium format `origin_table` STRING METADATA FROM 'value.source.table' VIRTUAL, -- from Debezium format `partition_id` BIGINT METADATA FROM 'partition' VIRTUAL, -- from Kafka connector `offset` BIGINT METADATA VIRTUAL, -- from Kafka connector @@ -389,7 +389,7 @@ Kafka 消息的消息键和消息体部分都可以使用某种 [格式]({{< ref ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING @@ -415,7 +415,7 @@ ROW<`user_id` BIGINT, `item_id` BIGINT, `behavior` STRING> ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING @@ -579,7 +579,7 @@ CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', ... @@ -594,7 +594,7 @@ CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', ... diff --git a/docs/content.zh/docs/connectors/table/upsert-kafka.md b/docs/content.zh/docs/connectors/table/upsert-kafka.md index 40df1fa20..a7e571fd8 100644 --- a/docs/content.zh/docs/connectors/table/upsert-kafka.md +++ b/docs/content.zh/docs/connectors/table/upsert-kafka.md @@ -221,7 +221,7 @@ prefixed with either the `'key'` or `'value'` plus format identifier. ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md index 3c9e739c2..1939c2daf 100644 --- a/docs/content/docs/connectors/table/kafka.md +++ b/docs/content/docs/connectors/table/kafka.md @@ -50,7 +50,7 @@ CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', 'topic' = 'user_behavior', @@ -129,7 +129,7 @@ The extended `CREATE TABLE` example demonstrates the syntax for exposing these m ```sql CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP(3) METADATA FROM 'timestamp', + `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `partition` BIGINT METADATA VIRTUAL, `offset` BIGINT METADATA VIRTUAL, `user_id` BIGINT, @@ -154,7 +154,7 @@ The following example shows how to access both Kafka and Debezium metadata field ```sql CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP(3) METADATA FROM 'value.source.timestamp' VIRTUAL, -- from Debezium format + `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'value.source.timestamp' VIRTUAL, -- from Debezium format `origin_table` STRING METADATA FROM 'value.source.table' VIRTUAL, -- from Debezium format `partition_id` BIGINT METADATA FROM 'partition' VIRTUAL, -- from Kafka connector `offset` BIGINT METADATA VIRTUAL, -- from Kafka connector @@ -435,7 +435,7 @@ options are prefixed with the format identifier. ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING @@ -461,7 +461,7 @@ prefixed with either the `'key'` or `'value'` plus format identifier. ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING @@ -635,7 +635,7 @@ CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', ... @@ -650,7 +650,7 @@ CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', ... diff --git a/docs/content/docs/connectors/table/upsert-kafka.md b/docs/content/docs/connectors/table/upsert-kafka.md index 12a23c5c2..26a377db4 100644 --- a/docs/content/docs/connectors/table/upsert-kafka.md +++ b/docs/content/docs/connectors/table/upsert-kafka.md @@ -240,7 +240,7 @@ prefixed with either the `'key'` or `'value'` plus format identifier. {{< tab "SQL" >}} ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, From 910fc5a3f06ab7d95f79c0fc4b5ce64e2aaa8db2 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 1 Jun 2023 13:15:41 -0700 Subject: [PATCH 217/322] [FLINK-31747] Remove Confluent Avro formats from externalized Kafka connector repo The formats will remain in the core apache/flink repo for now, as they can be commonly used by various different connectors and not just the Kafka connector. This closes #31. --- .idea/vcs.xml | 1 + .../table/formats/avro-confluent.md | 294 -------- .../docs/connectors/table/formats/debezium.md | 455 ------------- .../table/formats/avro-confluent.md | 301 --------- .../docs/connectors/table/formats/debezium.md | 454 ------------- .../pom.xml | 135 ---- .../test/TestAvroConsumerConfluent.java | 112 --- .../src/main/resources/avro/user.avsc | 27 - .../pom.xml | 4 +- flink-connector-kafka/pom.xml | 2 +- .../table/KafkaDynamicTableFactoryTest.java | 2 +- .../1483f895-db24-4888-a2fa-991d602eaafc | 0 .../bfcdf96e-7dcd-4715-9f39-73483823ec16 | 0 .../archunit-violations/stored.rules | 4 - .../flink-avro-confluent-registry/pom.xml | 204 ------ .../confluent/AvroConfluentFormatOptions.java | 131 ---- .../confluent/CachedSchemaCoderProvider.java | 84 --- ...uentRegistryAvroDeserializationSchema.java | 205 ------ ...fluentRegistryAvroSerializationSchema.java | 143 ---- .../ConfluentSchemaRegistryCoder.java | 93 --- .../confluent/RegistryAvroFormatFactory.java | 264 -------- .../DebeziumAvroDeserializationSchema.java | 207 ------ .../debezium/DebeziumAvroFormatFactory.java | 198 ------ .../DebeziumAvroSerializationSchema.java | 151 ----- .../org.apache.flink.table.factories.Factory | 17 - .../TestCodeArchitectureTest.java | 40 -- .../CachedSchemaCoderProviderTest.java | 143 ---- .../ConfluentSchemaRegistryCoderTest.java | 77 --- .../RegistryAvroFormatFactoryTest.java | 294 -------- .../RegistryAvroRowDataSeDeSchemaTest.java | 194 ------ .../DebeziumAvroFormatFactoryTest.java | 249 ------- .../debezium/DebeziumAvroSerDeSchemaTest.java | 242 ------- .../org.junit.jupiter.api.extension.Extension | 16 - .../src/test/resources/archunit.properties | 31 - .../test/resources/debezium-avro-delete.avro | Bin 147 -> 0 bytes .../test/resources/debezium-avro-insert.avro | Bin 143 -> 0 bytes .../test/resources/debezium-avro-update.avro | Bin 188 -> 0 bytes .../test/resources/debezium-test-schema.json | 191 ------ .../src/test/resources/test-keystore.jks | Bin 2327 -> 0 bytes .../flink-sql-avro-confluent-registry/pom.xml | 120 ---- .../src/main/resources/META-INF/NOTICE | 32 - .../licenses/LICENSE.jakarta-annotation-api | 637 ------------------ .../META-INF/licenses/LICENSE.jakarta-inject | 637 ------------------ .../licenses/LICENSE.jakarta-ws-rs-api | 637 ------------------ .../licenses/LICENSE.osgi-resource-locator | 637 ------------------ .../META-INF/licenses/LICENSE.zstd-jni | 26 - flink-formats-kafka/pom.xml | 41 -- pom.xml | 2 - 48 files changed, 5 insertions(+), 7729 deletions(-) delete mode 100644 docs/content.zh/docs/connectors/table/formats/avro-confluent.md delete mode 100644 docs/content.zh/docs/connectors/table/formats/debezium.md delete mode 100644 docs/content/docs/connectors/table/formats/avro-confluent.md delete mode 100644 docs/content/docs/connectors/table/formats/debezium.md delete mode 100644 flink-confluent-schema-registry-e2e-tests/pom.xml delete mode 100644 flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java delete mode 100644 flink-confluent-schema-registry-e2e-tests/src/main/resources/avro/user.avsc delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/1483f895-db24-4888-a2fa-991d602eaafc delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/bfcdf96e-7dcd-4715-9f39-73483823ec16 delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/stored.rules delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/pom.xml delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/archunit.properties delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-delete.avro delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-insert.avro delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-update.avro delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-test-schema.json delete mode 100644 flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/test-keystore.jks delete mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml delete mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE delete mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-annotation-api delete mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-inject delete mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-ws-rs-api delete mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.osgi-resource-locator delete mode 100644 flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.zstd-jni delete mode 100644 flink-formats-kafka/pom.xml diff --git a/.idea/vcs.xml b/.idea/vcs.xml index a5e4331ed..264965d21 100644 --- a/.idea/vcs.xml +++ b/.idea/vcs.xml @@ -20,5 +20,6 @@ + \ No newline at end of file diff --git a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md b/docs/content.zh/docs/connectors/table/formats/avro-confluent.md deleted file mode 100644 index fdc79540b..000000000 --- a/docs/content.zh/docs/connectors/table/formats/avro-confluent.md +++ /dev/null @@ -1,294 +0,0 @@ ---- -title: Confluent Avro -weight: 4 -type: docs -aliases: - - /zh/dev/table/connectors/formats/avro-confluent.html ---- - - -# Confluent Avro Format - -Format: Serialization Schema -Format: Deserialization Schema - - - -Avro Schema Registry (``avro-confluent``) 格式能让你读取被 ``io.confluent.kafka.serializers.KafkaAvroSerializer`` 序列化的记录,以及可以写入成能被 ``io.confluent.kafka.serializers.KafkaAvroDeserializer`` 反序列化的记录。 - -当以这种格式读取(反序列化)记录时,将根据记录中编码的 schema 版本 id 从配置的 Confluent Schema Registry 中获取 Avro writer schema ,而从 table schema 中推断出 reader schema。 - -当以这种格式写入(序列化)记录时,Avro schema 是从 table schema 中推断出来的,并会用来检索要与数据一起编码的 schema id。我们会在配置的 Confluent Schema Registry 中配置的 [subject](https://docs.confluent.io/current/schema-registry/index.html#schemas-subjects-and-topics) 下,检索 schema id。subject 通过 `avro-confluent.subject` 参数来制定。 - -Avro Schema Registry 格式只能与 [Apache Kafka SQL 连接器]({{< ref "docs/connectors/table/kafka" >}})或 [Upsert Kafka SQL 连接器]({{< ref "docs/connectors/table/upsert-kafka" >}})一起使用。 - -依赖 ------------- - -{{< sql_download_table "avro-confluent" >}} - -For Maven, SBT, Gradle, or other build automation tools, please also ensure that Confluent's maven repository at `https://packages.confluent.io/maven/` is configured in your project's build files. - -如何创建使用 Avro-Confluent 格式的表 ----------------- - -以下是一个使用 Kafka 连接器和 Confluent Avro 格式创建表的示例。 - -{{< tabs "3df131fd-0e20-4635-a8f9-3574a764db7a" >}} -{{< tab "SQL" >}} - -使用原始的 UTF-8 字符串作为 Kafka 的 key,Schema Registry 中注册的 Avro 记录作为 Kafka 的 values 的表的示例: - -```sql -CREATE TABLE user_created ( - - -- 该列映射到 Kafka 原始的 UTF-8 key - the_kafka_key STRING, - - -- 映射到 Kafka value 中的 Avro 字段的一些列 - id STRING, - name STRING, - email STRING - -) WITH ( - - 'connector' = 'kafka', - 'topic' = 'user_events_example1', - 'properties.bootstrap.servers' = 'localhost:9092', - - -- UTF-8 字符串作为 Kafka 的 keys,使用表中的 'the_kafka_key' 列 - 'key.format' = 'raw', - 'key.fields' = 'the_kafka_key', - - 'value.format' = 'avro-confluent', - 'value.avro-confluent.url' = 'http://localhost:8082', - 'value.fields-include' = 'EXCEPT_KEY' -) -``` - -我们可以像下面这样将数据写入到 kafka 表中: - -```sql -INSERT INTO user_created -SELECT - -- 将 user id 复制至映射到 kafka key 的列中 - id as the_kafka_key, - - -- 所有的 values - id, name, email -FROM some_table -``` - ---- - -Kafka 的 key 和 value 在 Schema Registry 中都注册为 Avro 记录的表的示例: - -```sql -CREATE TABLE user_created ( - - -- 该列映射到 Kafka key 中的 Avro 字段 'id' - kafka_key_id STRING, - - -- 映射到 Kafka value 中的 Avro 字段的一些列 - id STRING, - name STRING, - email STRING - -) WITH ( - - 'connector' = 'kafka', - 'topic' = 'user_events_example2', - 'properties.bootstrap.servers' = 'localhost:9092', - - -- 注意:由于哈希分区,在 Kafka key 的上下文中,schema 升级几乎从不向后也不向前兼容。 - 'key.format' = 'avro-confluent', - 'key.avro-confluent.url' = 'http://localhost:8082', - 'key.fields' = 'kafka_key_id', - - -- 在本例中,我们希望 Kafka 的 key 和 value 的 Avro 类型都包含 'id' 字段 - -- => 给表中与 Kafka key 字段关联的列添加一个前缀来避免冲突 - 'key.fields-prefix' = 'kafka_key_', - - 'value.format' = 'avro-confluent', - 'value.avro-confluent.url' = 'http://localhost:8082', - 'value.fields-include' = 'EXCEPT_KEY', - - -- 自 Flink 1.13 起,subjects 具有一个默认值, 但是可以被覆盖: - 'key.avro-confluent.subject' = 'user_events_example2-key2', - 'value.avro-confluent.subject' = 'user_events_example2-value2' -) -``` - ---- -使用 upsert-kafka 连接器,Kafka 的 value 在 Schema Registry 中注册为 Avro 记录的表的示例: - -```sql -CREATE TABLE user_created ( - - -- 该列映射到 Kafka 原始的 UTF-8 key - kafka_key_id STRING, - - -- 映射到 Kafka value 中的 Avro 字段的一些列 - id STRING, - name STRING, - email STRING, - - -- upsert-kafka 连接器需要一个主键来定义 upsert 行为 - PRIMARY KEY (kafka_key_id) NOT ENFORCED - -) WITH ( - - 'connector' = 'upsert-kafka', - 'topic' = 'user_events_example3', - 'properties.bootstrap.servers' = 'localhost:9092', - - -- UTF-8 字符串作为 Kafka 的 keys - -- 在本例中我们不指定 'key.fields',因为它由表的主键决定 - 'key.format' = 'raw', - - -- 在本例中,我们希望 Kafka 的 key 和 value 的 Avro 类型都包含 'id' 字段 - -- => 给表中与 Kafka key 字段关联的列添加一个前缀来避免冲突 - 'key.fields-prefix' = 'kafka_key_', - - 'value.format' = 'avro-confluent', - 'value.avro-confluent.url' = 'http://localhost:8082', - 'value.fields-include' = 'EXCEPT_KEY' -) -``` -{{< /tab >}} -{{< /tabs >}} - -Format 参数 ----------------- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
参数是否必选默认值类型描述
format
required(none)StringSpecify what format to use, here should be 'avro-confluent'.
avro-confluent.basic-auth.credentials-source
optional(none)StringBasic auth credentials source for Schema Registry
avro-confluent.basic-auth.user-info
optional(none)StringBasic auth user info for schema registry
avro-confluent.bearer-auth.credentials-source
optional(none)StringBearer auth credentials source for Schema Registry
avro-confluent.bearer-auth.token
optional(none)StringBearer auth token for Schema Registry
avro-confluent.properties
optional(none)MapProperties map that is forwarded to the underlying Schema Registry. This is useful for options that are not officially exposed via Flink config options. However, note that Flink options have higher precedence.
avro-confluent.ssl.keystore.location
optional(none)StringLocation / File of SSL keystore
avro-confluent.ssl.keystore.password
optional(none)StringPassword for SSL keystore
avro-confluent.ssl.truststore.location
optional(none)StringLocation / File of SSL truststore
avro-confluent.ssl.truststore.password
optional(none)StringPassword for SSL truststore
avro-confluent.schema
optional(none)StringThe schema registered or to be registered in the Confluent Schema Registry. If no schema is provided Flink converts the table schema to avro schema. The schema provided must match the table schema.
avro-confluent.subject
optional(none)StringThe Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, 'kafka' and 'upsert-kafka' connectors use '<topic_name>-value' or '<topic_name>-key' as the default subject name if this format is used as the value or key format. But for other connectors (e.g. 'filesystem'), the subject option is required when used as sink.
avro-confluent.url
required(none)StringThe URL of the Confluent Schema Registry to fetch/register schemas.
- -数据类型映射 ----------------- - -目前 Apache Flink 都是从 table schema 去推断反序列化期间的 Avro reader schema 和序列化期间的 Avro writer schema。显式地定义 Avro schema 暂不支持。 -[Apache Avro Format]({{< ref "docs/connectors/table/formats/avro" >}}#data-type-mapping)中描述了 Flink 数据类型和 Avro 类型的对应关系。 - -除了此处列出的类型之外,Flink 还支持读取/写入可为空(nullable)的类型。 Flink 将可为空的类型映射到 Avro `union(something, null)`, 其中 `something` 是从 Flink 类型转换的 Avro 类型。 - -您可以参考 [Avro Specification](https://avro.apache.org/docs/current/spec.html) 以获取有关 Avro 类型的更多信息。 diff --git a/docs/content.zh/docs/connectors/table/formats/debezium.md b/docs/content.zh/docs/connectors/table/formats/debezium.md deleted file mode 100644 index a6ac486f0..000000000 --- a/docs/content.zh/docs/connectors/table/formats/debezium.md +++ /dev/null @@ -1,455 +0,0 @@ ---- -title: Debezium -weight: 5 -type: docs -aliases: - - /zh/dev/table/connectors/formats/debezium.html ---- - - -# Debezium Format - -{{< label "Changelog-Data-Capture Format" >}} -{{< label "Format: Serialization Schema" >}} -{{< label "Format: Deserialization Schema" >}} - -[Debezium](https://debezium.io/) 是一个 CDC(Changelog Data Capture,变更数据捕获)的工具,可以把来自 MySQL、PostgreSQL、Oracle、Microsoft SQL Server 和许多其他数据库的更改实时流式传输到 Kafka 中。 Debezium 为变更日志提供了统一的格式结构,并支持使用 JSON 和 Apache Avro 序列化消息。 - -Flink 支持将 Debezium JSON 和 Avro 消息解析为 INSERT / UPDATE / DELETE 消息到 Flink SQL 系统中。在很多情况下,利用这个特性非常的有用,例如 - - 将增量数据从数据库同步到其他系统 - - 日志审计 - - 数据库的实时物化视图 - - 关联维度数据库的变更历史,等等。 - -Flink 还支持将 Flink SQL 中的 INSERT / UPDATE / DELETE 消息编码为 Debezium 格式的 JSON 或 Avro 消息,输出到 Kafka 等存储中。 -但需要注意的是,目前 Flink 还不支持将 UPDATE_BEFORE 和 UPDATE_AFTER 合并为一条 UPDATE 消息。因此,Flink 将 UPDATE_BEFORE 和 UPDATE_AFTER 分别编码为 DELETE 和 INSERT 类型的 Debezium 消息。 - -依赖 ------------- - -#### Debezium Avro - -{{< sql_download_table "debezium-avro-confluent" >}} - -#### Debezium Json - -{{< sql_download_table "debezium-json" >}} - -*注意: 请参考 [Debezium 文档](https://debezium.io/documentation/reference/1.3/index.html),了解如何设置 Debezium Kafka Connect 用来将变更日志同步到 Kafka 主题。* - - -如何使用 Debezium Format ----------------- - - -Debezium 为变更日志提供了统一的格式,这是一个 JSON 格式的从 MySQL product 表捕获的更新操作的简单示例: - -```json -{ - "before": { - "id": 111, - "name": "scooter", - "description": "Big 2-wheel scooter", - "weight": 5.18 - }, - "after": { - "id": 111, - "name": "scooter", - "description": "Big 2-wheel scooter", - "weight": 5.15 - }, - "source": {...}, - "op": "u", - "ts_ms": 1589362330904, - "transaction": null -} -``` - -*注意: 请参考 [Debezium 文档](https://debezium.io/documentation/reference/1.3/connectors/mysql.html#mysql-connector-events_debezium),了解每个字段的含义。* - -MySQL 产品表有4列(`id`、`name`、`description`、`weight`)。上面的 JSON 消息是 `products` 表上的一条更新事件,其中 `id = 111` 的行的 `weight` 值从 `5.18` 更改为 `5.15`。假设此消息已同步到 Kafka 主题 `products_binlog`,则可以使用以下 DDL 来使用此主题并解析更改事件。 - -{{< tabs "0b6703c1-021e-4506-a579-b72b8408c0cf" >}} -{{< tab "SQL" >}} -```sql -CREATE TABLE topic_products ( - -- schema 与 MySQL 的 products 表完全相同 - id BIGINT, - name STRING, - description STRING, - weight DECIMAL(10, 2) -) WITH ( - 'connector' = 'kafka', - 'topic' = 'products_binlog', - 'properties.bootstrap.servers' = 'localhost:9092', - 'properties.group.id' = 'testGroup', - -- 使用 'debezium-json' format 来解析 Debezium 的 JSON 消息 - -- 如果 Debezium 用 Avro 编码消息,请使用 'debezium-avro-confluent' - 'format' = 'debezium-json' -- 如果 Debezium 用 Avro 编码消息,请使用 'debezium-avro-confluent' -) -``` -{{< /tab >}} -{{< /tabs >}} - -在某些情况下,用户在设置 Debezium Kafka Connect 时,可能会开启 Kafka 的配置 `'value.converter.schemas.enable'`,用来在消息体中包含 schema 信息。然后,Debezium JSON 消息可能如下所示: - -```json -{ - "schema": {...}, - "payload": { - "before": { - "id": 111, - "name": "scooter", - "description": "Big 2-wheel scooter", - "weight": 5.18 - }, - "after": { - "id": 111, - "name": "scooter", - "description": "Big 2-wheel scooter", - "weight": 5.15 - }, - "source": {...}, - "op": "u", - "ts_ms": 1589362330904, - "transaction": null - } -} -``` - -为了解析这一类信息,你需要在上述 DDL WITH 子句中添加选项 `'debezium-json.schema-include' = 'true'`(默认为 false)。通常情况下,建议不要包含 schema 的描述,因为这样会使消息变得非常冗长,并降低解析性能。 - -在将主题注册为 Flink 表之后,可以将 Debezium 消息用作变更日志源。 - -{{< tabs "6a84a0e8-2e56-49db-9089-e836290f8239" >}} -{{< tab "SQL" >}} -```sql --- MySQL "products" 的实时物化视图 --- 计算相同产品的最新平均重量 -SELECT name, AVG(weight) FROM topic_products GROUP BY name; - --- 将 MySQL "products" 表的所有数据和增量更改同步到 --- Elasticsearch "products" 索引,供将来查找 -INSERT INTO elasticsearch_products -SELECT * FROM topic_products; -``` -{{< /tab >}} -{{< /tabs >}} - -Available Metadata ------------------- - -The following format metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition. - -Attention Format metadata fields are only available if the -corresponding connector forwards format metadata. Currently, only the Kafka connector is able to expose -metadata fields for its value format. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
KeyData TypeDescription
schemaSTRING NULLJSON string describing the schema of the payload. Null if the schema is not included in - the Debezium record.
ingestion-timestampTIMESTAMP_LTZ(3) NULLThe timestamp at which the connector processed the event. Corresponds to the ts_ms - field in the Debezium record.
source.timestampTIMESTAMP_LTZ(3) NULLThe timestamp at which the source system created the event. Corresponds to the source.ts_ms - field in the Debezium record.
source.databaseSTRING NULLThe originating database. Corresponds to the source.db field in the - Debezium record if available.
source.schemaSTRING NULLThe originating database schema. Corresponds to the source.schema field in the - Debezium record if available.
source.tableSTRING NULLThe originating database table. Corresponds to the source.table or source.collection - field in the Debezium record if available.
source.propertiesMAP<STRING, STRING> NULLMap of various source properties. Corresponds to the source field in the Debezium record.
- -The following example shows how to access Debezium metadata fields in Kafka: - -```sql -CREATE TABLE KafkaTable ( - origin_ts TIMESTAMP(3) METADATA FROM 'value.ingestion-timestamp' VIRTUAL, - event_time TIMESTAMP(3) METADATA FROM 'value.source.timestamp' VIRTUAL, - origin_database STRING METADATA FROM 'value.source.database' VIRTUAL, - origin_schema STRING METADATA FROM 'value.source.schema' VIRTUAL, - origin_table STRING METADATA FROM 'value.source.table' VIRTUAL, - origin_properties MAP METADATA FROM 'value.source.properties' VIRTUAL, - user_id BIGINT, - item_id BIGINT, - behavior STRING -) WITH ( - 'connector' = 'kafka', - 'topic' = 'user_behavior', - 'properties.bootstrap.servers' = 'localhost:9092', - 'properties.group.id' = 'testGroup', - 'scan.startup.mode' = 'earliest-offset', - 'value.format' = 'debezium-json' -); -``` - -Format 参数 ----------------- - -Flink 提供了 `debezium-avro-confluent` 和 `debezium-json` 两种 format 来解析 Debezium 生成的 JSON 格式和 Avro 格式的消息。 -请使用 `debezium-avro-confluent` 来解析 Debezium 的 Avro 消息,使用 `debezium-json` 来解析 Debezium 的 JSON 消息。 - -#### Debezium Avro - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
参数是否必选默认值类型描述
format
required(none)StringSpecify what format to use, here should be 'debezium-avro-confluent'.
debezium-avro-confluent.basic-auth.credentials-source
optional(none)StringBasic auth credentials source for Schema Registry
debezium-avro-confluent.basic-auth.user-info
optional(none)StringBasic auth user info for schema registry
debezium-avro-confluent.bearer-auth.credentials-source
optional(none)StringBearer auth credentials source for Schema Registry
debezium-avro-confluent.bearer-auth.token
optional(none)StringBearer auth token for Schema Registry
debezium-avro-confluent.properties
optional(none)MapProperties map that is forwarded to the underlying Schema Registry. This is useful for options that are not officially exposed via Flink config options. However, note that Flink options have higher precedence.
debezium-avro-confluent.ssl.keystore.location
optional(none)StringLocation / File of SSL keystore
debezium-avro-confluent.ssl.keystore.password
optional(none)StringPassword for SSL keystore
debezium-avro-confluent.ssl.truststore.location
optional(none)StringLocation / File of SSL truststore
debezium-avro-confluent.ssl.truststore.password
optional(none)StringPassword for SSL truststore
debezium-avro-confluent.schema
optional(none)StringThe schema registered or to be registered in the Confluent Schema Registry. If no schema is provided Flink converts the table schema to avro schema. The schema provided must match the Debezium schema which is a nullable record type including fields 'before', 'after', 'op'.
debezium-avro-confluent.subject
optional(none)StringThe Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, 'kafka' and 'upsert-kafka' connectors use '<topic_name>-value' or '<topic_name>-key' as the default subject name if this format is used as the value or key format. But for other connectors (e.g. 'filesystem'), the subject option is required when used as sink.
debezium-avro-confluent.url
required(none)StringThe URL of the Confluent Schema Registry to fetch/register schemas.
- -#### Debezium Json - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
参数是否必选默认值类型描述
format
必选(none)String指定要使用的格式,此处应为 'debezium-json'
debezium-json.schema-include
可选falseBoolean设置 Debezium Kafka Connect 时,用户可以启用 Kafka 配置 'value.converter.schemas.enable' 以在消息中包含 schema。此选项表明 Debezium JSON 消息是否包含 schema。
debezium-json.ignore-parse-errors
可选falseBoolean当解析异常时,是跳过当前字段或行,还是抛出错误失败(默认为 false,即抛出错误失败)。如果忽略字段的解析异常,则会将该字段值设置为null
debezium-json.timestamp-format.standard
可选'SQL'String声明输入和输出的时间戳格式。当前支持的格式为'SQL' 以及 'ISO-8601': -
    -
  • 可选参数 'SQL' 将会以 "yyyy-MM-dd HH:mm:ss.s{precision}" 的格式解析时间戳, 例如 '2020-12-30 12:13:14.123',且会以相同的格式输出。
  • -
  • 可选参数 'ISO-8601' 将会以 "yyyy-MM-ddTHH:mm:ss.s{precision}" 的格式解析输入时间戳, 例如 '2020-12-30T12:13:14.123' ,且会以相同的格式输出。
  • -
-
debezium-json.map-null-key.mode
选填'FAIL'String指定处理 Map 中 key 值为空的方法. 当前支持的值有 'FAIL', 'DROP''LITERAL': -
    -
  • Option 'FAIL' 将抛出异常,如果遇到 Map 中 key 值为空的数据。
  • -
  • Option 'DROP' 将丢弃 Map 中 key 值为空的数据项。
  • -
  • Option 'LITERAL' 将使用字符串常量来替换 Map 中的空 key 值。字符串常量的值由 'debezium-json.map-null-key.literal' 定义。
  • -
-
debezium-json.map-null-key.literal
选填'null'String'debezium-json.map-null-key.mode' 是 LITERAL 的时候,指定字符串常量替换 Map 中的空 key 值。
debezium-json.encode.decimal-as-plain-number
选填falseBoolean将所有 DECIMAL 类型的数据保持原状,不使用科学计数法表示。例:0.000000027 默认会表示为 2.7E-8。当此选项设为 true 时,则会表示为 0.000000027
- - -注意事项 ----------------- - -### 重复的变更事件 - -在正常的操作环境下,Debezium 应用能以 **exactly-once** 的语义投递每条变更事件。在这种情况下,Flink 消费 Debezium 产生的变更事件能够工作得很好。 -然而,当有故障发生时,Debezium 应用只能保证 **at-least-once** 的投递语义。可以查看 [Debezium 官方文档](https://debezium.io/documentation/faq/#what_happens_when_an_application_stops_or_crashes) 了解更多关于 Debezium 的消息投递语义。 -这也意味着,在非正常情况下,Debezium 可能会投递重复的变更事件到 Kafka 中,当 Flink 从 Kafka 中消费的时候就会得到重复的事件。 -这可能会导致 Flink query 的运行得到错误的结果或者非预期的异常。因此,建议在这种情况下,将作业参数 [`table.exec.source.cdc-events-duplicate`]({{< ref "docs/dev/table/config" >}}#table-exec-source-cdc-events-duplicate) 设置成 `true`,并在该 source 上定义 PRIMARY KEY。 -框架会生成一个额外的有状态算子,使用该 primary key 来对变更事件去重并生成一个规范化的 changelog 流。 - -### 消费 Debezium Postgres Connector 产生的数据 - -如果你正在使用 [Debezium PostgreSQL Connector](https://debezium.io/documentation/reference/1.2/connectors/postgresql.html) 捕获变更到 Kafka,请确保被监控表的 [REPLICA IDENTITY](https://www.postgresql.org/docs/current/sql-altertable.html#SQL-CREATETABLE-REPLICA-IDENTITY) 已经被配置成 `FULL` 了,默认值是 `DEFAULT`。 -否则,Flink SQL 将无法正确解析 Debezium 数据。 - -当配置为 `FULL` 时,更新和删除事件将完整包含所有列的之前的值。当为其他配置时,更新和删除事件的 "before" 字段将只包含 primary key 字段的值,或者为 null(没有 primary key)。 -你可以通过运行 `ALTER TABLE REPLICA IDENTITY FULL` 来更改 `REPLICA IDENTITY` 的配置。 -请阅读 [Debezium 关于 PostgreSQL REPLICA IDENTITY 的文档](https://debezium.io/documentation/reference/1.2/connectors/postgresql.html#postgresql-replica-identity) 了解更多。 - -数据类型映射 ----------------- - -目前,Debezium Format 使用 JSON Format 进行序列化和反序列化。有关数据类型映射的更多详细信息,请参考 [JSON Format 文档]({{< ref "docs/connectors/table/formats/json" >}}#data-type-mapping) 和 [Confluent Avro Format 文档]({{< ref "docs/connectors/table/formats/avro-confluent" >}}#data-type-mapping)。 - diff --git a/docs/content/docs/connectors/table/formats/avro-confluent.md b/docs/content/docs/connectors/table/formats/avro-confluent.md deleted file mode 100644 index 03a847713..000000000 --- a/docs/content/docs/connectors/table/formats/avro-confluent.md +++ /dev/null @@ -1,301 +0,0 @@ ---- -title: Confluent Avro -weight: 4 -type: docs -aliases: - - /dev/table/connectors/formats/avro-confluent.html ---- - - -# Confluent Avro Format - -{{< label "Format: Serialization Schema" >}} -{{< label "Format: Deserialization Schema" >}} - -The Avro Schema Registry (``avro-confluent``) format allows you to read records that were serialized by the ``io.confluent.kafka.serializers.KafkaAvroSerializer`` and to write records that can in turn be read by the ``io.confluent.kafka.serializers.KafkaAvroDeserializer``. - -When reading (deserializing) a record with this format the Avro writer schema is fetched from the configured Confluent Schema Registry based on the schema version id encoded in the record while the reader schema is inferred from table schema. - -When writing (serializing) a record with this format the Avro schema is inferred from the table schema and used to retrieve a schema id to be encoded with the data. The lookup is performed with in the configured Confluent Schema Registry under the [subject](https://docs.confluent.io/current/schema-registry/index.html#schemas-subjects-and-topics) given in `avro-confluent.subject`. - -The Avro Schema Registry format can only be used in conjunction with the [Apache Kafka SQL connector]({{< ref "docs/connectors/table/kafka" >}}) or the [Upsert Kafka SQL Connector]({{< ref "docs/connectors/table/upsert-kafka" >}}). - -Dependencies ------------- - -{{< sql_download_table "avro-confluent" >}} - -For Maven, SBT, Gradle, or other build automation tools, please also ensure that Confluent's maven repository at `https://packages.confluent.io/maven/` is configured in your project's build files. - -How to create tables with Avro-Confluent format --------------- - -Example of a table using raw UTF-8 string as Kafka key and Avro records registered in the Schema Registry as Kafka values: - -```sql -CREATE TABLE user_created ( - - -- one column mapped to the Kafka raw UTF-8 key - the_kafka_key STRING, - - -- a few columns mapped to the Avro fields of the Kafka value - id STRING, - name STRING, - email STRING - -) WITH ( - - 'connector' = 'kafka', - 'topic' = 'user_events_example1', - 'properties.bootstrap.servers' = 'localhost:9092', - - -- UTF-8 string as Kafka keys, using the 'the_kafka_key' table column - 'key.format' = 'raw', - 'key.fields' = 'the_kafka_key', - - 'value.format' = 'avro-confluent', - 'value.avro-confluent.url' = 'http://localhost:8082', - 'value.fields-include' = 'EXCEPT_KEY' -) -``` - -We can write data into the kafka table as follows: - -```sql -INSERT INTO user_created -SELECT - -- replicating the user id into a column mapped to the kafka key - id as the_kafka_key, - - -- all values - id, name, email -FROM some_table -``` - ---- - -Example of a table with both the Kafka key and value registered as Avro records in the Schema Registry: - -```sql -CREATE TABLE user_created ( - - -- one column mapped to the 'id' Avro field of the Kafka key - kafka_key_id STRING, - - -- a few columns mapped to the Avro fields of the Kafka value - id STRING, - name STRING, - email STRING - -) WITH ( - - 'connector' = 'kafka', - 'topic' = 'user_events_example2', - 'properties.bootstrap.servers' = 'localhost:9092', - - -- Watch out: schema evolution in the context of a Kafka key is almost never backward nor - -- forward compatible due to hash partitioning. - 'key.format' = 'avro-confluent', - 'key.avro-confluent.url' = 'http://localhost:8082', - 'key.fields' = 'kafka_key_id', - - -- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id' - -- => adding a prefix to the table column associated to the Kafka key field avoids clashes - 'key.fields-prefix' = 'kafka_key_', - - 'value.format' = 'avro-confluent', - 'value.avro-confluent.url' = 'http://localhost:8082', - 'value.fields-include' = 'EXCEPT_KEY', - - -- subjects have a default value since Flink 1.13, though can be overridden: - 'key.avro-confluent.subject' = 'user_events_example2-key2', - 'value.avro-confluent.subject' = 'user_events_example2-value2' -) -``` - ---- -Example of a table using the upsert-kafka connector with the Kafka value registered as an Avro record in the Schema Registry: - -```sql -CREATE TABLE user_created ( - - -- one column mapped to the Kafka raw UTF-8 key - kafka_key_id STRING, - - -- a few columns mapped to the Avro fields of the Kafka value - id STRING, - name STRING, - email STRING, - - -- upsert-kafka connector requires a primary key to define the upsert behavior - PRIMARY KEY (kafka_key_id) NOT ENFORCED - -) WITH ( - - 'connector' = 'upsert-kafka', - 'topic' = 'user_events_example3', - 'properties.bootstrap.servers' = 'localhost:9092', - - -- UTF-8 string as Kafka keys - -- We don't specify 'key.fields' in this case since it's dictated by the primary key of the table - 'key.format' = 'raw', - - -- In this example, we want the Avro types of both the Kafka key and value to contain the field 'id' - -- => adding a prefix to the table column associated to the kafka key field avoids clashes - 'key.fields-prefix' = 'kafka_key_', - - 'value.format' = 'avro-confluent', - 'value.avro-confluent.url' = 'http://localhost:8082', - 'value.fields-include' = 'EXCEPT_KEY' -) -``` - - -Format Options ----------------- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
OptionRequiredForwardedDefaultTypeDescription
format
requiredno(none)StringSpecify what format to use, here should be 'avro-confluent'.
avro-confluent.basic-auth.credentials-source
optionalyes(none)StringBasic auth credentials source for Schema Registry
avro-confluent.basic-auth.user-info
optionalyes(none)StringBasic auth user info for schema registry
avro-confluent.bearer-auth.credentials-source
optionalyes(none)StringBearer auth credentials source for Schema Registry
avro-confluent.bearer-auth.token
optionalyes(none)StringBearer auth token for Schema Registry
avro-confluent.properties
optionalyes(none)MapProperties map that is forwarded to the underlying Schema Registry. This is useful for options that are not officially exposed via Flink config options. However, note that Flink options have higher precedence.
avro-confluent.ssl.keystore.location
optionalyes(none)StringLocation / File of SSL keystore
avro-confluent.ssl.keystore.password
optionalyes(none)StringPassword for SSL keystore
avro-confluent.ssl.truststore.location
optionalyes(none)StringLocation / File of SSL truststore
avro-confluent.ssl.truststore.password
optionalyes(none)StringPassword for SSL truststore
avro-confluent.schema
optionalno(none)StringThe schema registered or to be registered in the Confluent Schema Registry. If no schema is provided Flink converts the table schema to avro schema. The schema provided must match the table schema.
avro-confluent.subject
optionalyes(none)StringThe Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, 'kafka' and 'upsert-kafka' connectors use '<topic_name>-value' or '<topic_name>-key' as the default subject name if this format is used as the value or key format. But for other connectors (e.g. 'filesystem'), the subject option is required when used as sink.
avro-confluent.url
requiredyes(none)StringThe URL of the Confluent Schema Registry to fetch/register schemas.
- -Data Type Mapping ----------------- - -Currently, Apache Flink always uses the table schema to derive the Avro reader schema during deserialization and Avro writer schema during serialization. Explicitly defining an Avro schema is not supported yet. -See the [Apache Avro Format]({{< ref "docs/connectors/table/formats/avro" >}}#data-type-mapping) for the mapping between Avro and Flink DataTypes. - -In addition to the types listed there, Flink supports reading/writing nullable types. Flink maps nullable types to Avro `union(something, null)`, where `something` is the Avro type converted from Flink type. - -You can refer to [Avro Specification](https://avro.apache.org/docs/current/spec.html) for more information about Avro types. diff --git a/docs/content/docs/connectors/table/formats/debezium.md b/docs/content/docs/connectors/table/formats/debezium.md deleted file mode 100644 index 571992a21..000000000 --- a/docs/content/docs/connectors/table/formats/debezium.md +++ /dev/null @@ -1,454 +0,0 @@ ---- -title: Debezium -weight: 5 -type: docs -aliases: - - /dev/table/connectors/formats/debezium.html ---- - - -# Debezium Format - -{{< label "Changelog-Data-Capture Format" >}} -{{< label "Format: Serialization Schema" >}} -{{< label "Format: Deserialization Schema" >}} - -[Debezium](https://debezium.io/) is a CDC (Changelog Data Capture) tool that can stream changes in real-time from MySQL, PostgreSQL, Oracle, Microsoft SQL Server and many other databases into Kafka. Debezium provides a unified format schema for changelog and supports to serialize messages using JSON and [Apache Avro](https://avro.apache.org/). - -Flink supports to interpret Debezium JSON and Avro messages as INSERT/UPDATE/DELETE messages into Flink SQL system. This is useful in many cases to leverage this feature, such as - - synchronizing incremental data from databases to other systems - - auditing logs - - real-time materialized views on databases - - temporal join changing history of a database table and so on. - -Flink also supports to encode the INSERT/UPDATE/DELETE messages in Flink SQL as Debezium JSON or Avro messages, and emit to external systems like Kafka. -However, currently Flink can't combine UPDATE_BEFORE and UPDATE_AFTER into a single UPDATE message. Therefore, Flink encodes UPDATE_BEFORE and UDPATE_AFTER as DELETE and INSERT Debezium messages. - -Dependencies ------------- - -#### Debezium Avro - -{{< sql_download_table "debezium-avro-confluent" >}} - -#### Debezium Json - -{{< sql_download_table "debezium-json" >}} - - -*Note: please refer to [Debezium documentation](https://debezium.io/documentation/reference/1.3/index.html) about how to setup a Debezium Kafka Connect to synchronize changelog to Kafka topics.* - - -How to use Debezium format ----------------- - -Debezium provides a unified format for changelog, here is a simple example for an update operation captured from a MySQL `products` table in JSON format: - -```json -{ - "before": { - "id": 111, - "name": "scooter", - "description": "Big 2-wheel scooter", - "weight": 5.18 - }, - "after": { - "id": 111, - "name": "scooter", - "description": "Big 2-wheel scooter", - "weight": 5.15 - }, - "source": {...}, - "op": "u", - "ts_ms": 1589362330904, - "transaction": null -} -``` - -*Note: please refer to [Debezium documentation](https://debezium.io/documentation/reference/1.3/connectors/mysql.html#mysql-connector-events_debezium) about the meaning of each fields.* - -The MySQL `products` table has 4 columns (`id`, `name`, `description` and `weight`). The above JSON message is an update change event on the `products` table where the `weight` value of the row with `id = 111` is changed from `5.18` to `5.15`. -Assuming this messages is synchronized to Kafka topic `products_binlog`, then we can use the following DDL to consume this topic and interpret the change events. - -```sql -CREATE TABLE topic_products ( - -- schema is totally the same to the MySQL "products" table - id BIGINT, - name STRING, - description STRING, - weight DECIMAL(10, 2) -) WITH ( - 'connector' = 'kafka', - 'topic' = 'products_binlog', - 'properties.bootstrap.servers' = 'localhost:9092', - 'properties.group.id' = 'testGroup', - -- using 'debezium-json' as the format to interpret Debezium JSON messages - -- please use 'debezium-avro-confluent' if Debezium encodes messages in Avro format - 'format' = 'debezium-json' -) -``` - -In some cases, users may setup the Debezium Kafka Connect with the Kafka configuration `'value.converter.schemas.enable'` enabled to include schema in the message. Then the Debezium JSON message may look like this: - -```json -{ - "schema": {...}, - "payload": { - "before": { - "id": 111, - "name": "scooter", - "description": "Big 2-wheel scooter", - "weight": 5.18 - }, - "after": { - "id": 111, - "name": "scooter", - "description": "Big 2-wheel scooter", - "weight": 5.15 - }, - "source": {...}, - "op": "u", - "ts_ms": 1589362330904, - "transaction": null - } -} -``` - -In order to interpret such messages, you need to add the option `'debezium-json.schema-include' = 'true'` into above DDL WITH clause (`false` by default). Usually, this is not recommended to include schema because this makes the messages very verbose and reduces parsing performance. - -After registering the topic as a Flink table, then you can consume the Debezium messages as a changelog source. - -```sql --- a real-time materialized view on the MySQL "products" --- which calculate the latest average of weight for the same products -SELECT name, AVG(weight) FROM topic_products GROUP BY name; - --- synchronize all the data and incremental changes of MySQL "products" table to --- Elasticsearch "products" index for future searching -INSERT INTO elasticsearch_products -SELECT * FROM topic_products; -``` - -Available Metadata ------------------- - -The following format metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition. - -Attention Format metadata fields are only available if the -corresponding connector forwards format metadata. Currently, only the Kafka connector is able to expose -metadata fields for its value format. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
KeyData TypeDescription
schemaSTRING NULLJSON string describing the schema of the payload. Null if the schema is not included in - the Debezium record.
ingestion-timestampTIMESTAMP_LTZ(3) NULLThe timestamp at which the connector processed the event. Corresponds to the ts_ms - field in the Debezium record.
source.timestampTIMESTAMP_LTZ(3) NULLThe timestamp at which the source system created the event. Corresponds to the source.ts_ms - field in the Debezium record.
source.databaseSTRING NULLThe originating database. Corresponds to the source.db field in the - Debezium record if available.
source.schemaSTRING NULLThe originating database schema. Corresponds to the source.schema field in the - Debezium record if available.
source.tableSTRING NULLThe originating database table. Corresponds to the source.table or source.collection - field in the Debezium record if available.
source.propertiesMAP<STRING, STRING> NULLMap of various source properties. Corresponds to the source field in the Debezium record.
- -The following example shows how to access Debezium metadata fields in Kafka: - -```sql -CREATE TABLE KafkaTable ( - origin_ts TIMESTAMP(3) METADATA FROM 'value.ingestion-timestamp' VIRTUAL, - event_time TIMESTAMP(3) METADATA FROM 'value.source.timestamp' VIRTUAL, - origin_database STRING METADATA FROM 'value.source.database' VIRTUAL, - origin_schema STRING METADATA FROM 'value.source.schema' VIRTUAL, - origin_table STRING METADATA FROM 'value.source.table' VIRTUAL, - origin_properties MAP METADATA FROM 'value.source.properties' VIRTUAL, - user_id BIGINT, - item_id BIGINT, - behavior STRING -) WITH ( - 'connector' = 'kafka', - 'topic' = 'user_behavior', - 'properties.bootstrap.servers' = 'localhost:9092', - 'properties.group.id' = 'testGroup', - 'scan.startup.mode' = 'earliest-offset', - 'value.format' = 'debezium-json' -); -``` - -Format Options ----------------- - -Flink provides `debezium-avro-confluent` and `debezium-json` formats to interpret Avro or Json messages produced by Debezium. -Use format `debezium-avro-confluent` to interpret Debezium Avro messages and format `debezium-json` to interpret Debezium Json messages. - -{{< tabs "a8edce02-58d5-4e0b-bc4b-75d05a98a0f9" >}} -{{< tab "Debezium Avro" >}} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
OptionRequiredDefaultTypeDescription
format
required(none)StringSpecify what format to use, here should be 'debezium-avro-confluent'.
debezium-avro-confluent.basic-auth.credentials-source
optional(none)StringBasic auth credentials source for Schema Registry
debezium-avro-confluent.basic-auth.user-info
optional(none)StringBasic auth user info for schema registry
debezium-avro-confluent.bearer-auth.credentials-source
optional(none)StringBearer auth credentials source for Schema Registry
debezium-avro-confluent.bearer-auth.token
optional(none)StringBearer auth token for Schema Registry
debezium-avro-confluent.properties
optional(none)MapProperties map that is forwarded to the underlying Schema Registry. This is useful for options that are not officially exposed via Flink config options. However, note that Flink options have higher precedence.
debezium-avro-confluent.ssl.keystore.location
optional(none)StringLocation / File of SSL keystore
debezium-avro-confluent.ssl.keystore.password
optional(none)StringPassword for SSL keystore
debezium-avro-confluent.ssl.truststore.location
optional(none)StringLocation / File of SSL truststore
debezium-avro-confluent.ssl.truststore.password
optional(none)StringPassword for SSL truststore
debezium-avro-confluent.schema
optional(none)StringThe schema registered or to be registered in the Confluent Schema Registry. If no schema is provided Flink converts the table schema to avro schema. The schema provided must match the Debezium schema which is a nullable record type including fields 'before', 'after', 'op'.
debezium-avro-confluent.subject
optional(none)StringThe Confluent Schema Registry subject under which to register the schema used by this format during serialization. By default, 'kafka' and 'upsert-kafka' connectors use '<topic_name>-value' or '<topic_name>-key' as the default subject name if this format is used as the value or key format. But for other connectors (e.g. 'filesystem'), the subject option is required when used as sink.
debezium-avro-confluent.url
required(none)StringThe URL of the Confluent Schema Registry to fetch/register schemas.
- -{{< /tab >}} -{{< tab "Debezium Json" >}} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
OptionRequiredDefaultTypeDescription
format
required(none)StringSpecify what format to use, here should be 'debezium-json'.
debezium-json.schema-include
optionalfalseBooleanWhen setting up a Debezium Kafka Connect, users may enable a Kafka configuration 'value.converter.schemas.enable' to include schema in the message. - This option indicates whether the Debezium JSON message includes the schema or not.
debezium-json.ignore-parse-errors
optionalfalseBooleanSkip fields and rows with parse errors instead of failing. - Fields are set to null in case of errors.
debezium-json.timestamp-format.standard
optional'SQL'StringSpecify the input and output timestamp format. Currently supported values are 'SQL' and 'ISO-8601': -
    -
  • Option 'SQL' will parse input timestamp in "yyyy-MM-dd HH:mm:ss.s{precision}" format, e.g '2020-12-30 12:13:14.123' and output timestamp in the same format.
  • -
  • Option 'ISO-8601'will parse input timestamp in "yyyy-MM-ddTHH:mm:ss.s{precision}" format, e.g '2020-12-30T12:13:14.123' and output timestamp in the same format.
  • -
-
debezium-json.map-null-key.mode
optional'FAIL'StringSpecify the handling mode when serializing null keys for map data. Currently supported values are 'FAIL', 'DROP' and 'LITERAL': -
    -
  • Option 'FAIL' will throw exception when encountering map with null key.
  • -
  • Option 'DROP' will drop null key entries for map data.
  • -
  • Option 'LITERAL' will replace null key with string literal. The string literal is defined by debezium-json.map-null-key.literal option.
  • -
-
debezium-json.map-null-key.literal
optional'null'StringSpecify string literal to replace null key when 'debezium-json.map-null-key.mode' is LITERAL.
debezium-json.encode.decimal-as-plain-number
optionalfalseBooleanEncode all decimals as plain numbers instead of possible scientific notations. By default, decimals may be written using scientific notation. For example, 0.000000027 is encoded as 2.7E-8 by default, and will be written as 0.000000027 if set this option to true.
- -{{< /tab >}} -{{< /tabs >}} - -Caveats ----------------- - -### Duplicate change events - -Under normal operating scenarios, the Debezium application delivers every change event **exactly-once**. Flink works pretty well when consuming Debezium produced events in this situation. -However, Debezium application works in **at-least-once** delivery if any failover happens. See more details about delivery guarantee from [Debezium documentation](https://debezium.io/documentation/faq/#what_happens_when_an_application_stops_or_crashes). -That means, in the abnormal situations, Debezium may deliver duplicate change events to Kafka and Flink will get the duplicate events. -This may cause Flink query to get wrong results or unexpected exceptions. Thus, it is recommended to set job configuration [`table.exec.source.cdc-events-duplicate`]({{< ref "docs/dev/table/config" >}}#table-exec-source-cdc-events-duplicate) to `true` and define PRIMARY KEY on the source in this situation. -Framework will generate an additional stateful operator, and use the primary key to deduplicate the change events and produce a normalized changelog stream. - -### Consuming data produced by Debezium Postgres Connector - -If you are using [Debezium Connector for PostgreSQL](https://debezium.io/documentation/reference/1.2/connectors/postgresql.html) to capture the changes to Kafka, please make sure the [REPLICA IDENTITY](https://www.postgresql.org/docs/current/sql-altertable.html#SQL-CREATETABLE-REPLICA-IDENTITY) configuration of the monitored PostgreSQL table has been set to `FULL` which is by default `DEFAULT`. -Otherwise, Flink SQL currently will fail to interpret the Debezium data. - -In `FULL` strategy, the UPDATE and DELETE events will contain the previous values of all the table’s columns. In other strategies, the "before" field of UPDATE and DELETE events will only contain primary key columns or null if no primary key. -You can change the `REPLICA IDENTITY` by running `ALTER TABLE REPLICA IDENTITY FULL`. -See more details in [Debezium Documentation for PostgreSQL REPLICA IDENTITY](https://debezium.io/documentation/reference/1.2/connectors/postgresql.html#postgresql-replica-identity). - -Data Type Mapping ----------------- - -Currently, the Debezium format uses JSON and Avro format for serialization and deserialization. Please refer to [JSON Format documentation]({{< ref "docs/connectors/table/formats/json" >}}#data-type-mapping) and [Confluent Avro Format documentation]({< ref "docs/connectors/table/formats/avro-confluent" >}}#data-type-mapping) for more details about the data type mapping. - diff --git a/flink-confluent-schema-registry-e2e-tests/pom.xml b/flink-confluent-schema-registry-e2e-tests/pom.xml deleted file mode 100644 index 5305a7de3..000000000 --- a/flink-confluent-schema-registry-e2e-tests/pom.xml +++ /dev/null @@ -1,135 +0,0 @@ - - - - org.apache.flink - flink-connector-kafka-parent - 3.1-SNAPSHOT - - 4.0.0 - - flink-confluent-schema-registry - Flink : E2E Tests : Confluent schema registry - - - UTF-8 - - - - - confluent - https://packages.confluent.io/maven/ - - - - - - org.apache.flink - flink-streaming-java - ${flink.version} - provided - - - org.apache.flink - flink-connector-kafka - ${project.version} - - - org.apache.flink - flink-avro - ${flink.version} - - - org.apache.flink - flink-avro-confluent-registry - ${project.version} - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - TestAvroConsumerConfluent - package - - shade - - - TestAvroConsumerConfluent - - - org.apache.flink.schema.registry.test.TestAvroConsumerConfluent - - - - - - - - org.apache.avro - avro-maven-plugin - ${avro.version} - - - generate-sources - - schema - - - ${project.basedir}/src/main/resources/avro/ - ${project.basedir}/target/generated-sources/ - PRIVATE - - **/*.avsc - - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - - - dependency-convergence - - enforce - - - true - - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - - **/example/avro/* - - - - - diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java deleted file mode 100644 index e3f414978..000000000 --- a/flink-confluent-schema-registry-e2e-tests/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.schema.registry.test; - -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.serialization.SimpleStringSchema; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; -import org.apache.flink.connector.kafka.sink.KafkaSink; -import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema; -import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; - -import example.avro.User; -import org.apache.avro.specific.SpecificRecordBase; -import org.apache.kafka.clients.producer.ProducerConfig; - -import java.util.Properties; - -/** - * A simple example that shows how to read from and write to Kafka with Confluent Schema Registry. - * This will read AVRO messages from the input topic, parse them into a POJO type via checking the - * Schema by calling Schema registry. Then this example publish the POJO type to kafka by converting - * the POJO to AVRO and verifying the schema. --input-topic test-input --output-string-topic - * test-output --output-avro-topic test-avro-output --output-subject --bootstrap.servers - * localhost:9092 --schema-registry-url http://localhost:8081 --group.id myconsumer - */ -public class TestAvroConsumerConfluent { - - public static void main(String[] args) throws Exception { - // parse input arguments - final ParameterTool parameterTool = ParameterTool.fromArgs(args); - - if (parameterTool.getNumberOfParameters() < 6) { - System.out.println( - "Missing parameters!\n" - + "Usage: Kafka --input-topic --output-string-topic --output-avro-topic " - + "--bootstrap.servers " - + "--schema-registry-url --group.id "); - return; - } - Properties config = new Properties(); - config.setProperty("bootstrap.servers", parameterTool.getRequired("bootstrap.servers")); - config.setProperty("group.id", parameterTool.getRequired("group.id")); - String schemaRegistryUrl = parameterTool.getRequired("schema-registry-url"); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStreamSource input = - env.addSource( - new FlinkKafkaConsumer<>( - parameterTool.getRequired("input-topic"), - ConfluentRegistryAvroDeserializationSchema.forSpecific( - User.class, schemaRegistryUrl), - config) - .setStartFromEarliest()); - - SingleOutputStreamOperator mapToString = - input.map((MapFunction) SpecificRecordBase::toString); - - KafkaSink stringSink = - KafkaSink.builder() - .setBootstrapServers( - config.getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) - .setRecordSerializer( - KafkaRecordSerializationSchema.builder() - .setValueSerializationSchema(new SimpleStringSchema()) - .setTopic(parameterTool.getRequired("output-string-topic")) - .build()) - .setKafkaProducerConfig(config) - .build(); - mapToString.sinkTo(stringSink); - - KafkaSink avroSink = - KafkaSink.builder() - .setBootstrapServers( - config.getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) - .setRecordSerializer( - KafkaRecordSerializationSchema.builder() - .setValueSerializationSchema( - ConfluentRegistryAvroSerializationSchema - .forSpecific( - User.class, - parameterTool.getRequired( - "output-subject"), - schemaRegistryUrl)) - .setTopic(parameterTool.getRequired("output-avro-topic")) - .build()) - .build(); - input.sinkTo(avroSink); - - env.execute("Kafka Confluent Schema Registry AVRO Example"); - } -} diff --git a/flink-confluent-schema-registry-e2e-tests/src/main/resources/avro/user.avsc b/flink-confluent-schema-registry-e2e-tests/src/main/resources/avro/user.avsc deleted file mode 100644 index aca9b83f9..000000000 --- a/flink-confluent-schema-registry-e2e-tests/src/main/resources/avro/user.avsc +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - {"namespace": "example.avro", - "type": "record", - "name": "User", - "fields": [ - {"name": "name", "type": "string", "default": ""}, - {"name": "favoriteNumber", "type": "string", "default": ""}, - {"name": "favoriteColor", "type": "string", "default": ""}, - {"name": "eventType","type": {"name": "EventType","type": "enum", "symbols": ["meeting"] }} - ] -} diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index 9c69af80a..be24c03e4 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -91,7 +91,7 @@ under the License. org.apache.flink flink-sql-avro-confluent-registry - ${project.version} + ${flink.version} test @@ -194,7 +194,7 @@ under the License. org.apache.flink flink-sql-avro-confluent-registry - ${project.version} + ${flink.version} avro-confluent.jar jar ${project.build.directory}/dependencies diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 0dd3e40a1..fe097505b 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -236,7 +236,7 @@ under the License. org.apache.flink flink-avro-confluent-registry - ${project.version} + ${flink.version} test diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java index b0d61499a..7ab050359 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java @@ -866,7 +866,7 @@ private SerializationSchema createConfluentAvroSerSchema( private SerializationSchema createDebeziumAvroSerSchema( RowType rowType, String subject) { - return new DebeziumAvroSerializationSchema(rowType, TEST_REGISTRY_URL, subject, null, null); + return new DebeziumAvroSerializationSchema(rowType, TEST_REGISTRY_URL, subject, null); } // -------------------------------------------------------------------------------------------- diff --git a/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/1483f895-db24-4888-a2fa-991d602eaafc b/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/1483f895-db24-4888-a2fa-991d602eaafc deleted file mode 100644 index e69de29bb..000000000 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/bfcdf96e-7dcd-4715-9f39-73483823ec16 b/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/bfcdf96e-7dcd-4715-9f39-73483823ec16 deleted file mode 100644 index e69de29bb..000000000 diff --git a/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/stored.rules b/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/stored.rules deleted file mode 100644 index 8f05e97f7..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/archunit-violations/stored.rules +++ /dev/null @@ -1,4 +0,0 @@ -# -#Mon Apr 04 16:32:09 CEST 2022 -Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=bfcdf96e-7dcd-4715-9f39-73483823ec16 -ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=1483f895-db24-4888-a2fa-991d602eaafc diff --git a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-avro-confluent-registry/pom.xml deleted file mode 100644 index 4cfc94b4d..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/pom.xml +++ /dev/null @@ -1,204 +0,0 @@ - - - - - org.apache.flink - flink-formats-kafka - 3.1-SNAPSHOT - - 4.0.0 - - flink-avro-confluent-registry - - Flink : Formats : Avro confluent registry - - - - confluent - https://packages.confluent.io/maven/ - - - - - - io.confluent - kafka-schema-registry-client - ${confluent.version} - - - org.apache.avro - avro - - - org.slf4j - slf4j-log4j12 - - - org.lz4 - lz4-java - - - io.swagger - swagger-core - - - com.google.guava - failureaccess - - - com.google.guava - listenablefuture - - - io.swagger.core.v3 - swagger-annotations - - - com.google.errorprone - error_prone_annotations - - - com.google.j2objc - j2objc-annotations - - - com.google.code.findbugs - jsr305 - - - org.checkerframework - checker-qual - - - - - org.apache.flink - flink-core - ${flink.version} - provided - - - org.apache.flink - flink-avro - ${flink.version} - - - org.apache.flink - flink-table-api-java - ${flink.version} - provided - - - - - - org.apache.flink - flink-table-common - ${flink.version} - provided - true - - - - org.apache.flink - flink-connector-files - ${flink.version} - provided - true - - - - - org.apache.flink - flink-table-api-java - ${flink.version} - test - test-jar - - - org.apache.flink - flink-table-runtime - ${flink.version} - test - - - org.apache.flink - flink-table-common - ${flink.version} - test - test-jar - - - org.apache.flink - flink-avro - ${flink.version} - test - test-jar - - - org.powermock - powermock-reflect - ${powermock.version} - test - - - - - - org.apache.flink - flink-architecture-tests-test - - - com.google.guava - guava - - - test - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-format-common - - - - - - - - - - diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java deleted file mode 100644 index 42e7353d0..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/AvroConfluentFormatOptions.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; - -import java.util.Map; - -/** Options for Schema Registry Avro format. */ -@PublicEvolving -public class AvroConfluentFormatOptions { - - public static final ConfigOption URL = - ConfigOptions.key("url") - .stringType() - .noDefaultValue() - .withFallbackKeys("schema-registry.url") - .withDescription( - "The URL of the Confluent Schema Registry to fetch/register schemas."); - - public static final ConfigOption SUBJECT = - ConfigOptions.key("subject") - .stringType() - .noDefaultValue() - .withFallbackKeys("schema-registry.subject") - .withDescription( - "The Confluent Schema Registry subject under which to register the " - + "schema used by this format during serialization. By default, " - + "'kafka' and 'upsert-kafka' connectors use '-value' " - + "or '-key' as the default subject name if this format " - + "is used as the value or key format. But for other connectors (e.g. 'filesystem'), " - + "the subject option is required when used as sink."); - - public static final ConfigOption SCHEMA = - ConfigOptions.key("schema") - .stringType() - .noDefaultValue() - .withFallbackKeys("schema-registry.schema") - .withDescription( - "The schema registered or to be registered in the Confluent Schema Registry. " - + "If no schema is provided Flink converts the table schema to avro schema. " - + "The schema provided must match the table schema ('avro-confluent') or " - + "the Debezium schema which is a nullable record type including " - + "fields 'before', 'after', 'op' ('debezium-avro-confluent')."); - - // -------------------------------------------------------------------------------------------- - // Commonly used options maintained by Flink for convenience - // -------------------------------------------------------------------------------------------- - - public static final ConfigOption SSL_KEYSTORE_LOCATION = - ConfigOptions.key("ssl.keystore.location") - .stringType() - .noDefaultValue() - .withDescription("Location / File of SSL keystore"); - - public static final ConfigOption SSL_KEYSTORE_PASSWORD = - ConfigOptions.key("ssl.keystore.password") - .stringType() - .noDefaultValue() - .withDescription("Password for SSL keystore"); - - public static final ConfigOption SSL_TRUSTSTORE_LOCATION = - ConfigOptions.key("ssl.truststore.location") - .stringType() - .noDefaultValue() - .withDescription("Location / File of SSL truststore"); - - public static final ConfigOption SSL_TRUSTSTORE_PASSWORD = - ConfigOptions.key("ssl.truststore.password") - .stringType() - .noDefaultValue() - .withDescription("Password for SSL truststore"); - - public static final ConfigOption BASIC_AUTH_CREDENTIALS_SOURCE = - ConfigOptions.key("basic-auth.credentials-source") - .stringType() - .noDefaultValue() - .withDescription("Basic auth credentials source for Schema Registry"); - - public static final ConfigOption BASIC_AUTH_USER_INFO = - ConfigOptions.key("basic-auth.user-info") - .stringType() - .noDefaultValue() - .withDescription("Basic auth user info for schema registry"); - - public static final ConfigOption BEARER_AUTH_CREDENTIALS_SOURCE = - ConfigOptions.key("bearer-auth.credentials-source") - .stringType() - .noDefaultValue() - .withDescription("Bearer auth credentials source for Schema Registry"); - - public static final ConfigOption BEARER_AUTH_TOKEN = - ConfigOptions.key("bearer-auth.token") - .stringType() - .noDefaultValue() - .withDescription("Bearer auth token for Schema Registry"); - - // -------------------------------------------------------------------------------------------- - // Fallback properties - // -------------------------------------------------------------------------------------------- - - public static final ConfigOption> PROPERTIES = - ConfigOptions.key("properties") - .mapType() - .noDefaultValue() - .withDescription( - "Properties map that is forwarded to the underlying Schema Registry. " - + "This is useful for options that are not officially exposed " - + "via Flink config options. However, note that Flink options " - + "have higher precedence."); - - private AvroConfluentFormatOptions() {} -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java deleted file mode 100644 index 53be09c74..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProvider.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.formats.avro.SchemaCoder; - -import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; - -import javax.annotation.Nullable; - -import java.util.Map; -import java.util.Objects; - -/** - * A {@link SchemaCoder.SchemaCoderProvider} that uses a cached schema registry client underneath. - */ -@Internal -class CachedSchemaCoderProvider implements SchemaCoder.SchemaCoderProvider { - - private static final long serialVersionUID = 8610401613495438381L; - private final String subject; - private final String url; - private final int identityMapCapacity; - private final @Nullable Map registryConfigs; - - CachedSchemaCoderProvider(String url, int identityMapCapacity) { - this(null, url, identityMapCapacity, null); - } - - CachedSchemaCoderProvider( - @Nullable String subject, - String url, - int identityMapCapacity, - @Nullable Map registryConfigs) { - this.subject = subject; - this.url = Objects.requireNonNull(url); - this.identityMapCapacity = identityMapCapacity; - this.registryConfigs = registryConfigs; - } - - @Override - public SchemaCoder get() { - return new ConfluentSchemaRegistryCoder( - this.subject, - new CachedSchemaRegistryClient(url, identityMapCapacity, registryConfigs)); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - CachedSchemaCoderProvider that = (CachedSchemaCoderProvider) o; - return identityMapCapacity == that.identityMapCapacity - && Objects.equals(subject, that.subject) - && url.equals(that.url) - && Objects.equals(registryConfigs, that.registryConfigs); - } - - @Override - public int hashCode() { - return Objects.hash(subject, url, identityMapCapacity, registryConfigs); - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java deleted file mode 100644 index c448a91b8..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroDeserializationSchema.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent; - -import org.apache.flink.formats.avro.AvroDeserializationSchema; -import org.apache.flink.formats.avro.RegistryAvroDeserializationSchema; -import org.apache.flink.formats.avro.SchemaCoder; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.specific.SpecificRecord; - -import javax.annotation.Nullable; - -import java.util.Map; - -/** - * Deserialization schema that deserializes from Avro binary format using {@link SchemaCoder} that - * uses Confluent Schema Registry. - * - * @param type of record it produces - */ -public class ConfluentRegistryAvroDeserializationSchema - extends RegistryAvroDeserializationSchema { - - private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; - - private static final long serialVersionUID = -1671641202177852775L; - - /** - * Creates a Avro deserialization schema. - * - * @param recordClazz class to which deserialize. Should be either {@link SpecificRecord} or - * {@link GenericRecord}. - * @param reader reader's Avro schema. Should be provided if recordClazz is {@link - * GenericRecord} - * @param schemaCoderProvider provider for schema coder that reads writer schema from Confluent - * Schema Registry - */ - private ConfluentRegistryAvroDeserializationSchema( - Class recordClazz, - @Nullable Schema reader, - SchemaCoder.SchemaCoderProvider schemaCoderProvider) { - super(recordClazz, reader, schemaCoderProvider); - } - - /** - * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link - * GenericRecord} using the provided reader schema and looks up the writer schema in the - * Confluent Schema Registry. - * - *

By default, this method supports up to 1000 cached schema versions. - * - * @param schema schema of produced records - * @param url url of schema registry to connect - * @return deserialized record in form of {@link GenericRecord} - */ - public static ConfluentRegistryAvroDeserializationSchema forGeneric( - Schema schema, String url) { - return forGeneric(schema, url, DEFAULT_IDENTITY_MAP_CAPACITY); - } - - /** - * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link - * GenericRecord} using the provided reader schema and looks up the writer schema in the - * Confluent Schema Registry. - * - * @param schema schema of produced records - * @param url url of schema registry to connect - * @param identityMapCapacity maximum number of cached schema versions - * @return deserialized record in form of {@link GenericRecord} - */ - public static ConfluentRegistryAvroDeserializationSchema forGeneric( - Schema schema, String url, int identityMapCapacity) { - return forGeneric(schema, url, identityMapCapacity, null); - } - - /** - * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link - * GenericRecord} using the provided reader schema and looks up the writer schema in the - * Confluent Schema Registry. - * - *

By default, this method supports up to 1000 cached schema versions. - * - * @param schema schema of produced records - * @param url URL of schema registry to connect - * @param registryConfigs map with additional schema registry configs (for example SSL - * properties) - * @return deserialized record in form of {@link GenericRecord} - */ - public static ConfluentRegistryAvroDeserializationSchema forGeneric( - Schema schema, String url, @Nullable Map registryConfigs) { - return forGeneric(schema, url, DEFAULT_IDENTITY_MAP_CAPACITY, registryConfigs); - } - - /** - * Creates {@link ConfluentRegistryAvroDeserializationSchema} that produces {@link - * GenericRecord} using the provided reader schema and looks up the writer schema in the - * Confluent Schema Registry. - * - * @param schema schema of produced records - * @param url URL of schema registry to connect - * @param identityMapCapacity maximum number of cached schema versions - * @param registryConfigs map with additional schema registry configs (for example SSL - * properties) - * @return deserialized record in form of {@link GenericRecord} - */ - public static ConfluentRegistryAvroDeserializationSchema forGeneric( - Schema schema, - String url, - int identityMapCapacity, - @Nullable Map registryConfigs) { - return new ConfluentRegistryAvroDeserializationSchema<>( - GenericRecord.class, - schema, - new CachedSchemaCoderProvider(null, url, identityMapCapacity, registryConfigs)); - } - - /** - * Creates {@link AvroDeserializationSchema} that produces classes that were generated from Avro - * schema and looks up the writer schema in the Confluent Schema Registry. - * - *

By default, this method supports up to 1000 cached schema versions. - * - * @param tClass class of record to be produced - * @param url url of schema registry to connect - * @return deserialized record - */ - public static - ConfluentRegistryAvroDeserializationSchema forSpecific(Class tClass, String url) { - return forSpecific(tClass, url, DEFAULT_IDENTITY_MAP_CAPACITY, null); - } - - /** - * Creates {@link AvroDeserializationSchema} that produces classes that were generated from Avro - * schema and looks up the writer schema in the Confluent Schema Registry. - * - * @param tClass class of record to be produced - * @param url url of schema registry to connect - * @param identityMapCapacity maximum number of cached schema versions - * @return deserialized record - */ - public static - ConfluentRegistryAvroDeserializationSchema forSpecific( - Class tClass, String url, int identityMapCapacity) { - return forSpecific(tClass, url, identityMapCapacity, null); - } - - /** - * Creates {@link AvroDeserializationSchema} that produces classes that were generated from Avro - * schema and looks up the writer schema in the Confluent Schema Registry. - * - *

By default, this method supports up to 1000 cached schema versions. - * - * @param tClass class of record to be produced - * @param url URL of schema registry to connect - * @param registryConfigs map with additional schema registry configs (for example SSL - * properties) - * @return deserialized record - */ - public static - ConfluentRegistryAvroDeserializationSchema forSpecific( - Class tClass, String url, @Nullable Map registryConfigs) { - return forSpecific(tClass, url, DEFAULT_IDENTITY_MAP_CAPACITY, registryConfigs); - } - - /** - * Creates {@link AvroDeserializationSchema} that produces classes that were generated from Avro - * schema and looks up the writer schema in the Confluent Schema Registry. - * - * @param tClass class of record to be produced - * @param url URL of schema registry to connect - * @param identityMapCapacity maximum number of cached schema versions - * @param registryConfigs map with additional schema registry configs (for example SSL - * properties) - * @return deserialized record - */ - public static - ConfluentRegistryAvroDeserializationSchema forSpecific( - Class tClass, - String url, - int identityMapCapacity, - @Nullable Map registryConfigs) { - return new ConfluentRegistryAvroDeserializationSchema<>( - tClass, - null, - new CachedSchemaCoderProvider(null, url, identityMapCapacity, registryConfigs)); - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java deleted file mode 100644 index b3b574c27..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentRegistryAvroSerializationSchema.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent; - -import org.apache.flink.formats.avro.AvroSerializationSchema; -import org.apache.flink.formats.avro.RegistryAvroSerializationSchema; -import org.apache.flink.formats.avro.SchemaCoder; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.specific.SpecificRecord; - -import javax.annotation.Nullable; - -import java.util.Map; - -/** - * Serialization schema that serializes to Avro binary format that uses Confluent Schema Registry. - * - * @param the type to be serialized - */ -public class ConfluentRegistryAvroSerializationSchema - extends RegistryAvroSerializationSchema { - - private static final int DEFAULT_IDENTITY_MAP_CAPACITY = 1000; - - private static final long serialVersionUID = -1771641202177852775L; - - /** - * Creates a Avro serialization schema. - * - * @param recordClazz class to serialize. Should be either {@link SpecificRecord} or {@link - * GenericRecord}. - * @param schema writer's Avro schema. Should be provided if recordClazz is {@link - * GenericRecord} - * @param schemaCoderProvider provider for schema coder that writes the writer schema to - * Confluent Schema Registry - */ - private ConfluentRegistryAvroSerializationSchema( - Class recordClazz, - Schema schema, - SchemaCoder.SchemaCoderProvider schemaCoderProvider) { - super(recordClazz, schema, schemaCoderProvider); - } - - /** - * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from - * Avro schema and writes the writer schema to Confluent Schema Registry. - * - * @param tClass the type to be serialized - * @param subject subject of schema registry to produce - * @param schemaRegistryUrl URL of schema registry to connect - * @return serialized record - */ - public static - ConfluentRegistryAvroSerializationSchema forSpecific( - Class tClass, String subject, String schemaRegistryUrl) { - return forSpecific(tClass, subject, schemaRegistryUrl, null); - } - - /** - * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from - * Avro schema and writes the writer schema to Confluent Schema Registry. - * - * @param tClass the type to be serialized - * @param subject subject of schema registry to produce - * @param schemaRegistryUrl URL of schema registry to connect - * @param registryConfigs map with additional schema registry configs (for example SSL - * properties) - * @return serialized record - */ - public static - ConfluentRegistryAvroSerializationSchema forSpecific( - Class tClass, - String subject, - String schemaRegistryUrl, - @Nullable Map registryConfigs) { - return new ConfluentRegistryAvroSerializationSchema<>( - tClass, - null, - new CachedSchemaCoderProvider( - subject, - schemaRegistryUrl, - DEFAULT_IDENTITY_MAP_CAPACITY, - registryConfigs)); - } - - /** - * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from - * Avro schema and writes the writer schema to Confluent Schema Registry. - * - * @param subject subject of schema registry to produce - * @param schema schema that will be used for serialization - * @param schemaRegistryUrl URL of schema registry to connect - * @return serialized record - */ - public static ConfluentRegistryAvroSerializationSchema forGeneric( - String subject, Schema schema, String schemaRegistryUrl) { - return forGeneric(subject, schema, schemaRegistryUrl, null); - } - - /** - * Creates {@link AvroSerializationSchema} that produces byte arrays that were generated from - * Avro schema and writes the writer schema to Confluent Schema Registry. - * - * @param subject subject of schema registry to produce - * @param schema schema that will be used for serialization - * @param schemaRegistryUrl URL of schema registry to connect - * @param registryConfigs map with additional schema registry configs (for example SSL - * properties) - * @return serialized record - */ - public static ConfluentRegistryAvroSerializationSchema forGeneric( - String subject, - Schema schema, - String schemaRegistryUrl, - @Nullable Map registryConfigs) { - return new ConfluentRegistryAvroSerializationSchema<>( - GenericRecord.class, - schema, - new CachedSchemaCoderProvider( - subject, - schemaRegistryUrl, - DEFAULT_IDENTITY_MAP_CAPACITY, - registryConfigs)); - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java deleted file mode 100644 index 7be314531..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoder.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent; - -import org.apache.flink.formats.avro.SchemaCoder; - -import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; -import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; -import org.apache.avro.Schema; - -import java.io.DataInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteBuffer; - -import static java.lang.String.format; - -/** Reads and Writes schema using Confluent Schema Registry protocol. */ -public class ConfluentSchemaRegistryCoder implements SchemaCoder { - - private final SchemaRegistryClient schemaRegistryClient; - private String subject; - private static final int CONFLUENT_MAGIC_BYTE = 0; - - /** - * Creates {@link SchemaCoder} that uses provided {@link SchemaRegistryClient} to connect to - * schema registry. - * - * @param schemaRegistryClient client to connect schema registry - * @param subject subject of schema registry to produce - */ - public ConfluentSchemaRegistryCoder(String subject, SchemaRegistryClient schemaRegistryClient) { - this.schemaRegistryClient = schemaRegistryClient; - this.subject = subject; - } - - /** - * Creates {@link SchemaCoder} that uses provided {@link SchemaRegistryClient} to connect to - * schema registry. - * - * @param schemaRegistryClient client to connect schema registry - */ - public ConfluentSchemaRegistryCoder(SchemaRegistryClient schemaRegistryClient) { - this.schemaRegistryClient = schemaRegistryClient; - } - - @Override - public Schema readSchema(InputStream in) throws IOException { - DataInputStream dataInputStream = new DataInputStream(in); - - if (dataInputStream.readByte() != 0) { - throw new IOException("Unknown data format. Magic number does not match"); - } else { - int schemaId = dataInputStream.readInt(); - - try { - return schemaRegistryClient.getById(schemaId); - } catch (RestClientException e) { - throw new IOException( - format("Could not find schema with id %s in registry", schemaId), e); - } - } - } - - @Override - public void writeSchema(Schema schema, OutputStream out) throws IOException { - try { - int registeredId = schemaRegistryClient.register(subject, schema); - out.write(CONFLUENT_MAGIC_BYTE); - byte[] schemaIdBytes = ByteBuffer.allocate(4).putInt(registeredId).array(); - out.write(schemaIdBytes); - } catch (RestClientException e) { - throw new IOException("Could not register schema in registry", e); - } - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java deleted file mode 100644 index e43d5a76e..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactory.java +++ /dev/null @@ -1,264 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; -import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; -import org.apache.flink.formats.avro.AvroToRowDataConverters; -import org.apache.flink.formats.avro.RowDataToAvroConverters; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.Projection; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.format.ProjectableDecodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DeserializationFormatFactory; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; - -import org.apache.avro.Schema; -import org.apache.avro.Schema.Parser; - -import javax.annotation.Nullable; - -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.lang.String.format; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_CREDENTIALS_SOURCE; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_CREDENTIALS_SOURCE; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_TOKEN; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.PROPERTIES; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SCHEMA; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_LOCATION; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_PASSWORD; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_LOCATION; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_PASSWORD; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SUBJECT; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.URL; - -/** - * Table format factory for providing configured instances of Schema Registry Avro to RowData {@link - * SerializationSchema} and {@link DeserializationSchema}. - */ -@Internal -public class RegistryAvroFormatFactory - implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "avro-confluent"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - - String schemaRegistryURL = formatOptions.get(URL); - Optional schemaString = formatOptions.getOptional(SCHEMA); - Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); - - return new ProjectableDecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, - DataType producedDataType, - int[][] projections) { - producedDataType = Projection.of(projections).project(producedDataType); - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final Schema schema = - schemaString - .map(s -> getAvroSchema(s, rowType)) - .orElse(AvroSchemaConverter.convertToSchema(rowType)); - final TypeInformation rowDataTypeInfo = - context.createTypeInformation(producedDataType); - return new AvroRowDataDeserializationSchema( - ConfluentRegistryAvroDeserializationSchema.forGeneric( - schema, schemaRegistryURL, optionalPropertiesMap), - AvroToRowDataConverters.createRowConverter(rowType), - rowDataTypeInfo); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - }; - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - FactoryUtil.validateFactoryOptions(this, formatOptions); - - String schemaRegistryURL = formatOptions.get(URL); - Optional subject = formatOptions.getOptional(SUBJECT); - Optional schemaString = formatOptions.getOptional(SCHEMA); - Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); - - if (!subject.isPresent()) { - throw new ValidationException( - String.format( - "Option %s.%s is required for serialization", - IDENTIFIER, SUBJECT.key())); - } - - return new EncodingFormat>() { - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - final Schema schema = - schemaString - .map(s -> getAvroSchema(s, rowType)) - .orElse(AvroSchemaConverter.convertToSchema(rowType)); - return new AvroRowDataSerializationSchema( - rowType, - ConfluentRegistryAvroSerializationSchema.forGeneric( - subject.get(), schema, schemaRegistryURL, optionalPropertiesMap), - RowDataToAvroConverters.createConverter(rowType)); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - Set> options = new HashSet<>(); - options.add(URL); - return options; - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(SUBJECT); - options.add(SCHEMA); - options.add(PROPERTIES); - options.add(SSL_KEYSTORE_LOCATION); - options.add(SSL_KEYSTORE_PASSWORD); - options.add(SSL_TRUSTSTORE_LOCATION); - options.add(SSL_TRUSTSTORE_PASSWORD); - options.add(BASIC_AUTH_CREDENTIALS_SOURCE); - options.add(BASIC_AUTH_USER_INFO); - options.add(BEARER_AUTH_CREDENTIALS_SOURCE); - options.add(BEARER_AUTH_TOKEN); - return options; - } - - @Override - public Set> forwardOptions() { - return Stream.of( - URL, - SUBJECT, - SCHEMA, - PROPERTIES, - SSL_KEYSTORE_LOCATION, - SSL_KEYSTORE_PASSWORD, - SSL_TRUSTSTORE_LOCATION, - SSL_TRUSTSTORE_PASSWORD, - BASIC_AUTH_CREDENTIALS_SOURCE, - BASIC_AUTH_USER_INFO, - BEARER_AUTH_CREDENTIALS_SOURCE, - BEARER_AUTH_TOKEN) - .collect(Collectors.toSet()); - } - - public static @Nullable Map buildOptionalPropertiesMap( - ReadableConfig formatOptions) { - final Map properties = new HashMap<>(); - - formatOptions.getOptional(PROPERTIES).ifPresent(properties::putAll); - - formatOptions - .getOptional(SSL_KEYSTORE_LOCATION) - .ifPresent(v -> properties.put("schema.registry.ssl.keystore.location", v)); - formatOptions - .getOptional(SSL_KEYSTORE_PASSWORD) - .ifPresent(v -> properties.put("schema.registry.ssl.keystore.password", v)); - formatOptions - .getOptional(SSL_TRUSTSTORE_LOCATION) - .ifPresent(v -> properties.put("schema.registry.ssl.truststore.location", v)); - formatOptions - .getOptional(SSL_TRUSTSTORE_PASSWORD) - .ifPresent(v -> properties.put("schema.registry.ssl.truststore.password", v)); - formatOptions - .getOptional(BASIC_AUTH_CREDENTIALS_SOURCE) - .ifPresent(v -> properties.put("basic.auth.credentials.source", v)); - formatOptions - .getOptional(BASIC_AUTH_USER_INFO) - .ifPresent(v -> properties.put("basic.auth.user.info", v)); - formatOptions - .getOptional(BEARER_AUTH_CREDENTIALS_SOURCE) - .ifPresent(v -> properties.put("bearer.auth.credentials.source", v)); - formatOptions - .getOptional(BEARER_AUTH_TOKEN) - .ifPresent(v -> properties.put("bearer.auth.token", v)); - - if (properties.isEmpty()) { - return null; - } - return properties; - } - - private static Schema getAvroSchema(String schemaString, RowType rowType) { - LogicalType convertedDataType = - AvroSchemaConverter.convertToDataType(schemaString).getLogicalType(); - - if (convertedDataType.isNullable()) { - convertedDataType = convertedDataType.copy(false); - } - - if (!convertedDataType.equals(rowType)) { - throw new IllegalArgumentException( - format( - "Schema provided for '%s' format does not match the table schema: %s", - IDENTIFIER, schemaString)); - } - - return new Parser().parse(schemaString); - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java deleted file mode 100644 index f72309e69..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroDeserializationSchema.java +++ /dev/null @@ -1,207 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent.debezium; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; -import org.apache.flink.formats.avro.AvroToRowDataConverters; -import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Collector; - -import org.apache.avro.Schema; -import org.apache.avro.Schema.Parser; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.util.Map; -import java.util.Objects; - -import static java.lang.String.format; -import static org.apache.flink.formats.avro.registry.confluent.debezium.DebeziumAvroFormatFactory.validateSchemaString; -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; - -/** - * Deserialization schema from Debezium Avro to Flink Table/SQL internal data structure {@link - * RowData}. The deserialization schema knows Debezium's schema definition and can extract the - * database data and convert into {@link RowData} with {@link RowKind}. Deserializes a byte[] - * message as a JSON object and reads the specified fields. Failures during deserialization - * are forwarded as wrapped IOExceptions. - * - * @see Debezium - */ -@Internal -public final class DebeziumAvroDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - - /** snapshot read. */ - private static final String OP_READ = "r"; - /** insert operation. */ - private static final String OP_CREATE = "c"; - /** update operation. */ - private static final String OP_UPDATE = "u"; - /** delete operation. */ - private static final String OP_DELETE = "d"; - - private static final String REPLICA_IDENTITY_EXCEPTION = - "The \"before\" field of %s message is null, " - + "if you are using Debezium Postgres Connector, " - + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; - - /** The deserializer to deserialize Debezium Avro data. */ - private final AvroRowDataDeserializationSchema avroDeserializer; - - /** TypeInformation of the produced {@link RowData}. */ - private final TypeInformation producedTypeInfo; - - public DebeziumAvroDeserializationSchema( - RowType rowType, - TypeInformation producedTypeInfo, - String schemaRegistryUrl, - @Nullable String schemaString, - @Nullable Map registryConfigs) { - this.producedTypeInfo = producedTypeInfo; - RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); - - validateSchemaString(schemaString, debeziumAvroRowType); - Schema schema = - schemaString == null - ? AvroSchemaConverter.convertToSchema(debeziumAvroRowType) - : new Parser().parse(schemaString); - - this.avroDeserializer = - new AvroRowDataDeserializationSchema( - ConfluentRegistryAvroDeserializationSchema.forGeneric( - schema, schemaRegistryUrl, registryConfigs), - AvroToRowDataConverters.createRowConverter(debeziumAvroRowType), - producedTypeInfo); - } - - @VisibleForTesting - DebeziumAvroDeserializationSchema( - TypeInformation producedTypeInfo, - AvroRowDataDeserializationSchema avroDeserializer) { - this.producedTypeInfo = producedTypeInfo; - this.avroDeserializer = avroDeserializer; - } - - @Override - public void open(InitializationContext context) throws Exception { - avroDeserializer.open(context); - } - - @Override - public RowData deserialize(byte[] message) throws IOException { - throw new RuntimeException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); - } - - @Override - public void deserialize(byte[] message, Collector out) throws IOException { - - if (message == null || message.length == 0) { - // skip tombstone messages - return; - } - try { - GenericRowData row = (GenericRowData) avroDeserializer.deserialize(message); - - GenericRowData before = (GenericRowData) row.getField(0); - GenericRowData after = (GenericRowData) row.getField(1); - String op = row.getField(2).toString(); - if (OP_CREATE.equals(op) || OP_READ.equals(op)) { - after.setRowKind(RowKind.INSERT); - out.collect(after); - } else if (OP_UPDATE.equals(op)) { - if (before == null) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - before.setRowKind(RowKind.UPDATE_BEFORE); - after.setRowKind(RowKind.UPDATE_AFTER); - out.collect(before); - out.collect(after); - } else if (OP_DELETE.equals(op)) { - if (before == null) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); - } - before.setRowKind(RowKind.DELETE); - out.collect(before); - } else { - throw new IOException( - format( - "Unknown \"op\" value \"%s\". The Debezium Avro message is '%s'", - op, new String(message))); - } - } catch (Throwable t) { - // a big try catch to protect the processing. - throw new IOException("Can't deserialize Debezium Avro message.", t); - } - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return producedTypeInfo; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DebeziumAvroDeserializationSchema that = (DebeziumAvroDeserializationSchema) o; - return Objects.equals(avroDeserializer, that.avroDeserializer) - && Objects.equals(producedTypeInfo, that.producedTypeInfo); - } - - @Override - public int hashCode() { - return Objects.hash(avroDeserializer, producedTypeInfo); - } - - public static RowType createDebeziumAvroRowType(DataType databaseSchema) { - // Debezium Avro contains other information, e.g. "source", "ts_ms" - // but we don't need them - return (RowType) - DataTypes.ROW( - DataTypes.FIELD("before", databaseSchema.nullable()), - DataTypes.FIELD("after", databaseSchema.nullable()), - DataTypes.FIELD("op", DataTypes.STRING())) - .getLogicalType(); - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java deleted file mode 100644 index 5925f23f7..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java +++ /dev/null @@ -1,198 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent.debezium; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.Projection; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.format.ProjectableDecodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DeserializationFormatFactory; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; - -import javax.annotation.Nullable; - -import java.util.HashSet; -import java.util.Map; -import java.util.Optional; -import java.util.Set; - -import static java.lang.String.format; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_CREDENTIALS_SOURCE; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_CREDENTIALS_SOURCE; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.BEARER_AUTH_TOKEN; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.PROPERTIES; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SCHEMA; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_LOCATION; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_KEYSTORE_PASSWORD; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_LOCATION; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SSL_TRUSTSTORE_PASSWORD; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.SUBJECT; -import static org.apache.flink.formats.avro.registry.confluent.AvroConfluentFormatOptions.URL; -import static org.apache.flink.formats.avro.registry.confluent.RegistryAvroFormatFactory.buildOptionalPropertiesMap; - -/** - * Format factory for providing configured instances of Debezium Avro to RowData {@link - * DeserializationSchema}. - */ -@Internal -public class DebeziumAvroFormatFactory - implements DeserializationFormatFactory, SerializationFormatFactory { - - public static final String IDENTIFIER = "debezium-avro-confluent"; - - @Override - public DecodingFormat> createDecodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - String schemaRegistryURL = formatOptions.get(URL); - String schema = formatOptions.getOptional(SCHEMA).orElse(null); - Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); - - return new ProjectableDecodingFormat>() { - @Override - public DeserializationSchema createRuntimeDecoder( - DynamicTableSource.Context context, - DataType producedDataType, - int[][] projections) { - producedDataType = Projection.of(projections).project(producedDataType); - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation producedTypeInfo = - context.createTypeInformation(producedDataType); - return new DebeziumAvroDeserializationSchema( - rowType, - producedTypeInfo, - schemaRegistryURL, - schema, - optionalPropertiesMap); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - }; - } - - @Override - public EncodingFormat> createEncodingFormat( - DynamicTableFactory.Context context, ReadableConfig formatOptions) { - - FactoryUtil.validateFactoryOptions(this, formatOptions); - String schemaRegistryURL = formatOptions.get(URL); - Optional subject = formatOptions.getOptional(SUBJECT); - String schema = formatOptions.getOptional(SCHEMA).orElse(null); - Map optionalPropertiesMap = buildOptionalPropertiesMap(formatOptions); - - if (!subject.isPresent()) { - throw new ValidationException( - String.format( - "Option '%s.%s' is required for serialization", - IDENTIFIER, SUBJECT.key())); - } - - return new EncodingFormat>() { - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.newBuilder() - .addContainedKind(RowKind.INSERT) - .addContainedKind(RowKind.UPDATE_BEFORE) - .addContainedKind(RowKind.UPDATE_AFTER) - .addContainedKind(RowKind.DELETE) - .build(); - } - - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, DataType consumedDataType) { - final RowType rowType = (RowType) consumedDataType.getLogicalType(); - return new DebeziumAvroSerializationSchema( - rowType, schemaRegistryURL, subject.get(), schema, optionalPropertiesMap); - } - }; - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - Set> options = new HashSet<>(); - options.add(URL); - return options; - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(SUBJECT); - options.add(PROPERTIES); - options.add(SCHEMA); - options.add(SSL_KEYSTORE_LOCATION); - options.add(SSL_KEYSTORE_PASSWORD); - options.add(SSL_TRUSTSTORE_LOCATION); - options.add(SSL_TRUSTSTORE_PASSWORD); - options.add(BASIC_AUTH_CREDENTIALS_SOURCE); - options.add(BASIC_AUTH_USER_INFO); - options.add(BEARER_AUTH_CREDENTIALS_SOURCE); - options.add(BEARER_AUTH_TOKEN); - return options; - } - - static void validateSchemaString(@Nullable String schemaString, RowType rowType) { - if (schemaString != null) { - LogicalType convertedDataType = - AvroSchemaConverter.convertToDataType(schemaString).getLogicalType(); - - if (!convertedDataType.equals(rowType)) { - throw new IllegalArgumentException( - format( - "Schema provided for '%s' format must be a nullable record type with fields 'before', 'after', 'op'" - + " and schema of fields 'before' and 'after' must match the table schema: %s", - IDENTIFIER, schemaString)); - } - } - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java b/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java deleted file mode 100644 index a5b4eb7ab..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerializationSchema.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent.debezium; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; -import org.apache.flink.formats.avro.RowDataToAvroConverters; -import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; - -import org.apache.avro.Schema; -import org.apache.avro.Schema.Parser; - -import javax.annotation.Nullable; - -import java.util.Map; -import java.util.Objects; - -import static java.lang.String.format; -import static org.apache.flink.formats.avro.registry.confluent.debezium.DebeziumAvroFormatFactory.validateSchemaString; -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; - -/** - * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium - * Avro. - */ -@Internal -public class DebeziumAvroSerializationSchema implements SerializationSchema { - private static final long serialVersionUID = 1L; - - /** insert operation. */ - private static final StringData OP_INSERT = StringData.fromString("c"); - /** delete operation. */ - private static final StringData OP_DELETE = StringData.fromString("d"); - - /** The deserializer to deserialize Debezium Avro data. */ - private final AvroRowDataSerializationSchema avroSerializer; - - private transient GenericRowData outputReuse; - - public DebeziumAvroSerializationSchema( - RowType rowType, - String schemaRegistryUrl, - String schemaRegistrySubject, - @Nullable String schemaString, - @Nullable Map registryConfigs) { - RowType debeziumAvroRowType = createDebeziumAvroRowType(fromLogicalToDataType(rowType)); - validateSchemaString(schemaString, debeziumAvroRowType); - Schema schema = - schemaString == null - ? AvroSchemaConverter.convertToSchema(debeziumAvroRowType) - : new Parser().parse(schemaString); - - this.avroSerializer = - new AvroRowDataSerializationSchema( - debeziumAvroRowType, - ConfluentRegistryAvroSerializationSchema.forGeneric( - schemaRegistrySubject, schema, schemaRegistryUrl, registryConfigs), - RowDataToAvroConverters.createConverter(debeziumAvroRowType)); - } - - @VisibleForTesting - DebeziumAvroSerializationSchema(AvroRowDataSerializationSchema avroSerializer) { - this.avroSerializer = avroSerializer; - } - - @Override - public void open(InitializationContext context) throws Exception { - avroSerializer.open(context); - outputReuse = new GenericRowData(3); - } - - @Override - public byte[] serialize(RowData rowData) { - try { - switch (rowData.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - outputReuse.setField(0, null); - outputReuse.setField(1, rowData); - outputReuse.setField(2, OP_INSERT); - return avroSerializer.serialize(outputReuse); - case UPDATE_BEFORE: - case DELETE: - outputReuse.setField(0, rowData); - outputReuse.setField(1, null); - outputReuse.setField(2, OP_DELETE); - return avroSerializer.serialize(outputReuse); - default: - throw new UnsupportedOperationException( - format( - "Unsupported operation '%s' for row kind.", - rowData.getRowKind())); - } - } catch (Throwable t) { - throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DebeziumAvroSerializationSchema that = (DebeziumAvroSerializationSchema) o; - return Objects.equals(avroSerializer, that.avroSerializer); - } - - @Override - public int hashCode() { - return Objects.hash(avroSerializer); - } - - public static RowType createDebeziumAvroRowType(DataType dataType) { - // Debezium Avro contains other information, e.g. "source", "ts_ms" - // but we don't need them - return (RowType) - DataTypes.ROW( - DataTypes.FIELD("before", dataType.nullable()), - DataTypes.FIELD("after", dataType.nullable()), - DataTypes.FIELD("op", DataTypes.STRING())) - .getLogicalType(); - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory deleted file mode 100644 index e1b78a27c..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ /dev/null @@ -1,17 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -org.apache.flink.formats.avro.registry.confluent.RegistryAvroFormatFactory -org.apache.flink.formats.avro.registry.confluent.debezium.DebeziumAvroFormatFactory diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java deleted file mode 100644 index 837f39e68..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.architecture; - -import org.apache.flink.architecture.common.ImportOptions; - -import com.tngtech.archunit.core.importer.ImportOption; -import com.tngtech.archunit.junit.AnalyzeClasses; -import com.tngtech.archunit.junit.ArchTest; -import com.tngtech.archunit.junit.ArchTests; - -/** Architecture tests for test code. */ -@AnalyzeClasses( - packages = {"org.apache.flink.formats.avro.registry.confluent"}, - importOptions = { - ImportOption.OnlyIncludeTests.class, - ImportOptions.ExcludeScalaImportOption.class, - ImportOptions.ExcludeShadedImportOption.class - }) -public class TestCodeArchitectureTest { - - @ArchTest - public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java deleted file mode 100644 index cddbf1349..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent; - -import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; -import io.confluent.kafka.schemaregistry.client.rest.RestService; -import io.confluent.kafka.schemaregistry.client.security.basicauth.BasicAuthCredentialProvider; -import io.confluent.kafka.schemaregistry.client.security.bearerauth.BearerAuthCredentialProvider; -import org.junit.jupiter.api.Test; -import org.powermock.reflect.Whitebox; - -import javax.net.ssl.SSLSocketFactory; - -import java.net.URISyntaxException; -import java.util.HashMap; -import java.util.Map; - -import static org.assertj.core.api.Assertions.assertThat; - -/** - * Tests for properties set by {@link RegistryAvroFormatFactory} in {@link - * CachedSchemaCoderProvider}. - */ -class CachedSchemaCoderProviderTest { - - @Test - void testThatSslIsNotInitializedForNoSslProperties() { - CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(new HashMap<>()); - SSLSocketFactory sslSocketFactory = getSslSocketFactoryFromProvider(provider); - - assertThat(sslSocketFactory).isNull(); - } - - @Test - void testThatSslIsInitializedForSslProperties() throws URISyntaxException { - String keystoreFile = getAbsolutePath("/test-keystore.jks"); - String keystorePassword = "123456"; - Map configs = new HashMap<>(); - configs.put("schema.registry.ssl.keystore.location", keystoreFile); - configs.put("schema.registry.ssl.keystore.password", keystorePassword); - configs.put("schema.registry.ssl.truststore.location", keystoreFile); - configs.put("schema.registry.ssl.truststore.password", keystorePassword); - - CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(configs); - SSLSocketFactory sslSocketFactory = getSslSocketFactoryFromProvider(provider); - - assertThat(sslSocketFactory).isNotNull(); - } - - @Test - void testThatBasicAuthIsNotInitializedForNoBasicAuthProperties() { - CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(new HashMap<>()); - BasicAuthCredentialProvider basicAuthCredentialProvider = - getBasicAuthFromProvider(provider); - - assertThat(basicAuthCredentialProvider).isNull(); - } - - @Test - void testThatBasicAuthIsInitializedForBasicAuthProperties() { - String userPassword = "user:pwd"; - Map configs = new HashMap<>(); - configs.put("basic.auth.credentials.source", "USER_INFO"); - configs.put("basic.auth.user.info", userPassword); - - CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(configs); - BasicAuthCredentialProvider basicAuthCredentialProvider = - getBasicAuthFromProvider(provider); - - assertThat(basicAuthCredentialProvider).isNotNull(); - assertThat(basicAuthCredentialProvider.getUserInfo(null)).isEqualTo(userPassword); - } - - @Test - void testThatBearerAuthIsNotInitializedForNoBearerAuthProperties() { - CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(new HashMap<>()); - BearerAuthCredentialProvider bearerAuthCredentialProvider = - getBearerAuthFromProvider(provider); - - assertThat(bearerAuthCredentialProvider).isNull(); - } - - @Test - void testThatBearerAuthIsInitializedForBearerAuthProperties() { - String token = "123456"; - Map configs = new HashMap<>(); - configs.put("bearer.auth.credentials.source", "STATIC_TOKEN"); - configs.put("bearer.auth.token", token); - - CachedSchemaCoderProvider provider = initCachedSchemaCoderProvider(configs); - BearerAuthCredentialProvider bearerAuthCredentialProvider = - getBearerAuthFromProvider(provider); - - assertThat(bearerAuthCredentialProvider).isNotNull(); - assertThat(bearerAuthCredentialProvider.getBearerToken(null)).isEqualTo(token); - } - - private String getAbsolutePath(String path) throws URISyntaxException { - return CachedSchemaCoderProviderTest.class.getResource(path).toURI().getPath(); - } - - private CachedSchemaCoderProvider initCachedSchemaCoderProvider(Map config) { - return new CachedSchemaCoderProvider("test", "someUrl", 1000, config); - } - - private SSLSocketFactory getSslSocketFactoryFromProvider(CachedSchemaCoderProvider provider) { - return getInternalStateFromRestService("sslSocketFactory", provider); - } - - private BasicAuthCredentialProvider getBasicAuthFromProvider( - CachedSchemaCoderProvider provider) { - return getInternalStateFromRestService("basicAuthCredentialProvider", provider); - } - - private BearerAuthCredentialProvider getBearerAuthFromProvider( - CachedSchemaCoderProvider provider) { - return getInternalStateFromRestService("bearerAuthCredentialProvider", provider); - } - - private T getInternalStateFromRestService(String name, CachedSchemaCoderProvider provider) { - CachedSchemaRegistryClient cachedSchemaRegistryClient = - Whitebox.getInternalState(provider.get(), "schemaRegistryClient"); - RestService restService = - Whitebox.getInternalState(cachedSchemaRegistryClient, "restService"); - return Whitebox.getInternalState(restService, name); - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java deleted file mode 100644 index 2c93348b0..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/ConfluentSchemaRegistryCoderTest.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent; - -import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.junit.jupiter.api.Test; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; -import java.io.IOException; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link ConfluentSchemaRegistryCoder}. */ -class ConfluentSchemaRegistryCoderTest { - - @Test - void testSpecificRecordWithConfluentSchemaRegistry() throws Exception { - MockSchemaRegistryClient client = new MockSchemaRegistryClient(); - - Schema schema = - SchemaBuilder.record("testRecord").fields().optionalString("testField").endRecord(); - int schemaId = client.register("testTopic", schema); - - ConfluentSchemaRegistryCoder registryCoder = new ConfluentSchemaRegistryCoder(client); - ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream(); - DataOutputStream dataOutputStream = new DataOutputStream(byteOutStream); - dataOutputStream.writeByte(0); - dataOutputStream.writeInt(schemaId); - dataOutputStream.flush(); - - ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray()); - Schema readSchema = registryCoder.readSchema(byteInStream); - - assertThat(readSchema).isEqualTo(schema); - assertThat(byteInStream).isEmpty(); - } - - @Test - void testMagicByteVerification() throws Exception { - MockSchemaRegistryClient client = new MockSchemaRegistryClient(); - int schemaId = client.register("testTopic", Schema.create(Schema.Type.BOOLEAN)); - - ConfluentSchemaRegistryCoder coder = new ConfluentSchemaRegistryCoder(client); - ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream(); - DataOutputStream dataOutputStream = new DataOutputStream(byteOutStream); - dataOutputStream.writeByte(5); - dataOutputStream.writeInt(schemaId); - dataOutputStream.flush(); - - try (ByteArrayInputStream byteInStream = - new ByteArrayInputStream(byteOutStream.toByteArray())) { - assertThatThrownBy(() -> coder.readSchema(byteInStream)) - .isInstanceOf(IOException.class); - } - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java deleted file mode 100644 index 6ace72b19..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroFormatFactoryTest.java +++ /dev/null @@ -1,294 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; -import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; -import org.apache.flink.formats.avro.AvroToRowDataConverters; -import org.apache.flink.formats.avro.RowDataToAvroConverters; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.TestDynamicTableFactory; -import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.logical.RowType; - -import org.apache.avro.Schema.Parser; -import org.junit.jupiter.api.Test; - -import java.util.HashMap; -import java.util.Map; -import java.util.function.Consumer; - -import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.Assert.assertThrows; - -/** Tests for the {@link RegistryAvroFormatFactory}. */ -class RegistryAvroFormatFactoryTest { - - private static final ResolvedSchema SCHEMA = - ResolvedSchema.of( - Column.physical("a", DataTypes.STRING()), - Column.physical("b", DataTypes.INT()), - Column.physical("c", DataTypes.BOOLEAN())); - - private static final RowType ROW_TYPE = - (RowType) SCHEMA.toPhysicalRowDataType().getLogicalType(); - - private static final String SUBJECT = "test-subject"; - private static final String REGISTRY_URL = "http://localhost:8081"; - private static final String SCHEMA_STRING = - "{\n" - + " \"type\": \"record\",\n" - + " \"name\": \"test_record\",\n" - + " \"fields\": [\n" - + " {\n" - + " \"name\": \"a\",\n" - + " \"type\": [\n" - + " \"null\",\n" - + " \"string\"\n" - + " ],\n" - + " \"default\": null\n" - + " },\n" - + " {\n" - + " \"name\": \"b\",\n" - + " \"type\": [\n" - + " \"null\",\n" - + " \"int\"\n" - + " ],\n" - + " \"default\": null\n" - + " },\n" - + " {\n" - + " \"name\": \"c\",\n" - + " \"type\": [\n" - + " \"null\",\n" - + " \"boolean\"\n" - + " ],\n" - + " \"default\": null\n" - + " }\n" - + " ]\n" - + "}"; - - private static final Map EXPECTED_OPTIONAL_PROPERTIES = new HashMap<>(); - - static { - EXPECTED_OPTIONAL_PROPERTIES.put( - "schema.registry.ssl.keystore.location", getAbsolutePath("/test-keystore.jks")); - EXPECTED_OPTIONAL_PROPERTIES.put("schema.registry.ssl.keystore.password", "123456"); - EXPECTED_OPTIONAL_PROPERTIES.put( - "schema.registry.ssl.truststore.location", getAbsolutePath("/test-keystore.jks")); - EXPECTED_OPTIONAL_PROPERTIES.put("schema.registry.ssl.truststore.password", "123456"); - EXPECTED_OPTIONAL_PROPERTIES.put("basic.auth.credentials.source", "USER_INFO"); - EXPECTED_OPTIONAL_PROPERTIES.put("basic.auth.user.info", "user:pwd"); - EXPECTED_OPTIONAL_PROPERTIES.put("bearer.auth.token", "CUSTOM"); - } - - @Test - void testDeserializationSchema() { - final AvroRowDataDeserializationSchema expectedDeser = - new AvroRowDataDeserializationSchema( - ConfluentRegistryAvroDeserializationSchema.forGeneric( - AvroSchemaConverter.convertToSchema(ROW_TYPE), REGISTRY_URL), - AvroToRowDataConverters.createRowConverter(ROW_TYPE), - InternalTypeInfo.of(ROW_TYPE)); - - final DynamicTableSource actualSource = createTableSource(SCHEMA, getDefaultOptions()); - assertThat(actualSource).isInstanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class); - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - DeserializationSchema actualDeser = - scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); - - assertThat(actualDeser).isEqualTo(expectedDeser); - } - - @Test - void testSerializationSchema() { - final AvroRowDataSerializationSchema expectedSer = - new AvroRowDataSerializationSchema( - ROW_TYPE, - ConfluentRegistryAvroSerializationSchema.forGeneric( - SUBJECT, - AvroSchemaConverter.convertToSchema(ROW_TYPE), - REGISTRY_URL), - RowDataToAvroConverters.createConverter(ROW_TYPE)); - - final DynamicTableSink actualSink = createTableSink(SCHEMA, getDefaultOptions()); - assertThat(actualSink).isInstanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class); - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = - sinkMock.valueFormat.createRuntimeEncoder(null, SCHEMA.toPhysicalRowDataType()); - - assertThat(actualSer).isEqualTo(expectedSer); - } - - @Test - void testMissingSubjectForSink() { - final Map options = - getModifiedOptions(opts -> opts.remove("avro-confluent.subject")); - - assertThatThrownBy(() -> createTableSink(SCHEMA, options)) - .isInstanceOf(ValidationException.class) - .satisfies( - anyCauseMatches( - ValidationException.class, - "Option avro-confluent.subject is required for serialization")); - } - - @Test - void testDeserializationSchemaWithOptionalProperties() { - final AvroRowDataDeserializationSchema expectedDeser = - new AvroRowDataDeserializationSchema( - ConfluentRegistryAvroDeserializationSchema.forGeneric( - new Parser().parse(SCHEMA_STRING), - REGISTRY_URL, - EXPECTED_OPTIONAL_PROPERTIES), - AvroToRowDataConverters.createRowConverter(ROW_TYPE), - InternalTypeInfo.of(ROW_TYPE)); - - final DynamicTableSource actualSource = createTableSource(SCHEMA, getOptionalProperties()); - assertThat(actualSource).isInstanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class); - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - DeserializationSchema actualDeser = - scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); - - assertThat(actualDeser).isEqualTo(expectedDeser); - } - - @Test - void testSerializationSchemaWithOptionalProperties() { - final AvroRowDataSerializationSchema expectedSer = - new AvroRowDataSerializationSchema( - ROW_TYPE, - ConfluentRegistryAvroSerializationSchema.forGeneric( - SUBJECT, - new Parser().parse(SCHEMA_STRING), - REGISTRY_URL, - EXPECTED_OPTIONAL_PROPERTIES), - RowDataToAvroConverters.createConverter(ROW_TYPE)); - - final DynamicTableSink actualSink = createTableSink(SCHEMA, getOptionalProperties()); - assertThat(actualSink).isInstanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class); - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - SerializationSchema actualSer = - sinkMock.valueFormat.createRuntimeEncoder(null, SCHEMA.toPhysicalRowDataType()); - - assertThat(actualSer).isEqualTo(expectedSer); - } - - @Test - public void testSerializationSchemaWithInvalidOptionalSchema() { - Map optionalProperties = getOptionalProperties(); - optionalProperties.put("avro-confluent.schema", SCHEMA_STRING.replace("int", "string")); - - final DynamicTableSink actualSink = createTableSink(SCHEMA, optionalProperties); - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - assertThrows( - IllegalArgumentException.class, - () -> - sinkMock.valueFormat.createRuntimeEncoder( - null, SCHEMA.toPhysicalRowDataType())); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Returns the full options modified by the given consumer {@code optionModifier}. - * - * @param optionModifier Consumer to modify the options - */ - private Map getModifiedOptions(Consumer> optionModifier) { - Map options = getDefaultOptions(); - optionModifier.accept(options); - return options; - } - - private Map getDefaultOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", RegistryAvroFormatFactory.IDENTIFIER); - options.put("avro-confluent.subject", SUBJECT); - options.put("avro-confluent.url", REGISTRY_URL); - return options; - } - - private Map getOptionalProperties() { - final Map properties = new HashMap<>(); - // defined via Flink maintained options - properties.put( - AvroConfluentFormatOptions.SSL_KEYSTORE_LOCATION.key(), - getAbsolutePath("/test-keystore.jks")); - properties.put(AvroConfluentFormatOptions.SSL_KEYSTORE_PASSWORD.key(), "123456"); - properties.put( - AvroConfluentFormatOptions.SSL_TRUSTSTORE_LOCATION.key(), - getAbsolutePath("/test-keystore.jks")); - properties.put(AvroConfluentFormatOptions.SSL_TRUSTSTORE_PASSWORD.key(), "123456"); - properties.put(AvroConfluentFormatOptions.BASIC_AUTH_CREDENTIALS_SOURCE.key(), "USER_INFO"); - properties.put(AvroConfluentFormatOptions.BASIC_AUTH_USER_INFO.key(), "user:pwd"); - // defined via general property map - properties.put("properties.bearer.auth.token", "CUSTOM"); - properties.put("schema", SCHEMA_STRING); - - return getModifiedOptions( - opts -> - properties.forEach( - (k, v) -> - opts.put( - String.format( - "%s.%s", - RegistryAvroFormatFactory.IDENTIFIER, k), - v))); - } - - private static String getAbsolutePath(String path) { - try { - return CachedSchemaCoderProviderTest.class.getResource(path).toURI().getPath(); - } catch (Exception ex) { - throw new RuntimeException(ex); - } - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java deleted file mode 100644 index a5046ac23..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/RegistryAvroRowDataSeDeSchemaTest.java +++ /dev/null @@ -1,194 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent; - -import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; -import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; -import org.apache.flink.formats.avro.AvroToRowDataConverters; -import org.apache.flink.formats.avro.RegistryAvroDeserializationSchema; -import org.apache.flink.formats.avro.RegistryAvroSerializationSchema; -import org.apache.flink.formats.avro.RowDataToAvroConverters; -import org.apache.flink.formats.avro.generated.Address; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.formats.avro.utils.TestDataGenerator; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.binary.BinaryStringData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; - -import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; -import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; -import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.io.IOException; -import java.util.Random; - -import static org.apache.flink.formats.avro.utils.AvroTestUtils.writeRecord; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** - * Tests for {@link AvroRowDataDeserializationSchema} and {@link AvroRowDataSerializationSchema} for - * schema registry avro. - */ -class RegistryAvroRowDataSeDeSchemaTest { - private static final Schema ADDRESS_SCHEMA = Address.getClassSchema(); - - private static final Schema ADDRESS_SCHEMA_COMPATIBLE = - new Schema.Parser() - .parse( - "" - + "{\"namespace\": \"org.apache.flink.formats.avro.generated\",\n" - + " \"type\": \"record\",\n" - + " \"name\": \"Address\",\n" - + " \"fields\": [\n" - + " {\"name\": \"num\", \"type\": \"int\"},\n" - + " {\"name\": \"street\", \"type\": \"string\"}\n" - + " ]\n" - + "}"); - - private static final String SUBJECT = "address-value"; - - private static SchemaRegistryClient client; - - private Address address; - - @BeforeAll - static void beforeClass() { - client = new MockSchemaRegistryClient(); - } - - @BeforeEach - void before() { - this.address = TestDataGenerator.generateRandomAddress(new Random()); - } - - @AfterEach - void after() throws IOException, RestClientException { - client.deleteSubject(SUBJECT); - } - - @Test - void testRowDataWriteReadWithFullSchema() throws Exception { - testRowDataWriteReadWithSchema(ADDRESS_SCHEMA); - } - - @Test - void testRowDataWriteReadWithCompatibleSchema() throws Exception { - testRowDataWriteReadWithSchema(ADDRESS_SCHEMA_COMPATIBLE); - // Validates new schema has been registered. - assertThat(client.getAllVersions(SUBJECT)).hasSize(1); - } - - @Test - void testRowDataWriteReadWithPreRegisteredSchema() throws Exception { - client.register(SUBJECT, ADDRESS_SCHEMA); - testRowDataWriteReadWithSchema(ADDRESS_SCHEMA); - // Validates it does not produce new schema. - assertThat(client.getAllVersions(SUBJECT)).hasSize(1); - } - - @Test - void testRowDataReadWithNonRegistryAvro() throws Exception { - DataType dataType = AvroSchemaConverter.convertToDataType(ADDRESS_SCHEMA.toString()); - RowType rowType = (RowType) dataType.getLogicalType(); - - AvroRowDataDeserializationSchema deserializer = - getDeserializationSchema(rowType, ADDRESS_SCHEMA); - - deserializer.open(null); - - client.register(SUBJECT, ADDRESS_SCHEMA); - byte[] oriBytes = writeRecord(address, ADDRESS_SCHEMA); - assertThatThrownBy(() -> deserializer.deserialize(oriBytes)) - .isInstanceOf(IOException.class) - .hasCause(new IOException("Unknown data format. Magic number does not match")); - } - - private void testRowDataWriteReadWithSchema(Schema schema) throws Exception { - DataType dataType = AvroSchemaConverter.convertToDataType(schema.toString()); - RowType rowType = (RowType) dataType.getLogicalType(); - - AvroRowDataSerializationSchema serializer = getSerializationSchema(rowType, schema); - Schema writeSchema = AvroSchemaConverter.convertToSchema(dataType.getLogicalType()); - AvroRowDataDeserializationSchema deserializer = - getDeserializationSchema(rowType, writeSchema); - - serializer.open(null); - deserializer.open(null); - - assertThat(deserializer.deserialize(null)).isNull(); - - RowData oriData = address2RowData(address); - byte[] serialized = serializer.serialize(oriData); - RowData rowData = deserializer.deserialize(serialized); - assertThat(rowData.getArity()).isEqualTo(schema.getFields().size()); - assertThat(rowData.getInt(0)).isEqualTo(address.getNum()); - assertThat(rowData.getString(1).toString()).isEqualTo(address.getStreet()); - if (schema != ADDRESS_SCHEMA_COMPATIBLE) { - assertThat(rowData.getString(2).toString()).isEqualTo(address.getCity()); - assertThat(rowData.getString(3).toString()).isEqualTo(address.getState()); - assertThat(rowData.getString(4).toString()).isEqualTo(address.getZip()); - } - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - private static AvroRowDataSerializationSchema getSerializationSchema( - RowType rowType, Schema avroSchema) { - ConfluentSchemaRegistryCoder registryCoder = - new ConfluentSchemaRegistryCoder(SUBJECT, client); - return new AvroRowDataSerializationSchema( - rowType, - new RegistryAvroSerializationSchema( - GenericRecord.class, avroSchema, () -> registryCoder), - RowDataToAvroConverters.createConverter(rowType)); - } - - private static AvroRowDataDeserializationSchema getDeserializationSchema( - RowType rowType, Schema avroSchema) { - ConfluentSchemaRegistryCoder registryCoder = - new ConfluentSchemaRegistryCoder(SUBJECT, client); - return new AvroRowDataDeserializationSchema( - new RegistryAvroDeserializationSchema( - GenericRecord.class, avroSchema, () -> registryCoder), - AvroToRowDataConverters.createRowConverter(rowType), - InternalTypeInfo.of(rowType)); - } - - private static RowData address2RowData(Address address) { - GenericRowData rowData = new GenericRowData(5); - rowData.setField(0, address.getNum()); - rowData.setField(1, new BinaryStringData(address.getStreet().toString())); - rowData.setField(2, new BinaryStringData(address.getCity().toString())); - rowData.setField(3, new BinaryStringData(address.getState().toString())); - rowData.setField(4, new BinaryStringData(address.getZip().toString())); - return rowData; - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java deleted file mode 100644 index b39cc1e28..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactoryTest.java +++ /dev/null @@ -1,249 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent.debezium; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.TestDynamicTableFactory; -import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; -import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.logical.RowType; - -import org.jetbrains.annotations.NotNull; -import org.junit.jupiter.api.Test; - -import java.util.HashMap; -import java.util.Map; - -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; -import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - -/** Tests for {@link DebeziumAvroFormatFactory}. */ -class DebeziumAvroFormatFactoryTest { - - private static final ResolvedSchema SCHEMA = - ResolvedSchema.of( - Column.physical("a", DataTypes.STRING()), - Column.physical("b", DataTypes.INT()), - Column.physical("c", DataTypes.BOOLEAN())); - - private static final String RECORD_SCHEMA = - " {\n" - + " \"type\": \"record\",\n" - + " \"name\": \"test\",\n" - + " \"fields\": [\n" - + " {\n" - + " \"name\": \"before\",\n" - + " \"type\": [\n" - + " \"null\",\n" - + " {\n" - + " \"type\": \"record\",\n" - + " \"name\": \"testSchema\",\n" - + " \"fields\": [\n" - + " {\n" - + " \"name\": \"a\",\n" - + " \"type\": [\n" - + " \"null\",\n" - + " \"string\"\n" - + " ],\n" - + " \"default\": null\n" - + " },\n" - + " {\n" - + " \"name\": \"b\",\n" - + " \"type\": [\n" - + " \"null\",\n" - + " \"int\"\n" - + " ],\n" - + " \"default\": null\n" - + " },\n" - + " {\n" - + " \"name\": \"c\",\n" - + " \"type\": [\n" - + " \"null\",\n" - + " \"boolean\"\n" - + " ],\n" - + " \"default\": null\n" - + " }\n" - + " ]\n" - + " }\n" - + " ],\n" - + " \"default\": null\n" - + " },\n" - + " {\n" - + " \"name\": \"after\",\n" - + " \"type\": [\n" - + " \"null\",\n" - + " \"testSchema\"\n" - + " ],\n" - + " \"default\": null\n" - + " },\n" - + " {\n" - + " \"name\": \"op\",\n" - + " \"type\": [\n" - + " \"null\",\n" - + " \"string\"\n" - + " ],\n" - + " \"default\": null\n" - + " }\n" - + " ]\n" - + " }\n"; - private static final String AVRO_SCHEMA = "[\n\"null\",\n" + RECORD_SCHEMA + "]"; - - private static final RowType ROW_TYPE = - (RowType) SCHEMA.toPhysicalRowDataType().getLogicalType(); - - private static final String SUBJECT = "test-debezium-avro"; - private static final String REGISTRY_URL = "http://localhost:8081"; - - @Test - void testSeDeSchema() { - final Map options = getAllOptions(); - final Map registryConfigs = getRegistryConfigs(); - - DebeziumAvroDeserializationSchema expectedDeser = - new DebeziumAvroDeserializationSchema( - ROW_TYPE, - InternalTypeInfo.of(ROW_TYPE), - REGISTRY_URL, - null, - registryConfigs); - DeserializationSchema actualDeser = createDeserializationSchema(options); - assertEquals(expectedDeser, actualDeser); - - DebeziumAvroSerializationSchema expectedSer = - new DebeziumAvroSerializationSchema( - ROW_TYPE, REGISTRY_URL, SUBJECT, null, registryConfigs); - SerializationSchema actualSer = createSerializationSchema(options); - assertEquals(expectedSer, actualSer); - } - - @Test - public void testSeDeSchemaWithSchemaOption() { - final Map options = getAllOptions(); - options.put("debezium-avro-confluent.schema", AVRO_SCHEMA); - - final Map registryConfigs = getRegistryConfigs(); - - DebeziumAvroDeserializationSchema expectedDeser = - new DebeziumAvroDeserializationSchema( - ROW_TYPE, - InternalTypeInfo.of(ROW_TYPE), - REGISTRY_URL, - AVRO_SCHEMA, - registryConfigs); - DeserializationSchema actualDeser = createDeserializationSchema(options); - assertThat(actualDeser).isEqualTo(expectedDeser); - - DebeziumAvroSerializationSchema expectedSer = - new DebeziumAvroSerializationSchema( - ROW_TYPE, REGISTRY_URL, SUBJECT, AVRO_SCHEMA, registryConfigs); - SerializationSchema actualSer = createSerializationSchema(options); - assertThat(actualSer).isEqualTo(expectedSer); - } - - @Test - public void testSeDeSchemaWithInvalidSchemaOption() { - final Map options = getAllOptions(); - options.put("debezium-avro-confluent.schema", RECORD_SCHEMA); - - assertThrows(IllegalArgumentException.class, () -> createDeserializationSchema(options)); - assertThrows(IllegalArgumentException.class, () -> createSerializationSchema(options)); - - String basicSchema = "[ \"null\", \"string\" ]"; - options.put("debezium-avro-confluent.schema", basicSchema); - assertThrows(IllegalArgumentException.class, () -> createDeserializationSchema(options)); - assertThrows(IllegalArgumentException.class, () -> createSerializationSchema(options)); - - String invalidSchema = - "[\"null\", " - + "{ \"type\" : \"record\"," - + "\"name\" : \"debezium\"," - + "\"fields\": [{\n" - + " \"default\": null,\n" - + " \"name\": \"op\",\n" - + " \"type\": [\n" - + " \"null\",\n" - + " \"string\"\n" - + " ]\n" - + " }]" - + "}]"; - options.put("debezium-avro-confluent.schema", invalidSchema); - assertThrows(IllegalArgumentException.class, () -> createDeserializationSchema(options)); - assertThrows(IllegalArgumentException.class, () -> createSerializationSchema(options)); - - String invalidRecordSchema = AVRO_SCHEMA.replace("int", "string"); - options.put("debezium-avro-confluent.schema", invalidRecordSchema); - assertThrows(IllegalArgumentException.class, () -> createDeserializationSchema(options)); - assertThrows(IllegalArgumentException.class, () -> createSerializationSchema(options)); - } - - @NotNull - private Map getRegistryConfigs() { - final Map registryConfigs = new HashMap<>(); - registryConfigs.put("basic.auth.user.info", "something1"); - registryConfigs.put("basic.auth.credentials.source", "something2"); - return registryConfigs; - } - - private Map getAllOptions() { - final Map options = new HashMap<>(); - options.put("connector", TestDynamicTableFactory.IDENTIFIER); - options.put("target", "MyTarget"); - options.put("buffer-size", "1000"); - - options.put("format", DebeziumAvroFormatFactory.IDENTIFIER); - options.put("debezium-avro-confluent.url", REGISTRY_URL); - options.put("debezium-avro-confluent.subject", SUBJECT); - options.put("debezium-avro-confluent.basic-auth.user-info", "something1"); - options.put("debezium-avro-confluent.basic-auth.credentials-source", "something2"); - return options; - } - - private static DeserializationSchema createDeserializationSchema( - Map options) { - final DynamicTableSource actualSource = createTableSource(SCHEMA, options); - assertThat(actualSource).isInstanceOf(TestDynamicTableFactory.DynamicTableSourceMock.class); - TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock = - (TestDynamicTableFactory.DynamicTableSourceMock) actualSource; - - return scanSourceMock.valueFormat.createRuntimeDecoder( - ScanRuntimeProviderContext.INSTANCE, SCHEMA.toPhysicalRowDataType()); - } - - private static SerializationSchema createSerializationSchema( - Map options) { - final DynamicTableSink actualSink = createTableSink(SCHEMA, options); - assertThat(actualSink).isInstanceOf(TestDynamicTableFactory.DynamicTableSinkMock.class); - TestDynamicTableFactory.DynamicTableSinkMock sinkMock = - (TestDynamicTableFactory.DynamicTableSinkMock) actualSink; - - return sinkMock.valueFormat.createRuntimeEncoder( - new SinkRuntimeProviderContext(false), SCHEMA.toPhysicalRowDataType()); - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java b/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java deleted file mode 100644 index 2640b50e4..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroSerDeSchemaTest.java +++ /dev/null @@ -1,242 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.avro.registry.confluent.debezium; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; -import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; -import org.apache.flink.formats.avro.AvroToRowDataConverters; -import org.apache.flink.formats.avro.RegistryAvroDeserializationSchema; -import org.apache.flink.formats.avro.RegistryAvroSerializationSchema; -import org.apache.flink.formats.avro.RowDataToAvroConverters; -import org.apache.flink.formats.avro.registry.confluent.ConfluentSchemaRegistryCoder; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.util.Collector; -import org.apache.flink.util.FileUtils; -import org.apache.flink.util.SimpleUserCodeClassLoader; -import org.apache.flink.util.UserCodeClassLoader; - -import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; -import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.flink.table.api.DataTypes.BIGINT; -import static org.apache.flink.table.api.DataTypes.DOUBLE; -import static org.apache.flink.table.api.DataTypes.FIELD; -import static org.apache.flink.table.api.DataTypes.ROW; -import static org.apache.flink.table.api.DataTypes.STRING; -import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link DebeziumAvroDeserializationSchema}. */ -class DebeziumAvroSerDeSchemaTest { - - private static final String SUBJECT = "testDebeziumAvro"; - - private static final RowType rowType = - (RowType) - ROW( - FIELD("id", BIGINT()), - FIELD("name", STRING()), - FIELD("description", STRING()), - FIELD("weight", DOUBLE())) - .getLogicalType(); - - private static final Schema DEBEZIUM_SCHEMA_COMPATIBLE_TEST = - new Schema.Parser().parse(new String(readBytesFromFile("debezium-test-schema.json"))); - - private final SchemaRegistryClient client = new MockSchemaRegistryClient(); - - @Test - void testSerializationDeserialization() throws Exception { - - RowType rowTypeDe = - DebeziumAvroDeserializationSchema.createDebeziumAvroRowType( - fromLogicalToDataType(rowType)); - RowType rowTypeSe = - DebeziumAvroSerializationSchema.createDebeziumAvroRowType( - fromLogicalToDataType(rowType)); - - DebeziumAvroSerializationSchema dbzSerializer = - new DebeziumAvroSerializationSchema(getSerializationSchema(rowTypeSe)); - dbzSerializer.open(new MockInitializationContext()); - - byte[] serialize = dbzSerializer.serialize(debeziumRow2RowData()); - - client.register(SUBJECT, DEBEZIUM_SCHEMA_COMPATIBLE_TEST); - DebeziumAvroDeserializationSchema dbzDeserializer = - new DebeziumAvroDeserializationSchema( - InternalTypeInfo.of(rowType), getDeserializationSchema(rowTypeDe)); - dbzDeserializer.open(new MockInitializationContext()); - - SimpleCollector collector = new SimpleCollector(); - dbzDeserializer.deserialize(serialize, collector); - - List actual = - collector.list.stream().map(Object::toString).collect(Collectors.toList()); - - List expected = - Collections.singletonList("+I(107,rocks,box of assorted rocks,5.3)"); - assertThat(actual).isEqualTo(expected); - } - - @Test - void testInsertDataDeserialization() throws Exception { - List actual = testDeserialization("debezium-avro-insert.avro"); - - List expected = - Collections.singletonList("+I(1,lisi,test debezium avro data,21.799999237060547)"); - assertThat(actual).isEqualTo(expected); - } - - @Test - void testUpdateDataDeserialization() throws Exception { - List actual = testDeserialization("debezium-avro-update.avro"); - - List expected = - Arrays.asList( - "-U(1,lisi,test debezium avro data,21.799999237060547)", - "+U(1,zhangsan,test debezium avro data,21.799999237060547)"); - assertThat(actual).isEqualTo(expected); - } - - @Test - void testDeleteDataDeserialization() throws Exception { - List actual = testDeserialization("debezium-avro-delete.avro"); - - List expected = - Collections.singletonList( - "-D(1,zhangsan,test debezium avro data,21.799999237060547)"); - assertThat(actual).isEqualTo(expected); - } - - public List testDeserialization(String dataPath) throws Exception { - RowType rowTypeDe = - DebeziumAvroDeserializationSchema.createDebeziumAvroRowType( - fromLogicalToDataType(rowType)); - - client.register(SUBJECT, DEBEZIUM_SCHEMA_COMPATIBLE_TEST, 1, 81); - - DebeziumAvroDeserializationSchema dbzDeserializer = - new DebeziumAvroDeserializationSchema( - InternalTypeInfo.of(rowType), getDeserializationSchema(rowTypeDe)); - dbzDeserializer.open(new MockInitializationContext()); - - SimpleCollector collector = new SimpleCollector(); - dbzDeserializer.deserialize(readBytesFromFile(dataPath), collector); - - return collector.list.stream().map(Object::toString).collect(Collectors.toList()); - } - - private AvroRowDataDeserializationSchema getDeserializationSchema(RowType rowType) { - - final ConfluentSchemaRegistryCoder registryCoder = - new ConfluentSchemaRegistryCoder(SUBJECT, client); - - return new AvroRowDataDeserializationSchema( - new RegistryAvroDeserializationSchema<>( - GenericRecord.class, - AvroSchemaConverter.convertToSchema(rowType), - () -> registryCoder), - AvroToRowDataConverters.createRowConverter(rowType), - InternalTypeInfo.of(rowType)); - } - - private AvroRowDataSerializationSchema getSerializationSchema(RowType rowType) { - - ConfluentSchemaRegistryCoder registryCoder = - new ConfluentSchemaRegistryCoder(SUBJECT, client); - return new AvroRowDataSerializationSchema( - rowType, - new RegistryAvroSerializationSchema<>( - GenericRecord.class, - AvroSchemaConverter.convertToSchema(rowType), - () -> registryCoder), - RowDataToAvroConverters.createConverter(rowType)); - } - - private static RowData debeziumRow2RowData() { - GenericRowData rowData = new GenericRowData(4); - rowData.setField(0, 107L); - rowData.setField(1, StringData.fromString("rocks")); - rowData.setField(2, StringData.fromString("box of assorted rocks")); - rowData.setField(3, 5.3D); - return rowData; - } - - private static byte[] readBytesFromFile(String filePath) { - try { - URL url = DebeziumAvroSerDeSchemaTest.class.getClassLoader().getResource(filePath); - assertThat(url).isNotNull(); - Path path = new File(url.getFile()).toPath(); - return FileUtils.readAllBytes(path); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - private static class SimpleCollector implements Collector { - - private final List list = new ArrayList<>(); - - @Override - public void collect(RowData record) { - list.add(record); - } - - @Override - public void close() { - // do nothing - } - } - - private static class MockInitializationContext - implements DeserializationSchema.InitializationContext, - SerializationSchema.InitializationContext { - @Override - public MetricGroup getMetricGroup() { - return new UnregisteredMetricsGroup(); - } - - @Override - public UserCodeClassLoader getUserCodeClassLoader() { - return SimpleUserCodeClassLoader.create(getClass().getClassLoader()); - } - } -} diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension deleted file mode 100644 index 28999133c..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -org.apache.flink.util.TestLoggerExtension \ No newline at end of file diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/archunit.properties b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/archunit.properties deleted file mode 100644 index 15be88c95..000000000 --- a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/archunit.properties +++ /dev/null @@ -1,31 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# By default we allow removing existing violations, but fail when new violations are added. -freeze.store.default.allowStoreUpdate=true - -# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. -#freeze.store.default.allowStoreCreation=true - -# Enable this to add allow new violations to be recorded. -# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new -# violation, please try to avoid creating the violation. If the violation was created due to a -# shortcoming of the rule, file a JIRA issue so the rule can be improved. -#freeze.refreeze=true - -freeze.store.default.path=archunit-violations diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-delete.avro b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-delete.avro deleted file mode 100644 index 84e3ba6586792feb46d12704b1e3104e7043f5c3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 147 zcmZQzU|TEHO-ilGEX`F&EGx=aNJ%V7WCE%@aORAu z1A~~Mo)HkbW#%R3aOGAO7v@N$mFDE6W#;7MrskC_SpM~XUn~PxT4GLdDi_E!LnfYr l)S}}2JSGMOuzKC3%sf2<5HLM6O^Sgjfq^N7Y1c-GJ^*2#FjD{k diff --git a/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-insert.avro b/flink-formats-kafka/flink-avro-confluent-registry/src/test/resources/debezium-avro-insert.avro deleted file mode 100644 index 342f2da150dec5b31738a04707775a58c0ae30ca..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 143 zcmZQzU|lJ8TEHO-ilGEX`F&EGx=aNJ%V7WCE%>aORAu15l+v zRYqc7dU0YNVbx-WdPYF#mYJ8B!7nPB=*WJt|9k)}ivqF|jJ4Y@}?r{#K zk`Z&u%G@s@LJ~=;k+s|}ulIf4_nhba@q2#H^ZEXM-@m?J0!vCB1OyXU@J^VNR{V+h zB^lrjU>*xz4`IP;wlR;u0!#fv0xQ2WZ)W&dcPxu)mqFwrrrO$Iq#RpyT|Irl$=ZUgEQZ7qW-+lpQJ3D zeHH3^E$_%zqh0Aw`|Ed)`W>xsHj;#tK~{@|H_1=fAMERz{2zj|H#1O&aQ7x<9du8)Fr_-*z&$voFt`SH& z+@9lmm%{rd`;13og_V}o3ADmxB2Vr=8?g}ZdCA&K-f$u29jq62NQgDi&Whpo+I_Z; z3Hp@yvN*F5vhp-Q?_~N;TC~7>jX6HIux@#?6E-2b^@L~B7U8SpbII#`k1X230~WaP z(p4F2<#OF@30C2;=JRk@qBT!vC6&q~SOD|S_xAw?n;PQJj7lrk+jEMcb zzp_La)A}frEqI2`i40Oe7yL?lG=O!c>NHGC2~|Ps{iS(>w4=yQ?hi?38He)~Om}k_ zq^$F0n1a+HCB^DfLyj-@d$bSKA`vmV38&CsdOl&>{eXVAp2{nkNPL2!w|P6+S1$9v zY`(H8-5YD>uS!Db z5iNeWTFA1tT-QD+a)7o)^FR|Vq&E8RDQv5s`H;uba{IGuL_TU>l8sU%OH)bz?)= z8F%LuotIlAr`QcYW+kTXlNuA}sz(~uO;!5V%2d2@ZrXA~m-rZ#+}ECrko7I2GuK{?YTuCNaUT zu}1aN7d+Okj}?5iQorYp(|6=p>94CqRD`!r!tj#X(CY@=gIs`NUcw(1$tg^MTG+=9 zW3#Fl0>eJdE<0ZFg)a*~R><4aXkV)ikuPs~vxW5IVh1`0EZ2ns(a8{Av71A;W7jQJVbqgJ9s3*Zb;37q=cd)i^c!3iP#1h?rFvZ{7d^$j#_ zLM|-et)Jbk&T_z5t@rirhw0@z3)41lIl;qRj-*wlg37+AR)5;CcNIro(h@3{cD2zP zn<6pdBFE2ReRfYN_igbS8mw}K-)X>i>hilDVx5-wK20sL z^BsmiRbzft8vHr@K23WhaCj9&r+O(bRv66t>yE<&f1ZHk${qRMYzD}@2@i5-qv}Od zYwO5WPfRM`jiN;%jz#FxYSVpF4El^gFiN#uC+MT|Zt4cOu}a&s^17HhNDn@DjOczX z4vAud3}>txkbH;CIOBa%sWuX_`e?CH0uSJ@=k1HPz17VY!58W9rdrz8&kJ6i`pUm0 zH`Hgg5$@#97%!I|GH&G`&EFllH)RDkYb#uGDQg5{zjX(rCptgb?+Gd$mstUthWr8W z_b&;?Am?+=_{v(lnZ3aQd%0xAnT9f362H~D5NqOQGndmcseuc7FrCvMr=-```qDqP zce)kgv6aXv2^(E;qe4tWSCJPTLRWYcov`G0{NC@kofx!tS80fXMzJ2rvvsgY=Ynih zsN0W9ilOV&z0Tw&1Y_WZ06(kxQ*JsQ=`%G{`hIC9{StnfP%a#qNho*L{E{vGK?x69 zJiZ*ojXiiybpGqAtJFo+V-QD;)&?j5_3iAA!F&B#-k)C-XPK8+?kQR`sHLZ0iUr1o zt6V$I1R;|!LvisR1$J3;IBcK-ZW-VpeB3VqlIgA&xiW->QG^Hi=jRWF>Zm+gYHr4( znz0IFB~<|>!~fv0$I<2Jj7_WG+}+06hrC;6k2C5^$F2Qz6+jfp#`NYeWL`J_0M2tW zzeB(41H)_~x3)?QyJzc~hV?o@ygEmXIvKxTuk*n0^TK>(*PVGIn3VOTspT{mQg3So zLMAYDBi==zw={8BPt-%%`Ccg;qK+vEkwqz+1|e^iiNbkE*^v{4TxY4vK_a(_AFk`p zz4C!+cr}1-U``Fzg#AM+8>GD8olYPV_7S9DP;C?#qznZ>&=@_@Pc;ZleNTnf2U*1h k-4yEm76{rle`cxUCE~`C_PiC*|C`ynX6Pyi42;kFH{XCk00000 diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml b/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml deleted file mode 100644 index 464affef8..000000000 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/pom.xml +++ /dev/null @@ -1,120 +0,0 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-formats-kafka - 3.1-SNAPSHOT - - - flink-sql-avro-confluent-registry - Flink : Formats : SQL Avro Confluent Registry - - jar - - - true - - - - - confluent - https://packages.confluent.io/maven/ - - - - - - org.apache.flink - flink-avro-confluent-registry - ${project.version} - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - *:* - - - com.google.code.findbugs:jsr305 - - - - - org.apache.kafka - org.apache.flink.avro.registry.confluent.shaded.org.apache.kafka - - - io.confluent - org.apache.flink.avro.registry.confluent.shaded.io.confluent - - - - com.fasterxml.jackson - org.apache.flink.avro.shaded.com.fasterxml.jackson - - - org.apache.avro - org.apache.flink.avro.shaded.org.apache.avro - - - org.apache.commons.compress - org.apache.flink.avro.shaded.org.apache.commons.compress - - - - - org.apache.kafka:* - - kafka/kafka-version.properties - LICENSE - - NOTICE - common/** - - - - - - - - - - diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE deleted file mode 100644 index 474ed028c..000000000 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ /dev/null @@ -1,32 +0,0 @@ -flink-sql-avro-confluent-registry -Copyright 2014-2023 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - -- com.fasterxml.jackson.core:jackson-annotations:2.13.4 -- com.fasterxml.jackson.core:jackson-core:2.13.4 -- com.fasterxml.jackson.core:jackson-databind:2.13.4.2 -- com.google.guava:guava:30.1.1-jre -- io.confluent:common-config:7.2.2 -- io.confluent:common-utils:7.2.2 -- io.confluent:kafka-schema-registry-client:7.2.2 -- org.apache.avro:avro:1.11.1 -- org.apache.commons:commons-compress:1.21 -- org.apache.kafka:kafka-clients:3.4.0 -- org.glassfish.jersey.core:jersey-common:2.30 -- org.xerial.snappy:snappy-java:1.1.8.3 - -The binary distribution of this product bundles these dependencies under the Eclipse Public License - v 2.0 (https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt) - -- jakarta.annotation:jakarta.annotation-api:1.3.5 -- jakarta.ws.rs:jakarta.ws.rs-api:2.1.6 -- org.glassfish.hk2.external:jakarta.inject:2.6.1 -- org.glassfish.hk2:osgi-resource-locator:1.0.3 - -This project bundles the following dependencies under the BSD license. -See bundled license files for details. - -- com.github.luben:zstd-jni:1.5.2-1 diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-annotation-api b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-annotation-api deleted file mode 100644 index 5de3d1b40..000000000 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-annotation-api +++ /dev/null @@ -1,637 +0,0 @@ -# Eclipse Public License - v 2.0 - - THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE - PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION - OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. - - 1. DEFINITIONS - - "Contribution" means: - - a) in the case of the initial Contributor, the initial content - Distributed under this Agreement, and - - b) in the case of each subsequent Contributor: - i) changes to the Program, and - ii) additions to the Program; - where such changes and/or additions to the Program originate from - and are Distributed by that particular Contributor. A Contribution - "originates" from a Contributor if it was added to the Program by - such Contributor itself or anyone acting on such Contributor's behalf. - Contributions do not include changes or additions to the Program that - are not Modified Works. - - "Contributor" means any person or entity that Distributes the Program. - - "Licensed Patents" mean patent claims licensable by a Contributor which - are necessarily infringed by the use or sale of its Contribution alone - or when combined with the Program. - - "Program" means the Contributions Distributed in accordance with this - Agreement. - - "Recipient" means anyone who receives the Program under this Agreement - or any Secondary License (as applicable), including Contributors. - - "Derivative Works" shall mean any work, whether in Source Code or other - form, that is based on (or derived from) the Program and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. - - "Modified Works" shall mean any work in Source Code or other form that - results from an addition to, deletion from, or modification of the - contents of the Program, including, for purposes of clarity any new file - in Source Code form that contains any contents of the Program. Modified - Works shall not include works that contain only declarations, - interfaces, types, classes, structures, or files of the Program solely - in each case in order to link to, bind by name, or subclass the Program - or Modified Works thereof. - - "Distribute" means the acts of a) distributing or b) making available - in any manner that enables the transfer of a copy. - - "Source Code" means the form of a Program preferred for making - modifications, including but not limited to software source code, - documentation source, and configuration files. - - "Secondary License" means either the GNU General Public License, - Version 2.0, or any later versions of that license, including any - exceptions or additional permissions as identified by the initial - Contributor. - - 2. GRANT OF RIGHTS - - a) Subject to the terms of this Agreement, each Contributor hereby - grants Recipient a non-exclusive, worldwide, royalty-free copyright - license to reproduce, prepare Derivative Works of, publicly display, - publicly perform, Distribute and sublicense the Contribution of such - Contributor, if any, and such Derivative Works. - - b) Subject to the terms of this Agreement, each Contributor hereby - grants Recipient a non-exclusive, worldwide, royalty-free patent - license under Licensed Patents to make, use, sell, offer to sell, - import and otherwise transfer the Contribution of such Contributor, - if any, in Source Code or other form. This patent license shall - apply to the combination of the Contribution and the Program if, at - the time the Contribution is added by the Contributor, such addition - of the Contribution causes such combination to be covered by the - Licensed Patents. The patent license shall not apply to any other - combinations which include the Contribution. No hardware per se is - licensed hereunder. - - c) Recipient understands that although each Contributor grants the - licenses to its Contributions set forth herein, no assurances are - provided by any Contributor that the Program does not infringe the - patent or other intellectual property rights of any other entity. - Each Contributor disclaims any liability to Recipient for claims - brought by any other entity based on infringement of intellectual - property rights or otherwise. As a condition to exercising the - rights and licenses granted hereunder, each Recipient hereby - assumes sole responsibility to secure any other intellectual - property rights needed, if any. For example, if a third party - patent license is required to allow Recipient to Distribute the - Program, it is Recipient's responsibility to acquire that license - before distributing the Program. - - d) Each Contributor represents that to its knowledge it has - sufficient copyright rights in its Contribution, if any, to grant - the copyright license set forth in this Agreement. - - e) Notwithstanding the terms of any Secondary License, no - Contributor makes additional grants to any Recipient (other than - those set forth in this Agreement) as a result of such Recipient's - receipt of the Program under the terms of a Secondary License - (if permitted under the terms of Section 3). - - 3. REQUIREMENTS - - 3.1 If a Contributor Distributes the Program in any form, then: - - a) the Program must also be made available as Source Code, in - accordance with section 3.2, and the Contributor must accompany - the Program with a statement that the Source Code for the Program - is available under this Agreement, and informs Recipients how to - obtain it in a reasonable manner on or through a medium customarily - used for software exchange; and - - b) the Contributor may Distribute the Program under a license - different than this Agreement, provided that such license: - i) effectively disclaims on behalf of all other Contributors all - warranties and conditions, express and implied, including - warranties or conditions of title and non-infringement, and - implied warranties or conditions of merchantability and fitness - for a particular purpose; - - ii) effectively excludes on behalf of all other Contributors all - liability for damages, including direct, indirect, special, - incidental and consequential damages, such as lost profits; - - iii) does not attempt to limit or alter the recipients' rights - in the Source Code under section 3.2; and - - iv) requires any subsequent distribution of the Program by any - party to be under a license that satisfies the requirements - of this section 3. - - 3.2 When the Program is Distributed as Source Code: - - a) it must be made available under this Agreement, or if the - Program (i) is combined with other material in a separate file or - files made available under a Secondary License, and (ii) the initial - Contributor attached to the Source Code the notice described in - Exhibit A of this Agreement, then the Program may be made available - under the terms of such Secondary Licenses, and - - b) a copy of this Agreement must be included with each copy of - the Program. - - 3.3 Contributors may not remove or alter any copyright, patent, - trademark, attribution notices, disclaimers of warranty, or limitations - of liability ("notices") contained within the Program from any copy of - the Program which they Distribute, provided that Contributors may add - their own appropriate notices. - - 4. COMMERCIAL DISTRIBUTION - - Commercial distributors of software may accept certain responsibilities - with respect to end users, business partners and the like. While this - license is intended to facilitate the commercial use of the Program, - the Contributor who includes the Program in a commercial product - offering should do so in a manner which does not create potential - liability for other Contributors. Therefore, if a Contributor includes - the Program in a commercial product offering, such Contributor - ("Commercial Contributor") hereby agrees to defend and indemnify every - other Contributor ("Indemnified Contributor") against any losses, - damages and costs (collectively "Losses") arising from claims, lawsuits - and other legal actions brought by a third party against the Indemnified - Contributor to the extent caused by the acts or omissions of such - Commercial Contributor in connection with its distribution of the Program - in a commercial product offering. The obligations in this section do not - apply to any claims or Losses relating to any actual or alleged - intellectual property infringement. In order to qualify, an Indemnified - Contributor must: a) promptly notify the Commercial Contributor in - writing of such claim, and b) allow the Commercial Contributor to control, - and cooperate with the Commercial Contributor in, the defense and any - related settlement negotiations. The Indemnified Contributor may - participate in any such claim at its own expense. - - For example, a Contributor might include the Program in a commercial - product offering, Product X. That Contributor is then a Commercial - Contributor. If that Commercial Contributor then makes performance - claims, or offers warranties related to Product X, those performance - claims and warranties are such Commercial Contributor's responsibility - alone. Under this section, the Commercial Contributor would have to - defend claims against the other Contributors related to those performance - claims and warranties, and if a court requires any other Contributor to - pay any damages as a result, the Commercial Contributor must pay - those damages. - - 5. NO WARRANTY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT - PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED ON AN "AS IS" - BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR - IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF - TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR - PURPOSE. Each Recipient is solely responsible for determining the - appropriateness of using and distributing the Program and assumes all - risks associated with its exercise of rights under this Agreement, - including but not limited to the risks and costs of program errors, - compliance with applicable laws, damage to or loss of data, programs - or equipment, and unavailability or interruption of operations. - - 6. DISCLAIMER OF LIABILITY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT - PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS - SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, - EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST - PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE - EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE - POSSIBILITY OF SUCH DAMAGES. - - 7. GENERAL - - If any provision of this Agreement is invalid or unenforceable under - applicable law, it shall not affect the validity or enforceability of - the remainder of the terms of this Agreement, and without further - action by the parties hereto, such provision shall be reformed to the - minimum extent necessary to make such provision valid and enforceable. - - If Recipient institutes patent litigation against any entity - (including a cross-claim or counterclaim in a lawsuit) alleging that the - Program itself (excluding combinations of the Program with other software - or hardware) infringes such Recipient's patent(s), then such Recipient's - rights granted under Section 2(b) shall terminate as of the date such - litigation is filed. - - All Recipient's rights under this Agreement shall terminate if it - fails to comply with any of the material terms or conditions of this - Agreement and does not cure such failure in a reasonable period of - time after becoming aware of such noncompliance. If all Recipient's - rights under this Agreement terminate, Recipient agrees to cease use - and distribution of the Program as soon as reasonably practicable. - However, Recipient's obligations under this Agreement and any licenses - granted by Recipient relating to the Program shall continue and survive. - - Everyone is permitted to copy and distribute copies of this Agreement, - but in order to avoid inconsistency the Agreement is copyrighted and - may only be modified in the following manner. The Agreement Steward - reserves the right to publish new versions (including revisions) of - this Agreement from time to time. No one other than the Agreement - Steward has the right to modify this Agreement. The Eclipse Foundation - is the initial Agreement Steward. The Eclipse Foundation may assign the - responsibility to serve as the Agreement Steward to a suitable separate - entity. Each new version of the Agreement will be given a distinguishing - version number. The Program (including Contributions) may always be - Distributed subject to the version of the Agreement under which it was - received. In addition, after a new version of the Agreement is published, - Contributor may elect to Distribute the Program (including its - Contributions) under the new version. - - Except as expressly stated in Sections 2(a) and 2(b) above, Recipient - receives no rights or licenses to the intellectual property of any - Contributor under this Agreement, whether expressly, by implication, - estoppel or otherwise. All rights in the Program not expressly granted - under this Agreement are reserved. Nothing in this Agreement is intended - to be enforceable by any entity that is not a Contributor or Recipient. - No third-party beneficiary rights are created under this Agreement. - - Exhibit A - Form of Secondary Licenses Notice - - "This Source Code may also be made available under the following - Secondary Licenses when the conditions for such availability set forth - in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), - version(s), and exceptions or additional permissions here}." - - Simply including a copy of this Agreement, including this Exhibit A - is not sufficient to license the Source Code under Secondary Licenses. - - If it is not possible or desirable to put the notice in a particular - file, then You may include the notice in a location (such as a LICENSE - file in a relevant directory) where a recipient would be likely to - look for such a notice. - - You may add additional accurate notices of copyright ownership. - ---- - -## The GNU General Public License (GPL) Version 2, June 1991 - - Copyright (C) 1989, 1991 Free Software Foundation, Inc. - 51 Franklin Street, Fifth Floor - Boston, MA 02110-1335 - USA - - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - - Preamble - - The licenses for most software are designed to take away your freedom to - share and change it. By contrast, the GNU General Public License is - intended to guarantee your freedom to share and change free software--to - make sure the software is free for all its users. This General Public - License applies to most of the Free Software Foundation's software and - to any other program whose authors commit to using it. (Some other Free - Software Foundation software is covered by the GNU Library General - Public License instead.) You can apply it to your programs, too. - - When we speak of free software, we are referring to freedom, not price. - Our General Public Licenses are designed to make sure that you have the - freedom to distribute copies of free software (and charge for this - service if you wish), that you receive source code or can get it if you - want it, that you can change the software or use pieces of it in new - free programs; and that you know you can do these things. - - To protect your rights, we need to make restrictions that forbid anyone - to deny you these rights or to ask you to surrender the rights. These - restrictions translate to certain responsibilities for you if you - distribute copies of the software, or if you modify it. - - For example, if you distribute copies of such a program, whether gratis - or for a fee, you must give the recipients all the rights that you have. - You must make sure that they, too, receive or can get the source code. - And you must show them these terms so they know their rights. - - We protect your rights with two steps: (1) copyright the software, and - (2) offer you this license which gives you legal permission to copy, - distribute and/or modify the software. - - Also, for each author's protection and ours, we want to make certain - that everyone understands that there is no warranty for this free - software. If the software is modified by someone else and passed on, we - want its recipients to know that what they have is not the original, so - that any problems introduced by others will not reflect on the original - authors' reputations. - - Finally, any free program is threatened constantly by software patents. - We wish to avoid the danger that redistributors of a free program will - individually obtain patent licenses, in effect making the program - proprietary. To prevent this, we have made it clear that any patent must - be licensed for everyone's free use or not licensed at all. - - The precise terms and conditions for copying, distribution and - modification follow. - - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. This License applies to any program or other work which contains a - notice placed by the copyright holder saying it may be distributed under - the terms of this General Public License. The "Program", below, refers - to any such program or work, and a "work based on the Program" means - either the Program or any derivative work under copyright law: that is - to say, a work containing the Program or a portion of it, either - verbatim or with modifications and/or translated into another language. - (Hereinafter, translation is included without limitation in the term - "modification".) Each licensee is addressed as "you". - - Activities other than copying, distribution and modification are not - covered by this License; they are outside its scope. The act of running - the Program is not restricted, and the output from the Program is - covered only if its contents constitute a work based on the Program - (independent of having been made by running the Program). Whether that - is true depends on what the Program does. - - 1. You may copy and distribute verbatim copies of the Program's source - code as you receive it, in any medium, provided that you conspicuously - and appropriately publish on each copy an appropriate copyright notice - and disclaimer of warranty; keep intact all the notices that refer to - this License and to the absence of any warranty; and give any other - recipients of the Program a copy of this License along with the Program. - - You may charge a fee for the physical act of transferring a copy, and - you may at your option offer warranty protection in exchange for a fee. - - 2. You may modify your copy or copies of the Program or any portion of - it, thus forming a work based on the Program, and copy and distribute - such modifications or work under the terms of Section 1 above, provided - that you also meet all of these conditions: - - a) You must cause the modified files to carry prominent notices - stating that you changed the files and the date of any change. - - b) You must cause any work that you distribute or publish, that in - whole or in part contains or is derived from the Program or any part - thereof, to be licensed as a whole at no charge to all third parties - under the terms of this License. - - c) If the modified program normally reads commands interactively - when run, you must cause it, when started running for such - interactive use in the most ordinary way, to print or display an - announcement including an appropriate copyright notice and a notice - that there is no warranty (or else, saying that you provide a - warranty) and that users may redistribute the program under these - conditions, and telling the user how to view a copy of this License. - (Exception: if the Program itself is interactive but does not - normally print such an announcement, your work based on the Program - is not required to print an announcement.) - - These requirements apply to the modified work as a whole. If - identifiable sections of that work are not derived from the Program, and - can be reasonably considered independent and separate works in - themselves, then this License, and its terms, do not apply to those - sections when you distribute them as separate works. But when you - distribute the same sections as part of a whole which is a work based on - the Program, the distribution of the whole must be on the terms of this - License, whose permissions for other licensees extend to the entire - whole, and thus to each and every part regardless of who wrote it. - - Thus, it is not the intent of this section to claim rights or contest - your rights to work written entirely by you; rather, the intent is to - exercise the right to control the distribution of derivative or - collective works based on the Program. - - In addition, mere aggregation of another work not based on the Program - with the Program (or with a work based on the Program) on a volume of a - storage or distribution medium does not bring the other work under the - scope of this License. - - 3. You may copy and distribute the Program (or a work based on it, - under Section 2) in object code or executable form under the terms of - Sections 1 and 2 above provided that you also do one of the following: - - a) Accompany it with the complete corresponding machine-readable - source code, which must be distributed under the terms of Sections 1 - and 2 above on a medium customarily used for software interchange; or, - - b) Accompany it with a written offer, valid for at least three - years, to give any third party, for a charge no more than your cost - of physically performing source distribution, a complete - machine-readable copy of the corresponding source code, to be - distributed under the terms of Sections 1 and 2 above on a medium - customarily used for software interchange; or, - - c) Accompany it with the information you received as to the offer to - distribute corresponding source code. (This alternative is allowed - only for noncommercial distribution and only if you received the - program in object code or executable form with such an offer, in - accord with Subsection b above.) - - The source code for a work means the preferred form of the work for - making modifications to it. For an executable work, complete source code - means all the source code for all modules it contains, plus any - associated interface definition files, plus the scripts used to control - compilation and installation of the executable. However, as a special - exception, the source code distributed need not include anything that is - normally distributed (in either source or binary form) with the major - components (compiler, kernel, and so on) of the operating system on - which the executable runs, unless that component itself accompanies the - executable. - - If distribution of executable or object code is made by offering access - to copy from a designated place, then offering equivalent access to copy - the source code from the same place counts as distribution of the source - code, even though third parties are not compelled to copy the source - along with the object code. - - 4. You may not copy, modify, sublicense, or distribute the Program - except as expressly provided under this License. Any attempt otherwise - to copy, modify, sublicense or distribute the Program is void, and will - automatically terminate your rights under this License. However, parties - who have received copies, or rights, from you under this License will - not have their licenses terminated so long as such parties remain in - full compliance. - - 5. You are not required to accept this License, since you have not - signed it. However, nothing else grants you permission to modify or - distribute the Program or its derivative works. These actions are - prohibited by law if you do not accept this License. Therefore, by - modifying or distributing the Program (or any work based on the - Program), you indicate your acceptance of this License to do so, and all - its terms and conditions for copying, distributing or modifying the - Program or works based on it. - - 6. Each time you redistribute the Program (or any work based on the - Program), the recipient automatically receives a license from the - original licensor to copy, distribute or modify the Program subject to - these terms and conditions. You may not impose any further restrictions - on the recipients' exercise of the rights granted herein. You are not - responsible for enforcing compliance by third parties to this License. - - 7. If, as a consequence of a court judgment or allegation of patent - infringement or for any other reason (not limited to patent issues), - conditions are imposed on you (whether by court order, agreement or - otherwise) that contradict the conditions of this License, they do not - excuse you from the conditions of this License. If you cannot distribute - so as to satisfy simultaneously your obligations under this License and - any other pertinent obligations, then as a consequence you may not - distribute the Program at all. For example, if a patent license would - not permit royalty-free redistribution of the Program by all those who - receive copies directly or indirectly through you, then the only way you - could satisfy both it and this License would be to refrain entirely from - distribution of the Program. - - If any portion of this section is held invalid or unenforceable under - any particular circumstance, the balance of the section is intended to - apply and the section as a whole is intended to apply in other - circumstances. - - It is not the purpose of this section to induce you to infringe any - patents or other property right claims or to contest validity of any - such claims; this section has the sole purpose of protecting the - integrity of the free software distribution system, which is implemented - by public license practices. Many people have made generous - contributions to the wide range of software distributed through that - system in reliance on consistent application of that system; it is up to - the author/donor to decide if he or she is willing to distribute - software through any other system and a licensee cannot impose that choice. - - This section is intended to make thoroughly clear what is believed to be - a consequence of the rest of this License. - - 8. If the distribution and/or use of the Program is restricted in - certain countries either by patents or by copyrighted interfaces, the - original copyright holder who places the Program under this License may - add an explicit geographical distribution limitation excluding those - countries, so that distribution is permitted only in or among countries - not thus excluded. In such case, this License incorporates the - limitation as if written in the body of this License. - - 9. The Free Software Foundation may publish revised and/or new - versions of the General Public License from time to time. Such new - versions will be similar in spirit to the present version, but may - differ in detail to address new problems or concerns. - - Each version is given a distinguishing version number. If the Program - specifies a version number of this License which applies to it and "any - later version", you have the option of following the terms and - conditions either of that version or of any later version published by - the Free Software Foundation. If the Program does not specify a version - number of this License, you may choose any version ever published by the - Free Software Foundation. - - 10. If you wish to incorporate parts of the Program into other free - programs whose distribution conditions are different, write to the - author to ask for permission. For software which is copyrighted by the - Free Software Foundation, write to the Free Software Foundation; we - sometimes make exceptions for this. Our decision will be guided by the - two goals of preserving the free status of all derivatives of our free - software and of promoting the sharing and reuse of software generally. - - NO WARRANTY - - 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO - WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. - EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR - OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, - EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED - WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE - ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH - YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL - NECESSARY SERVICING, REPAIR OR CORRECTION. - - 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN - WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY - AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR - DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL - DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM - (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED - INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF - THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR - OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. - - END OF TERMS AND CONDITIONS - - How to Apply These Terms to Your New Programs - - If you develop a new program, and you want it to be of the greatest - possible use to the public, the best way to achieve this is to make it - free software which everyone can redistribute and change under these terms. - - To do so, attach the following notices to the program. It is safest to - attach them to the start of each source file to most effectively convey - the exclusion of warranty; and each file should have at least the - "copyright" line and a pointer to where the full notice is found. - - One line to give the program's name and a brief idea of what it does. - Copyright (C) - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, but - WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - General Public License for more details. - - You should have received a copy of the GNU General Public License - along with this program; if not, write to the Free Software - Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA - - Also add information on how to contact you by electronic and paper mail. - - If the program is interactive, make it output a short notice like this - when it starts in an interactive mode: - - Gnomovision version 69, Copyright (C) year name of author - Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type - `show w'. This is free software, and you are welcome to redistribute - it under certain conditions; type `show c' for details. - - The hypothetical commands `show w' and `show c' should show the - appropriate parts of the General Public License. Of course, the commands - you use may be called something other than `show w' and `show c'; they - could even be mouse-clicks or menu items--whatever suits your program. - - You should also get your employer (if you work as a programmer) or your - school, if any, to sign a "copyright disclaimer" for the program, if - necessary. Here is a sample; alter the names: - - Yoyodyne, Inc., hereby disclaims all copyright interest in the - program `Gnomovision' (which makes passes at compilers) written by - James Hacker. - - signature of Ty Coon, 1 April 1989 - Ty Coon, President of Vice - - This General Public License does not permit incorporating your program - into proprietary programs. If your program is a subroutine library, you - may consider it more useful to permit linking proprietary applications - with the library. If this is what you want to do, use the GNU Library - General Public License instead of this License. - ---- - -## CLASSPATH EXCEPTION - - Linking this library statically or dynamically with other modules is - making a combined work based on this library. Thus, the terms and - conditions of the GNU General Public License version 2 cover the whole - combination. - - As a special exception, the copyright holders of this library give you - permission to link this library with independent modules to produce an - executable, regardless of the license terms of these independent - modules, and to copy and distribute the resulting executable under - terms of your choice, provided that you also meet, for each linked - independent module, the terms and conditions of the license of that - module. An independent module is a module which is not derived from or - based on this library. If you modify this library, you may extend this - exception to your version of the library, but you are not obligated to - do so. If you do not wish to do so, delete this exception statement - from your version. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-inject b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-inject deleted file mode 100644 index 6667b0612..000000000 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-inject +++ /dev/null @@ -1,637 +0,0 @@ -# Eclipse Public License - v 2.0 - - THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE - PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION - OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. - - 1. DEFINITIONS - - "Contribution" means: - - a) in the case of the initial Contributor, the initial content - Distributed under this Agreement, and - - b) in the case of each subsequent Contributor: - i) changes to the Program, and - ii) additions to the Program; - where such changes and/or additions to the Program originate from - and are Distributed by that particular Contributor. A Contribution - "originates" from a Contributor if it was added to the Program by - such Contributor itself or anyone acting on such Contributor's behalf. - Contributions do not include changes or additions to the Program that - are not Modified Works. - - "Contributor" means any person or entity that Distributes the Program. - - "Licensed Patents" mean patent claims licensable by a Contributor which - are necessarily infringed by the use or sale of its Contribution alone - or when combined with the Program. - - "Program" means the Contributions Distributed in accordance with this - Agreement. - - "Recipient" means anyone who receives the Program under this Agreement - or any Secondary License (as applicable), including Contributors. - - "Derivative Works" shall mean any work, whether in Source Code or other - form, that is based on (or derived from) the Program and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. - - "Modified Works" shall mean any work in Source Code or other form that - results from an addition to, deletion from, or modification of the - contents of the Program, including, for purposes of clarity any new file - in Source Code form that contains any contents of the Program. Modified - Works shall not include works that contain only declarations, - interfaces, types, classes, structures, or files of the Program solely - in each case in order to link to, bind by name, or subclass the Program - or Modified Works thereof. - - "Distribute" means the acts of a) distributing or b) making available - in any manner that enables the transfer of a copy. - - "Source Code" means the form of a Program preferred for making - modifications, including but not limited to software source code, - documentation source, and configuration files. - - "Secondary License" means either the GNU General Public License, - Version 2.0, or any later versions of that license, including any - exceptions or additional permissions as identified by the initial - Contributor. - - 2. GRANT OF RIGHTS - - a) Subject to the terms of this Agreement, each Contributor hereby - grants Recipient a non-exclusive, worldwide, royalty-free copyright - license to reproduce, prepare Derivative Works of, publicly display, - publicly perform, Distribute and sublicense the Contribution of such - Contributor, if any, and such Derivative Works. - - b) Subject to the terms of this Agreement, each Contributor hereby - grants Recipient a non-exclusive, worldwide, royalty-free patent - license under Licensed Patents to make, use, sell, offer to sell, - import and otherwise transfer the Contribution of such Contributor, - if any, in Source Code or other form. This patent license shall - apply to the combination of the Contribution and the Program if, at - the time the Contribution is added by the Contributor, such addition - of the Contribution causes such combination to be covered by the - Licensed Patents. The patent license shall not apply to any other - combinations which include the Contribution. No hardware per se is - licensed hereunder. - - c) Recipient understands that although each Contributor grants the - licenses to its Contributions set forth herein, no assurances are - provided by any Contributor that the Program does not infringe the - patent or other intellectual property rights of any other entity. - Each Contributor disclaims any liability to Recipient for claims - brought by any other entity based on infringement of intellectual - property rights or otherwise. As a condition to exercising the - rights and licenses granted hereunder, each Recipient hereby - assumes sole responsibility to secure any other intellectual - property rights needed, if any. For example, if a third party - patent license is required to allow Recipient to Distribute the - Program, it is Recipient's responsibility to acquire that license - before distributing the Program. - - d) Each Contributor represents that to its knowledge it has - sufficient copyright rights in its Contribution, if any, to grant - the copyright license set forth in this Agreement. - - e) Notwithstanding the terms of any Secondary License, no - Contributor makes additional grants to any Recipient (other than - those set forth in this Agreement) as a result of such Recipient's - receipt of the Program under the terms of a Secondary License - (if permitted under the terms of Section 3). - - 3. REQUIREMENTS - - 3.1 If a Contributor Distributes the Program in any form, then: - - a) the Program must also be made available as Source Code, in - accordance with section 3.2, and the Contributor must accompany - the Program with a statement that the Source Code for the Program - is available under this Agreement, and informs Recipients how to - obtain it in a reasonable manner on or through a medium customarily - used for software exchange; and - - b) the Contributor may Distribute the Program under a license - different than this Agreement, provided that such license: - i) effectively disclaims on behalf of all other Contributors all - warranties and conditions, express and implied, including - warranties or conditions of title and non-infringement, and - implied warranties or conditions of merchantability and fitness - for a particular purpose; - - ii) effectively excludes on behalf of all other Contributors all - liability for damages, including direct, indirect, special, - incidental and consequential damages, such as lost profits; - - iii) does not attempt to limit or alter the recipients' rights - in the Source Code under section 3.2; and - - iv) requires any subsequent distribution of the Program by any - party to be under a license that satisfies the requirements - of this section 3. - - 3.2 When the Program is Distributed as Source Code: - - a) it must be made available under this Agreement, or if the - Program (i) is combined with other material in a separate file or - files made available under a Secondary License, and (ii) the initial - Contributor attached to the Source Code the notice described in - Exhibit A of this Agreement, then the Program may be made available - under the terms of such Secondary Licenses, and - - b) a copy of this Agreement must be included with each copy of - the Program. - - 3.3 Contributors may not remove or alter any copyright, patent, - trademark, attribution notices, disclaimers of warranty, or limitations - of liability ("notices") contained within the Program from any copy of - the Program which they Distribute, provided that Contributors may add - their own appropriate notices. - - 4. COMMERCIAL DISTRIBUTION - - Commercial distributors of software may accept certain responsibilities - with respect to end users, business partners and the like. While this - license is intended to facilitate the commercial use of the Program, - the Contributor who includes the Program in a commercial product - offering should do so in a manner which does not create potential - liability for other Contributors. Therefore, if a Contributor includes - the Program in a commercial product offering, such Contributor - ("Commercial Contributor") hereby agrees to defend and indemnify every - other Contributor ("Indemnified Contributor") against any losses, - damages and costs (collectively "Losses") arising from claims, lawsuits - and other legal actions brought by a third party against the Indemnified - Contributor to the extent caused by the acts or omissions of such - Commercial Contributor in connection with its distribution of the Program - in a commercial product offering. The obligations in this section do not - apply to any claims or Losses relating to any actual or alleged - intellectual property infringement. In order to qualify, an Indemnified - Contributor must: a) promptly notify the Commercial Contributor in - writing of such claim, and b) allow the Commercial Contributor to control, - and cooperate with the Commercial Contributor in, the defense and any - related settlement negotiations. The Indemnified Contributor may - participate in any such claim at its own expense. - - For example, a Contributor might include the Program in a commercial - product offering, Product X. That Contributor is then a Commercial - Contributor. If that Commercial Contributor then makes performance - claims, or offers warranties related to Product X, those performance - claims and warranties are such Commercial Contributor's responsibility - alone. Under this section, the Commercial Contributor would have to - defend claims against the other Contributors related to those performance - claims and warranties, and if a court requires any other Contributor to - pay any damages as a result, the Commercial Contributor must pay - those damages. - - 5. NO WARRANTY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT - PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED ON AN "AS IS" - BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR - IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF - TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR - PURPOSE. Each Recipient is solely responsible for determining the - appropriateness of using and distributing the Program and assumes all - risks associated with its exercise of rights under this Agreement, - including but not limited to the risks and costs of program errors, - compliance with applicable laws, damage to or loss of data, programs - or equipment, and unavailability or interruption of operations. - - 6. DISCLAIMER OF LIABILITY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT - PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS - SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, - EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST - PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE - EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE - POSSIBILITY OF SUCH DAMAGES. - - 7. GENERAL - - If any provision of this Agreement is invalid or unenforceable under - applicable law, it shall not affect the validity or enforceability of - the remainder of the terms of this Agreement, and without further - action by the parties hereto, such provision shall be reformed to the - minimum extent necessary to make such provision valid and enforceable. - - If Recipient institutes patent litigation against any entity - (including a cross-claim or counterclaim in a lawsuit) alleging that the - Program itself (excluding combinations of the Program with other software - or hardware) infringes such Recipient's patent(s), then such Recipient's - rights granted under Section 2(b) shall terminate as of the date such - litigation is filed. - - All Recipient's rights under this Agreement shall terminate if it - fails to comply with any of the material terms or conditions of this - Agreement and does not cure such failure in a reasonable period of - time after becoming aware of such noncompliance. If all Recipient's - rights under this Agreement terminate, Recipient agrees to cease use - and distribution of the Program as soon as reasonably practicable. - However, Recipient's obligations under this Agreement and any licenses - granted by Recipient relating to the Program shall continue and survive. - - Everyone is permitted to copy and distribute copies of this Agreement, - but in order to avoid inconsistency the Agreement is copyrighted and - may only be modified in the following manner. The Agreement Steward - reserves the right to publish new versions (including revisions) of - this Agreement from time to time. No one other than the Agreement - Steward has the right to modify this Agreement. The Eclipse Foundation - is the initial Agreement Steward. The Eclipse Foundation may assign the - responsibility to serve as the Agreement Steward to a suitable separate - entity. Each new version of the Agreement will be given a distinguishing - version number. The Program (including Contributions) may always be - Distributed subject to the version of the Agreement under which it was - received. In addition, after a new version of the Agreement is published, - Contributor may elect to Distribute the Program (including its - Contributions) under the new version. - - Except as expressly stated in Sections 2(a) and 2(b) above, Recipient - receives no rights or licenses to the intellectual property of any - Contributor under this Agreement, whether expressly, by implication, - estoppel or otherwise. All rights in the Program not expressly granted - under this Agreement are reserved. Nothing in this Agreement is intended - to be enforceable by any entity that is not a Contributor or Recipient. - No third-party beneficiary rights are created under this Agreement. - - Exhibit A - Form of Secondary Licenses Notice - - "This Source Code may also be made available under the following - Secondary Licenses when the conditions for such availability set forth - in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), - version(s), and exceptions or additional permissions here}." - - Simply including a copy of this Agreement, including this Exhibit A - is not sufficient to license the Source Code under Secondary Licenses. - - If it is not possible or desirable to put the notice in a particular - file, then You may include the notice in a location (such as a LICENSE - file in a relevant directory) where a recipient would be likely to - look for such a notice. - - You may add additional accurate notices of copyright ownership. - ---- - -## The GNU General Public License (GPL) Version 2, June 1991 - - Copyright (C) 1989, 1991 Free Software Foundation, Inc. - 51 Franklin Street, Fifth Floor - Boston, MA 02110-1335 - USA - - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - - Preamble - - The licenses for most software are designed to take away your freedom to - share and change it. By contrast, the GNU General Public License is - intended to guarantee your freedom to share and change free software--to - make sure the software is free for all its users. This General Public - License applies to most of the Free Software Foundation's software and - to any other program whose authors commit to using it. (Some other Free - Software Foundation software is covered by the GNU Library General - Public License instead.) You can apply it to your programs, too. - - When we speak of free software, we are referring to freedom, not price. - Our General Public Licenses are designed to make sure that you have the - freedom to distribute copies of free software (and charge for this - service if you wish), that you receive source code or can get it if you - want it, that you can change the software or use pieces of it in new - free programs; and that you know you can do these things. - - To protect your rights, we need to make restrictions that forbid anyone - to deny you these rights or to ask you to surrender the rights. These - restrictions translate to certain responsibilities for you if you - distribute copies of the software, or if you modify it. - - For example, if you distribute copies of such a program, whether gratis - or for a fee, you must give the recipients all the rights that you have. - You must make sure that they, too, receive or can get the source code. - And you must show them these terms so they know their rights. - - We protect your rights with two steps: (1) copyright the software, and - (2) offer you this license which gives you legal permission to copy, - distribute and/or modify the software. - - Also, for each author's protection and ours, we want to make certain - that everyone understands that there is no warranty for this free - software. If the software is modified by someone else and passed on, we - want its recipients to know that what they have is not the original, so - that any problems introduced by others will not reflect on the original - authors' reputations. - - Finally, any free program is threatened constantly by software patents. - We wish to avoid the danger that redistributors of a free program will - individually obtain patent licenses, in effect making the program - proprietary. To prevent this, we have made it clear that any patent must - be licensed for everyone's free use or not licensed at all. - - The precise terms and conditions for copying, distribution and - modification follow. - - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. This License applies to any program or other work which contains a - notice placed by the copyright holder saying it may be distributed under - the terms of this General Public License. The "Program", below, refers - to any such program or work, and a "work based on the Program" means - either the Program or any derivative work under copyright law: that is - to say, a work containing the Program or a portion of it, either - verbatim or with modifications and/or translated into another language. - (Hereinafter, translation is included without limitation in the term - "modification".) Each licensee is addressed as "you". - - Activities other than copying, distribution and modification are not - covered by this License; they are outside its scope. The act of running - the Program is not restricted, and the output from the Program is - covered only if its contents constitute a work based on the Program - (independent of having been made by running the Program). Whether that - is true depends on what the Program does. - - 1. You may copy and distribute verbatim copies of the Program's source - code as you receive it, in any medium, provided that you conspicuously - and appropriately publish on each copy an appropriate copyright notice - and disclaimer of warranty; keep intact all the notices that refer to - this License and to the absence of any warranty; and give any other - recipients of the Program a copy of this License along with the Program. - - You may charge a fee for the physical act of transferring a copy, and - you may at your option offer warranty protection in exchange for a fee. - - 2. You may modify your copy or copies of the Program or any portion of - it, thus forming a work based on the Program, and copy and distribute - such modifications or work under the terms of Section 1 above, provided - that you also meet all of these conditions: - - a) You must cause the modified files to carry prominent notices - stating that you changed the files and the date of any change. - - b) You must cause any work that you distribute or publish, that in - whole or in part contains or is derived from the Program or any part - thereof, to be licensed as a whole at no charge to all third parties - under the terms of this License. - - c) If the modified program normally reads commands interactively - when run, you must cause it, when started running for such - interactive use in the most ordinary way, to print or display an - announcement including an appropriate copyright notice and a notice - that there is no warranty (or else, saying that you provide a - warranty) and that users may redistribute the program under these - conditions, and telling the user how to view a copy of this License. - (Exception: if the Program itself is interactive but does not - normally print such an announcement, your work based on the Program - is not required to print an announcement.) - - These requirements apply to the modified work as a whole. If - identifiable sections of that work are not derived from the Program, and - can be reasonably considered independent and separate works in - themselves, then this License, and its terms, do not apply to those - sections when you distribute them as separate works. But when you - distribute the same sections as part of a whole which is a work based on - the Program, the distribution of the whole must be on the terms of this - License, whose permissions for other licensees extend to the entire - whole, and thus to each and every part regardless of who wrote it. - - Thus, it is not the intent of this section to claim rights or contest - your rights to work written entirely by you; rather, the intent is to - exercise the right to control the distribution of derivative or - collective works based on the Program. - - In addition, mere aggregation of another work not based on the Program - with the Program (or with a work based on the Program) on a volume of a - storage or distribution medium does not bring the other work under the - scope of this License. - - 3. You may copy and distribute the Program (or a work based on it, - under Section 2) in object code or executable form under the terms of - Sections 1 and 2 above provided that you also do one of the following: - - a) Accompany it with the complete corresponding machine-readable - source code, which must be distributed under the terms of Sections 1 - and 2 above on a medium customarily used for software interchange; or, - - b) Accompany it with a written offer, valid for at least three - years, to give any third party, for a charge no more than your cost - of physically performing source distribution, a complete - machine-readable copy of the corresponding source code, to be - distributed under the terms of Sections 1 and 2 above on a medium - customarily used for software interchange; or, - - c) Accompany it with the information you received as to the offer to - distribute corresponding source code. (This alternative is allowed - only for noncommercial distribution and only if you received the - program in object code or executable form with such an offer, in - accord with Subsection b above.) - - The source code for a work means the preferred form of the work for - making modifications to it. For an executable work, complete source code - means all the source code for all modules it contains, plus any - associated interface definition files, plus the scripts used to control - compilation and installation of the executable. However, as a special - exception, the source code distributed need not include anything that is - normally distributed (in either source or binary form) with the major - components (compiler, kernel, and so on) of the operating system on - which the executable runs, unless that component itself accompanies the - executable. - - If distribution of executable or object code is made by offering access - to copy from a designated place, then offering equivalent access to copy - the source code from the same place counts as distribution of the source - code, even though third parties are not compelled to copy the source - along with the object code. - - 4. You may not copy, modify, sublicense, or distribute the Program - except as expressly provided under this License. Any attempt otherwise - to copy, modify, sublicense or distribute the Program is void, and will - automatically terminate your rights under this License. However, parties - who have received copies, or rights, from you under this License will - not have their licenses terminated so long as such parties remain in - full compliance. - - 5. You are not required to accept this License, since you have not - signed it. However, nothing else grants you permission to modify or - distribute the Program or its derivative works. These actions are - prohibited by law if you do not accept this License. Therefore, by - modifying or distributing the Program (or any work based on the - Program), you indicate your acceptance of this License to do so, and all - its terms and conditions for copying, distributing or modifying the - Program or works based on it. - - 6. Each time you redistribute the Program (or any work based on the - Program), the recipient automatically receives a license from the - original licensor to copy, distribute or modify the Program subject to - these terms and conditions. You may not impose any further restrictions - on the recipients' exercise of the rights granted herein. You are not - responsible for enforcing compliance by third parties to this License. - - 7. If, as a consequence of a court judgment or allegation of patent - infringement or for any other reason (not limited to patent issues), - conditions are imposed on you (whether by court order, agreement or - otherwise) that contradict the conditions of this License, they do not - excuse you from the conditions of this License. If you cannot distribute - so as to satisfy simultaneously your obligations under this License and - any other pertinent obligations, then as a consequence you may not - distribute the Program at all. For example, if a patent license would - not permit royalty-free redistribution of the Program by all those who - receive copies directly or indirectly through you, then the only way you - could satisfy both it and this License would be to refrain entirely from - distribution of the Program. - - If any portion of this section is held invalid or unenforceable under - any particular circumstance, the balance of the section is intended to - apply and the section as a whole is intended to apply in other - circumstances. - - It is not the purpose of this section to induce you to infringe any - patents or other property right claims or to contest validity of any - such claims; this section has the sole purpose of protecting the - integrity of the free software distribution system, which is implemented - by public license practices. Many people have made generous - contributions to the wide range of software distributed through that - system in reliance on consistent application of that system; it is up to - the author/donor to decide if he or she is willing to distribute - software through any other system and a licensee cannot impose that choice. - - This section is intended to make thoroughly clear what is believed to be - a consequence of the rest of this License. - - 8. If the distribution and/or use of the Program is restricted in - certain countries either by patents or by copyrighted interfaces, the - original copyright holder who places the Program under this License may - add an explicit geographical distribution limitation excluding those - countries, so that distribution is permitted only in or among countries - not thus excluded. In such case, this License incorporates the - limitation as if written in the body of this License. - - 9. The Free Software Foundation may publish revised and/or new - versions of the General Public License from time to time. Such new - versions will be similar in spirit to the present version, but may - differ in detail to address new problems or concerns. - - Each version is given a distinguishing version number. If the Program - specifies a version number of this License which applies to it and "any - later version", you have the option of following the terms and - conditions either of that version or of any later version published by - the Free Software Foundation. If the Program does not specify a version - number of this License, you may choose any version ever published by the - Free Software Foundation. - - 10. If you wish to incorporate parts of the Program into other free - programs whose distribution conditions are different, write to the - author to ask for permission. For software which is copyrighted by the - Free Software Foundation, write to the Free Software Foundation; we - sometimes make exceptions for this. Our decision will be guided by the - two goals of preserving the free status of all derivatives of our free - software and of promoting the sharing and reuse of software generally. - - NO WARRANTY - - 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO - WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. - EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR - OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, - EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED - WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE - ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH - YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL - NECESSARY SERVICING, REPAIR OR CORRECTION. - - 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN - WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY - AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR - DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL - DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM - (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED - INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF - THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR - OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. - - END OF TERMS AND CONDITIONS - - How to Apply These Terms to Your New Programs - - If you develop a new program, and you want it to be of the greatest - possible use to the public, the best way to achieve this is to make it - free software which everyone can redistribute and change under these terms. - - To do so, attach the following notices to the program. It is safest to - attach them to the start of each source file to most effectively convey - the exclusion of warranty; and each file should have at least the - "copyright" line and a pointer to where the full notice is found. - - One line to give the program's name and a brief idea of what it does. - Copyright (C) - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, but - WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - General Public License for more details. - - You should have received a copy of the GNU General Public License - along with this program; if not, write to the Free Software - Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA - - Also add information on how to contact you by electronic and paper mail. - - If the program is interactive, make it output a short notice like this - when it starts in an interactive mode: - - Gnomovision version 69, Copyright (C) year name of author - Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type - `show w'. This is free software, and you are welcome to redistribute - it under certain conditions; type `show c' for details. - - The hypothetical commands `show w' and `show c' should show the - appropriate parts of the General Public License. Of course, the commands - you use may be called something other than `show w' and `show c'; they - could even be mouse-clicks or menu items--whatever suits your program. - - You should also get your employer (if you work as a programmer) or your - school, if any, to sign a "copyright disclaimer" for the program, if - necessary. Here is a sample; alter the names: - - Yoyodyne, Inc., hereby disclaims all copyright interest in the - program `Gnomovision' (which makes passes at compilers) written by - James Hacker. - - signature of Ty Coon, 1 April 1989 - Ty Coon, President of Vice - - This General Public License does not permit incorporating your program - into proprietary programs. If your program is a subroutine library, you - may consider it more useful to permit linking proprietary applications - with the library. If this is what you want to do, use the GNU Library - General Public License instead of this License. - ---- - -## CLASSPATH EXCEPTION - - Linking this library statically or dynamically with other modules is - making a combined work based on this library. Thus, the terms and - conditions of the GNU General Public License version 2 cover the whole - combination. - - As a special exception, the copyright holders of this library give you - permission to link this library with independent modules to produce an - executable, regardless of the license terms of these independent - modules, and to copy and distribute the resulting executable under - terms of your choice, provided that you also meet, for each linked - independent module, the terms and conditions of the license of that - module. An independent module is a module which is not derived from or - based on this library. If you modify this library, you may extend this - exception to your version of the library, but you are not obligated to - do so. If you do not wish to do so, delete this exception statement - from your version. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-ws-rs-api b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-ws-rs-api deleted file mode 100644 index 5de3d1b40..000000000 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.jakarta-ws-rs-api +++ /dev/null @@ -1,637 +0,0 @@ -# Eclipse Public License - v 2.0 - - THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE - PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION - OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. - - 1. DEFINITIONS - - "Contribution" means: - - a) in the case of the initial Contributor, the initial content - Distributed under this Agreement, and - - b) in the case of each subsequent Contributor: - i) changes to the Program, and - ii) additions to the Program; - where such changes and/or additions to the Program originate from - and are Distributed by that particular Contributor. A Contribution - "originates" from a Contributor if it was added to the Program by - such Contributor itself or anyone acting on such Contributor's behalf. - Contributions do not include changes or additions to the Program that - are not Modified Works. - - "Contributor" means any person or entity that Distributes the Program. - - "Licensed Patents" mean patent claims licensable by a Contributor which - are necessarily infringed by the use or sale of its Contribution alone - or when combined with the Program. - - "Program" means the Contributions Distributed in accordance with this - Agreement. - - "Recipient" means anyone who receives the Program under this Agreement - or any Secondary License (as applicable), including Contributors. - - "Derivative Works" shall mean any work, whether in Source Code or other - form, that is based on (or derived from) the Program and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. - - "Modified Works" shall mean any work in Source Code or other form that - results from an addition to, deletion from, or modification of the - contents of the Program, including, for purposes of clarity any new file - in Source Code form that contains any contents of the Program. Modified - Works shall not include works that contain only declarations, - interfaces, types, classes, structures, or files of the Program solely - in each case in order to link to, bind by name, or subclass the Program - or Modified Works thereof. - - "Distribute" means the acts of a) distributing or b) making available - in any manner that enables the transfer of a copy. - - "Source Code" means the form of a Program preferred for making - modifications, including but not limited to software source code, - documentation source, and configuration files. - - "Secondary License" means either the GNU General Public License, - Version 2.0, or any later versions of that license, including any - exceptions or additional permissions as identified by the initial - Contributor. - - 2. GRANT OF RIGHTS - - a) Subject to the terms of this Agreement, each Contributor hereby - grants Recipient a non-exclusive, worldwide, royalty-free copyright - license to reproduce, prepare Derivative Works of, publicly display, - publicly perform, Distribute and sublicense the Contribution of such - Contributor, if any, and such Derivative Works. - - b) Subject to the terms of this Agreement, each Contributor hereby - grants Recipient a non-exclusive, worldwide, royalty-free patent - license under Licensed Patents to make, use, sell, offer to sell, - import and otherwise transfer the Contribution of such Contributor, - if any, in Source Code or other form. This patent license shall - apply to the combination of the Contribution and the Program if, at - the time the Contribution is added by the Contributor, such addition - of the Contribution causes such combination to be covered by the - Licensed Patents. The patent license shall not apply to any other - combinations which include the Contribution. No hardware per se is - licensed hereunder. - - c) Recipient understands that although each Contributor grants the - licenses to its Contributions set forth herein, no assurances are - provided by any Contributor that the Program does not infringe the - patent or other intellectual property rights of any other entity. - Each Contributor disclaims any liability to Recipient for claims - brought by any other entity based on infringement of intellectual - property rights or otherwise. As a condition to exercising the - rights and licenses granted hereunder, each Recipient hereby - assumes sole responsibility to secure any other intellectual - property rights needed, if any. For example, if a third party - patent license is required to allow Recipient to Distribute the - Program, it is Recipient's responsibility to acquire that license - before distributing the Program. - - d) Each Contributor represents that to its knowledge it has - sufficient copyright rights in its Contribution, if any, to grant - the copyright license set forth in this Agreement. - - e) Notwithstanding the terms of any Secondary License, no - Contributor makes additional grants to any Recipient (other than - those set forth in this Agreement) as a result of such Recipient's - receipt of the Program under the terms of a Secondary License - (if permitted under the terms of Section 3). - - 3. REQUIREMENTS - - 3.1 If a Contributor Distributes the Program in any form, then: - - a) the Program must also be made available as Source Code, in - accordance with section 3.2, and the Contributor must accompany - the Program with a statement that the Source Code for the Program - is available under this Agreement, and informs Recipients how to - obtain it in a reasonable manner on or through a medium customarily - used for software exchange; and - - b) the Contributor may Distribute the Program under a license - different than this Agreement, provided that such license: - i) effectively disclaims on behalf of all other Contributors all - warranties and conditions, express and implied, including - warranties or conditions of title and non-infringement, and - implied warranties or conditions of merchantability and fitness - for a particular purpose; - - ii) effectively excludes on behalf of all other Contributors all - liability for damages, including direct, indirect, special, - incidental and consequential damages, such as lost profits; - - iii) does not attempt to limit or alter the recipients' rights - in the Source Code under section 3.2; and - - iv) requires any subsequent distribution of the Program by any - party to be under a license that satisfies the requirements - of this section 3. - - 3.2 When the Program is Distributed as Source Code: - - a) it must be made available under this Agreement, or if the - Program (i) is combined with other material in a separate file or - files made available under a Secondary License, and (ii) the initial - Contributor attached to the Source Code the notice described in - Exhibit A of this Agreement, then the Program may be made available - under the terms of such Secondary Licenses, and - - b) a copy of this Agreement must be included with each copy of - the Program. - - 3.3 Contributors may not remove or alter any copyright, patent, - trademark, attribution notices, disclaimers of warranty, or limitations - of liability ("notices") contained within the Program from any copy of - the Program which they Distribute, provided that Contributors may add - their own appropriate notices. - - 4. COMMERCIAL DISTRIBUTION - - Commercial distributors of software may accept certain responsibilities - with respect to end users, business partners and the like. While this - license is intended to facilitate the commercial use of the Program, - the Contributor who includes the Program in a commercial product - offering should do so in a manner which does not create potential - liability for other Contributors. Therefore, if a Contributor includes - the Program in a commercial product offering, such Contributor - ("Commercial Contributor") hereby agrees to defend and indemnify every - other Contributor ("Indemnified Contributor") against any losses, - damages and costs (collectively "Losses") arising from claims, lawsuits - and other legal actions brought by a third party against the Indemnified - Contributor to the extent caused by the acts or omissions of such - Commercial Contributor in connection with its distribution of the Program - in a commercial product offering. The obligations in this section do not - apply to any claims or Losses relating to any actual or alleged - intellectual property infringement. In order to qualify, an Indemnified - Contributor must: a) promptly notify the Commercial Contributor in - writing of such claim, and b) allow the Commercial Contributor to control, - and cooperate with the Commercial Contributor in, the defense and any - related settlement negotiations. The Indemnified Contributor may - participate in any such claim at its own expense. - - For example, a Contributor might include the Program in a commercial - product offering, Product X. That Contributor is then a Commercial - Contributor. If that Commercial Contributor then makes performance - claims, or offers warranties related to Product X, those performance - claims and warranties are such Commercial Contributor's responsibility - alone. Under this section, the Commercial Contributor would have to - defend claims against the other Contributors related to those performance - claims and warranties, and if a court requires any other Contributor to - pay any damages as a result, the Commercial Contributor must pay - those damages. - - 5. NO WARRANTY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT - PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED ON AN "AS IS" - BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR - IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF - TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR - PURPOSE. Each Recipient is solely responsible for determining the - appropriateness of using and distributing the Program and assumes all - risks associated with its exercise of rights under this Agreement, - including but not limited to the risks and costs of program errors, - compliance with applicable laws, damage to or loss of data, programs - or equipment, and unavailability or interruption of operations. - - 6. DISCLAIMER OF LIABILITY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT - PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS - SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, - EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST - PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE - EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE - POSSIBILITY OF SUCH DAMAGES. - - 7. GENERAL - - If any provision of this Agreement is invalid or unenforceable under - applicable law, it shall not affect the validity or enforceability of - the remainder of the terms of this Agreement, and without further - action by the parties hereto, such provision shall be reformed to the - minimum extent necessary to make such provision valid and enforceable. - - If Recipient institutes patent litigation against any entity - (including a cross-claim or counterclaim in a lawsuit) alleging that the - Program itself (excluding combinations of the Program with other software - or hardware) infringes such Recipient's patent(s), then such Recipient's - rights granted under Section 2(b) shall terminate as of the date such - litigation is filed. - - All Recipient's rights under this Agreement shall terminate if it - fails to comply with any of the material terms or conditions of this - Agreement and does not cure such failure in a reasonable period of - time after becoming aware of such noncompliance. If all Recipient's - rights under this Agreement terminate, Recipient agrees to cease use - and distribution of the Program as soon as reasonably practicable. - However, Recipient's obligations under this Agreement and any licenses - granted by Recipient relating to the Program shall continue and survive. - - Everyone is permitted to copy and distribute copies of this Agreement, - but in order to avoid inconsistency the Agreement is copyrighted and - may only be modified in the following manner. The Agreement Steward - reserves the right to publish new versions (including revisions) of - this Agreement from time to time. No one other than the Agreement - Steward has the right to modify this Agreement. The Eclipse Foundation - is the initial Agreement Steward. The Eclipse Foundation may assign the - responsibility to serve as the Agreement Steward to a suitable separate - entity. Each new version of the Agreement will be given a distinguishing - version number. The Program (including Contributions) may always be - Distributed subject to the version of the Agreement under which it was - received. In addition, after a new version of the Agreement is published, - Contributor may elect to Distribute the Program (including its - Contributions) under the new version. - - Except as expressly stated in Sections 2(a) and 2(b) above, Recipient - receives no rights or licenses to the intellectual property of any - Contributor under this Agreement, whether expressly, by implication, - estoppel or otherwise. All rights in the Program not expressly granted - under this Agreement are reserved. Nothing in this Agreement is intended - to be enforceable by any entity that is not a Contributor or Recipient. - No third-party beneficiary rights are created under this Agreement. - - Exhibit A - Form of Secondary Licenses Notice - - "This Source Code may also be made available under the following - Secondary Licenses when the conditions for such availability set forth - in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), - version(s), and exceptions or additional permissions here}." - - Simply including a copy of this Agreement, including this Exhibit A - is not sufficient to license the Source Code under Secondary Licenses. - - If it is not possible or desirable to put the notice in a particular - file, then You may include the notice in a location (such as a LICENSE - file in a relevant directory) where a recipient would be likely to - look for such a notice. - - You may add additional accurate notices of copyright ownership. - ---- - -## The GNU General Public License (GPL) Version 2, June 1991 - - Copyright (C) 1989, 1991 Free Software Foundation, Inc. - 51 Franklin Street, Fifth Floor - Boston, MA 02110-1335 - USA - - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - - Preamble - - The licenses for most software are designed to take away your freedom to - share and change it. By contrast, the GNU General Public License is - intended to guarantee your freedom to share and change free software--to - make sure the software is free for all its users. This General Public - License applies to most of the Free Software Foundation's software and - to any other program whose authors commit to using it. (Some other Free - Software Foundation software is covered by the GNU Library General - Public License instead.) You can apply it to your programs, too. - - When we speak of free software, we are referring to freedom, not price. - Our General Public Licenses are designed to make sure that you have the - freedom to distribute copies of free software (and charge for this - service if you wish), that you receive source code or can get it if you - want it, that you can change the software or use pieces of it in new - free programs; and that you know you can do these things. - - To protect your rights, we need to make restrictions that forbid anyone - to deny you these rights or to ask you to surrender the rights. These - restrictions translate to certain responsibilities for you if you - distribute copies of the software, or if you modify it. - - For example, if you distribute copies of such a program, whether gratis - or for a fee, you must give the recipients all the rights that you have. - You must make sure that they, too, receive or can get the source code. - And you must show them these terms so they know their rights. - - We protect your rights with two steps: (1) copyright the software, and - (2) offer you this license which gives you legal permission to copy, - distribute and/or modify the software. - - Also, for each author's protection and ours, we want to make certain - that everyone understands that there is no warranty for this free - software. If the software is modified by someone else and passed on, we - want its recipients to know that what they have is not the original, so - that any problems introduced by others will not reflect on the original - authors' reputations. - - Finally, any free program is threatened constantly by software patents. - We wish to avoid the danger that redistributors of a free program will - individually obtain patent licenses, in effect making the program - proprietary. To prevent this, we have made it clear that any patent must - be licensed for everyone's free use or not licensed at all. - - The precise terms and conditions for copying, distribution and - modification follow. - - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. This License applies to any program or other work which contains a - notice placed by the copyright holder saying it may be distributed under - the terms of this General Public License. The "Program", below, refers - to any such program or work, and a "work based on the Program" means - either the Program or any derivative work under copyright law: that is - to say, a work containing the Program or a portion of it, either - verbatim or with modifications and/or translated into another language. - (Hereinafter, translation is included without limitation in the term - "modification".) Each licensee is addressed as "you". - - Activities other than copying, distribution and modification are not - covered by this License; they are outside its scope. The act of running - the Program is not restricted, and the output from the Program is - covered only if its contents constitute a work based on the Program - (independent of having been made by running the Program). Whether that - is true depends on what the Program does. - - 1. You may copy and distribute verbatim copies of the Program's source - code as you receive it, in any medium, provided that you conspicuously - and appropriately publish on each copy an appropriate copyright notice - and disclaimer of warranty; keep intact all the notices that refer to - this License and to the absence of any warranty; and give any other - recipients of the Program a copy of this License along with the Program. - - You may charge a fee for the physical act of transferring a copy, and - you may at your option offer warranty protection in exchange for a fee. - - 2. You may modify your copy or copies of the Program or any portion of - it, thus forming a work based on the Program, and copy and distribute - such modifications or work under the terms of Section 1 above, provided - that you also meet all of these conditions: - - a) You must cause the modified files to carry prominent notices - stating that you changed the files and the date of any change. - - b) You must cause any work that you distribute or publish, that in - whole or in part contains or is derived from the Program or any part - thereof, to be licensed as a whole at no charge to all third parties - under the terms of this License. - - c) If the modified program normally reads commands interactively - when run, you must cause it, when started running for such - interactive use in the most ordinary way, to print or display an - announcement including an appropriate copyright notice and a notice - that there is no warranty (or else, saying that you provide a - warranty) and that users may redistribute the program under these - conditions, and telling the user how to view a copy of this License. - (Exception: if the Program itself is interactive but does not - normally print such an announcement, your work based on the Program - is not required to print an announcement.) - - These requirements apply to the modified work as a whole. If - identifiable sections of that work are not derived from the Program, and - can be reasonably considered independent and separate works in - themselves, then this License, and its terms, do not apply to those - sections when you distribute them as separate works. But when you - distribute the same sections as part of a whole which is a work based on - the Program, the distribution of the whole must be on the terms of this - License, whose permissions for other licensees extend to the entire - whole, and thus to each and every part regardless of who wrote it. - - Thus, it is not the intent of this section to claim rights or contest - your rights to work written entirely by you; rather, the intent is to - exercise the right to control the distribution of derivative or - collective works based on the Program. - - In addition, mere aggregation of another work not based on the Program - with the Program (or with a work based on the Program) on a volume of a - storage or distribution medium does not bring the other work under the - scope of this License. - - 3. You may copy and distribute the Program (or a work based on it, - under Section 2) in object code or executable form under the terms of - Sections 1 and 2 above provided that you also do one of the following: - - a) Accompany it with the complete corresponding machine-readable - source code, which must be distributed under the terms of Sections 1 - and 2 above on a medium customarily used for software interchange; or, - - b) Accompany it with a written offer, valid for at least three - years, to give any third party, for a charge no more than your cost - of physically performing source distribution, a complete - machine-readable copy of the corresponding source code, to be - distributed under the terms of Sections 1 and 2 above on a medium - customarily used for software interchange; or, - - c) Accompany it with the information you received as to the offer to - distribute corresponding source code. (This alternative is allowed - only for noncommercial distribution and only if you received the - program in object code or executable form with such an offer, in - accord with Subsection b above.) - - The source code for a work means the preferred form of the work for - making modifications to it. For an executable work, complete source code - means all the source code for all modules it contains, plus any - associated interface definition files, plus the scripts used to control - compilation and installation of the executable. However, as a special - exception, the source code distributed need not include anything that is - normally distributed (in either source or binary form) with the major - components (compiler, kernel, and so on) of the operating system on - which the executable runs, unless that component itself accompanies the - executable. - - If distribution of executable or object code is made by offering access - to copy from a designated place, then offering equivalent access to copy - the source code from the same place counts as distribution of the source - code, even though third parties are not compelled to copy the source - along with the object code. - - 4. You may not copy, modify, sublicense, or distribute the Program - except as expressly provided under this License. Any attempt otherwise - to copy, modify, sublicense or distribute the Program is void, and will - automatically terminate your rights under this License. However, parties - who have received copies, or rights, from you under this License will - not have their licenses terminated so long as such parties remain in - full compliance. - - 5. You are not required to accept this License, since you have not - signed it. However, nothing else grants you permission to modify or - distribute the Program or its derivative works. These actions are - prohibited by law if you do not accept this License. Therefore, by - modifying or distributing the Program (or any work based on the - Program), you indicate your acceptance of this License to do so, and all - its terms and conditions for copying, distributing or modifying the - Program or works based on it. - - 6. Each time you redistribute the Program (or any work based on the - Program), the recipient automatically receives a license from the - original licensor to copy, distribute or modify the Program subject to - these terms and conditions. You may not impose any further restrictions - on the recipients' exercise of the rights granted herein. You are not - responsible for enforcing compliance by third parties to this License. - - 7. If, as a consequence of a court judgment or allegation of patent - infringement or for any other reason (not limited to patent issues), - conditions are imposed on you (whether by court order, agreement or - otherwise) that contradict the conditions of this License, they do not - excuse you from the conditions of this License. If you cannot distribute - so as to satisfy simultaneously your obligations under this License and - any other pertinent obligations, then as a consequence you may not - distribute the Program at all. For example, if a patent license would - not permit royalty-free redistribution of the Program by all those who - receive copies directly or indirectly through you, then the only way you - could satisfy both it and this License would be to refrain entirely from - distribution of the Program. - - If any portion of this section is held invalid or unenforceable under - any particular circumstance, the balance of the section is intended to - apply and the section as a whole is intended to apply in other - circumstances. - - It is not the purpose of this section to induce you to infringe any - patents or other property right claims or to contest validity of any - such claims; this section has the sole purpose of protecting the - integrity of the free software distribution system, which is implemented - by public license practices. Many people have made generous - contributions to the wide range of software distributed through that - system in reliance on consistent application of that system; it is up to - the author/donor to decide if he or she is willing to distribute - software through any other system and a licensee cannot impose that choice. - - This section is intended to make thoroughly clear what is believed to be - a consequence of the rest of this License. - - 8. If the distribution and/or use of the Program is restricted in - certain countries either by patents or by copyrighted interfaces, the - original copyright holder who places the Program under this License may - add an explicit geographical distribution limitation excluding those - countries, so that distribution is permitted only in or among countries - not thus excluded. In such case, this License incorporates the - limitation as if written in the body of this License. - - 9. The Free Software Foundation may publish revised and/or new - versions of the General Public License from time to time. Such new - versions will be similar in spirit to the present version, but may - differ in detail to address new problems or concerns. - - Each version is given a distinguishing version number. If the Program - specifies a version number of this License which applies to it and "any - later version", you have the option of following the terms and - conditions either of that version or of any later version published by - the Free Software Foundation. If the Program does not specify a version - number of this License, you may choose any version ever published by the - Free Software Foundation. - - 10. If you wish to incorporate parts of the Program into other free - programs whose distribution conditions are different, write to the - author to ask for permission. For software which is copyrighted by the - Free Software Foundation, write to the Free Software Foundation; we - sometimes make exceptions for this. Our decision will be guided by the - two goals of preserving the free status of all derivatives of our free - software and of promoting the sharing and reuse of software generally. - - NO WARRANTY - - 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO - WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. - EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR - OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, - EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED - WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE - ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH - YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL - NECESSARY SERVICING, REPAIR OR CORRECTION. - - 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN - WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY - AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR - DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL - DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM - (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED - INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF - THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR - OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. - - END OF TERMS AND CONDITIONS - - How to Apply These Terms to Your New Programs - - If you develop a new program, and you want it to be of the greatest - possible use to the public, the best way to achieve this is to make it - free software which everyone can redistribute and change under these terms. - - To do so, attach the following notices to the program. It is safest to - attach them to the start of each source file to most effectively convey - the exclusion of warranty; and each file should have at least the - "copyright" line and a pointer to where the full notice is found. - - One line to give the program's name and a brief idea of what it does. - Copyright (C) - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, but - WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - General Public License for more details. - - You should have received a copy of the GNU General Public License - along with this program; if not, write to the Free Software - Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA - - Also add information on how to contact you by electronic and paper mail. - - If the program is interactive, make it output a short notice like this - when it starts in an interactive mode: - - Gnomovision version 69, Copyright (C) year name of author - Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type - `show w'. This is free software, and you are welcome to redistribute - it under certain conditions; type `show c' for details. - - The hypothetical commands `show w' and `show c' should show the - appropriate parts of the General Public License. Of course, the commands - you use may be called something other than `show w' and `show c'; they - could even be mouse-clicks or menu items--whatever suits your program. - - You should also get your employer (if you work as a programmer) or your - school, if any, to sign a "copyright disclaimer" for the program, if - necessary. Here is a sample; alter the names: - - Yoyodyne, Inc., hereby disclaims all copyright interest in the - program `Gnomovision' (which makes passes at compilers) written by - James Hacker. - - signature of Ty Coon, 1 April 1989 - Ty Coon, President of Vice - - This General Public License does not permit incorporating your program - into proprietary programs. If your program is a subroutine library, you - may consider it more useful to permit linking proprietary applications - with the library. If this is what you want to do, use the GNU Library - General Public License instead of this License. - ---- - -## CLASSPATH EXCEPTION - - Linking this library statically or dynamically with other modules is - making a combined work based on this library. Thus, the terms and - conditions of the GNU General Public License version 2 cover the whole - combination. - - As a special exception, the copyright holders of this library give you - permission to link this library with independent modules to produce an - executable, regardless of the license terms of these independent - modules, and to copy and distribute the resulting executable under - terms of your choice, provided that you also meet, for each linked - independent module, the terms and conditions of the license of that - module. An independent module is a module which is not derived from or - based on this library. If you modify this library, you may extend this - exception to your version of the library, but you are not obligated to - do so. If you do not wish to do so, delete this exception statement - from your version. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.osgi-resource-locator b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.osgi-resource-locator deleted file mode 100644 index 5de3d1b40..000000000 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.osgi-resource-locator +++ /dev/null @@ -1,637 +0,0 @@ -# Eclipse Public License - v 2.0 - - THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE - PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION - OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. - - 1. DEFINITIONS - - "Contribution" means: - - a) in the case of the initial Contributor, the initial content - Distributed under this Agreement, and - - b) in the case of each subsequent Contributor: - i) changes to the Program, and - ii) additions to the Program; - where such changes and/or additions to the Program originate from - and are Distributed by that particular Contributor. A Contribution - "originates" from a Contributor if it was added to the Program by - such Contributor itself or anyone acting on such Contributor's behalf. - Contributions do not include changes or additions to the Program that - are not Modified Works. - - "Contributor" means any person or entity that Distributes the Program. - - "Licensed Patents" mean patent claims licensable by a Contributor which - are necessarily infringed by the use or sale of its Contribution alone - or when combined with the Program. - - "Program" means the Contributions Distributed in accordance with this - Agreement. - - "Recipient" means anyone who receives the Program under this Agreement - or any Secondary License (as applicable), including Contributors. - - "Derivative Works" shall mean any work, whether in Source Code or other - form, that is based on (or derived from) the Program and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. - - "Modified Works" shall mean any work in Source Code or other form that - results from an addition to, deletion from, or modification of the - contents of the Program, including, for purposes of clarity any new file - in Source Code form that contains any contents of the Program. Modified - Works shall not include works that contain only declarations, - interfaces, types, classes, structures, or files of the Program solely - in each case in order to link to, bind by name, or subclass the Program - or Modified Works thereof. - - "Distribute" means the acts of a) distributing or b) making available - in any manner that enables the transfer of a copy. - - "Source Code" means the form of a Program preferred for making - modifications, including but not limited to software source code, - documentation source, and configuration files. - - "Secondary License" means either the GNU General Public License, - Version 2.0, or any later versions of that license, including any - exceptions or additional permissions as identified by the initial - Contributor. - - 2. GRANT OF RIGHTS - - a) Subject to the terms of this Agreement, each Contributor hereby - grants Recipient a non-exclusive, worldwide, royalty-free copyright - license to reproduce, prepare Derivative Works of, publicly display, - publicly perform, Distribute and sublicense the Contribution of such - Contributor, if any, and such Derivative Works. - - b) Subject to the terms of this Agreement, each Contributor hereby - grants Recipient a non-exclusive, worldwide, royalty-free patent - license under Licensed Patents to make, use, sell, offer to sell, - import and otherwise transfer the Contribution of such Contributor, - if any, in Source Code or other form. This patent license shall - apply to the combination of the Contribution and the Program if, at - the time the Contribution is added by the Contributor, such addition - of the Contribution causes such combination to be covered by the - Licensed Patents. The patent license shall not apply to any other - combinations which include the Contribution. No hardware per se is - licensed hereunder. - - c) Recipient understands that although each Contributor grants the - licenses to its Contributions set forth herein, no assurances are - provided by any Contributor that the Program does not infringe the - patent or other intellectual property rights of any other entity. - Each Contributor disclaims any liability to Recipient for claims - brought by any other entity based on infringement of intellectual - property rights or otherwise. As a condition to exercising the - rights and licenses granted hereunder, each Recipient hereby - assumes sole responsibility to secure any other intellectual - property rights needed, if any. For example, if a third party - patent license is required to allow Recipient to Distribute the - Program, it is Recipient's responsibility to acquire that license - before distributing the Program. - - d) Each Contributor represents that to its knowledge it has - sufficient copyright rights in its Contribution, if any, to grant - the copyright license set forth in this Agreement. - - e) Notwithstanding the terms of any Secondary License, no - Contributor makes additional grants to any Recipient (other than - those set forth in this Agreement) as a result of such Recipient's - receipt of the Program under the terms of a Secondary License - (if permitted under the terms of Section 3). - - 3. REQUIREMENTS - - 3.1 If a Contributor Distributes the Program in any form, then: - - a) the Program must also be made available as Source Code, in - accordance with section 3.2, and the Contributor must accompany - the Program with a statement that the Source Code for the Program - is available under this Agreement, and informs Recipients how to - obtain it in a reasonable manner on or through a medium customarily - used for software exchange; and - - b) the Contributor may Distribute the Program under a license - different than this Agreement, provided that such license: - i) effectively disclaims on behalf of all other Contributors all - warranties and conditions, express and implied, including - warranties or conditions of title and non-infringement, and - implied warranties or conditions of merchantability and fitness - for a particular purpose; - - ii) effectively excludes on behalf of all other Contributors all - liability for damages, including direct, indirect, special, - incidental and consequential damages, such as lost profits; - - iii) does not attempt to limit or alter the recipients' rights - in the Source Code under section 3.2; and - - iv) requires any subsequent distribution of the Program by any - party to be under a license that satisfies the requirements - of this section 3. - - 3.2 When the Program is Distributed as Source Code: - - a) it must be made available under this Agreement, or if the - Program (i) is combined with other material in a separate file or - files made available under a Secondary License, and (ii) the initial - Contributor attached to the Source Code the notice described in - Exhibit A of this Agreement, then the Program may be made available - under the terms of such Secondary Licenses, and - - b) a copy of this Agreement must be included with each copy of - the Program. - - 3.3 Contributors may not remove or alter any copyright, patent, - trademark, attribution notices, disclaimers of warranty, or limitations - of liability ("notices") contained within the Program from any copy of - the Program which they Distribute, provided that Contributors may add - their own appropriate notices. - - 4. COMMERCIAL DISTRIBUTION - - Commercial distributors of software may accept certain responsibilities - with respect to end users, business partners and the like. While this - license is intended to facilitate the commercial use of the Program, - the Contributor who includes the Program in a commercial product - offering should do so in a manner which does not create potential - liability for other Contributors. Therefore, if a Contributor includes - the Program in a commercial product offering, such Contributor - ("Commercial Contributor") hereby agrees to defend and indemnify every - other Contributor ("Indemnified Contributor") against any losses, - damages and costs (collectively "Losses") arising from claims, lawsuits - and other legal actions brought by a third party against the Indemnified - Contributor to the extent caused by the acts or omissions of such - Commercial Contributor in connection with its distribution of the Program - in a commercial product offering. The obligations in this section do not - apply to any claims or Losses relating to any actual or alleged - intellectual property infringement. In order to qualify, an Indemnified - Contributor must: a) promptly notify the Commercial Contributor in - writing of such claim, and b) allow the Commercial Contributor to control, - and cooperate with the Commercial Contributor in, the defense and any - related settlement negotiations. The Indemnified Contributor may - participate in any such claim at its own expense. - - For example, a Contributor might include the Program in a commercial - product offering, Product X. That Contributor is then a Commercial - Contributor. If that Commercial Contributor then makes performance - claims, or offers warranties related to Product X, those performance - claims and warranties are such Commercial Contributor's responsibility - alone. Under this section, the Commercial Contributor would have to - defend claims against the other Contributors related to those performance - claims and warranties, and if a court requires any other Contributor to - pay any damages as a result, the Commercial Contributor must pay - those damages. - - 5. NO WARRANTY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT - PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED ON AN "AS IS" - BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR - IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF - TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR - PURPOSE. Each Recipient is solely responsible for determining the - appropriateness of using and distributing the Program and assumes all - risks associated with its exercise of rights under this Agreement, - including but not limited to the risks and costs of program errors, - compliance with applicable laws, damage to or loss of data, programs - or equipment, and unavailability or interruption of operations. - - 6. DISCLAIMER OF LIABILITY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT - PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS - SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, - EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST - PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE - EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE - POSSIBILITY OF SUCH DAMAGES. - - 7. GENERAL - - If any provision of this Agreement is invalid or unenforceable under - applicable law, it shall not affect the validity or enforceability of - the remainder of the terms of this Agreement, and without further - action by the parties hereto, such provision shall be reformed to the - minimum extent necessary to make such provision valid and enforceable. - - If Recipient institutes patent litigation against any entity - (including a cross-claim or counterclaim in a lawsuit) alleging that the - Program itself (excluding combinations of the Program with other software - or hardware) infringes such Recipient's patent(s), then such Recipient's - rights granted under Section 2(b) shall terminate as of the date such - litigation is filed. - - All Recipient's rights under this Agreement shall terminate if it - fails to comply with any of the material terms or conditions of this - Agreement and does not cure such failure in a reasonable period of - time after becoming aware of such noncompliance. If all Recipient's - rights under this Agreement terminate, Recipient agrees to cease use - and distribution of the Program as soon as reasonably practicable. - However, Recipient's obligations under this Agreement and any licenses - granted by Recipient relating to the Program shall continue and survive. - - Everyone is permitted to copy and distribute copies of this Agreement, - but in order to avoid inconsistency the Agreement is copyrighted and - may only be modified in the following manner. The Agreement Steward - reserves the right to publish new versions (including revisions) of - this Agreement from time to time. No one other than the Agreement - Steward has the right to modify this Agreement. The Eclipse Foundation - is the initial Agreement Steward. The Eclipse Foundation may assign the - responsibility to serve as the Agreement Steward to a suitable separate - entity. Each new version of the Agreement will be given a distinguishing - version number. The Program (including Contributions) may always be - Distributed subject to the version of the Agreement under which it was - received. In addition, after a new version of the Agreement is published, - Contributor may elect to Distribute the Program (including its - Contributions) under the new version. - - Except as expressly stated in Sections 2(a) and 2(b) above, Recipient - receives no rights or licenses to the intellectual property of any - Contributor under this Agreement, whether expressly, by implication, - estoppel or otherwise. All rights in the Program not expressly granted - under this Agreement are reserved. Nothing in this Agreement is intended - to be enforceable by any entity that is not a Contributor or Recipient. - No third-party beneficiary rights are created under this Agreement. - - Exhibit A - Form of Secondary Licenses Notice - - "This Source Code may also be made available under the following - Secondary Licenses when the conditions for such availability set forth - in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), - version(s), and exceptions or additional permissions here}." - - Simply including a copy of this Agreement, including this Exhibit A - is not sufficient to license the Source Code under Secondary Licenses. - - If it is not possible or desirable to put the notice in a particular - file, then You may include the notice in a location (such as a LICENSE - file in a relevant directory) where a recipient would be likely to - look for such a notice. - - You may add additional accurate notices of copyright ownership. - ---- - -## The GNU General Public License (GPL) Version 2, June 1991 - - Copyright (C) 1989, 1991 Free Software Foundation, Inc. - 51 Franklin Street, Fifth Floor - Boston, MA 02110-1335 - USA - - Everyone is permitted to copy and distribute verbatim copies - of this license document, but changing it is not allowed. - - Preamble - - The licenses for most software are designed to take away your freedom to - share and change it. By contrast, the GNU General Public License is - intended to guarantee your freedom to share and change free software--to - make sure the software is free for all its users. This General Public - License applies to most of the Free Software Foundation's software and - to any other program whose authors commit to using it. (Some other Free - Software Foundation software is covered by the GNU Library General - Public License instead.) You can apply it to your programs, too. - - When we speak of free software, we are referring to freedom, not price. - Our General Public Licenses are designed to make sure that you have the - freedom to distribute copies of free software (and charge for this - service if you wish), that you receive source code or can get it if you - want it, that you can change the software or use pieces of it in new - free programs; and that you know you can do these things. - - To protect your rights, we need to make restrictions that forbid anyone - to deny you these rights or to ask you to surrender the rights. These - restrictions translate to certain responsibilities for you if you - distribute copies of the software, or if you modify it. - - For example, if you distribute copies of such a program, whether gratis - or for a fee, you must give the recipients all the rights that you have. - You must make sure that they, too, receive or can get the source code. - And you must show them these terms so they know their rights. - - We protect your rights with two steps: (1) copyright the software, and - (2) offer you this license which gives you legal permission to copy, - distribute and/or modify the software. - - Also, for each author's protection and ours, we want to make certain - that everyone understands that there is no warranty for this free - software. If the software is modified by someone else and passed on, we - want its recipients to know that what they have is not the original, so - that any problems introduced by others will not reflect on the original - authors' reputations. - - Finally, any free program is threatened constantly by software patents. - We wish to avoid the danger that redistributors of a free program will - individually obtain patent licenses, in effect making the program - proprietary. To prevent this, we have made it clear that any patent must - be licensed for everyone's free use or not licensed at all. - - The precise terms and conditions for copying, distribution and - modification follow. - - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. This License applies to any program or other work which contains a - notice placed by the copyright holder saying it may be distributed under - the terms of this General Public License. The "Program", below, refers - to any such program or work, and a "work based on the Program" means - either the Program or any derivative work under copyright law: that is - to say, a work containing the Program or a portion of it, either - verbatim or with modifications and/or translated into another language. - (Hereinafter, translation is included without limitation in the term - "modification".) Each licensee is addressed as "you". - - Activities other than copying, distribution and modification are not - covered by this License; they are outside its scope. The act of running - the Program is not restricted, and the output from the Program is - covered only if its contents constitute a work based on the Program - (independent of having been made by running the Program). Whether that - is true depends on what the Program does. - - 1. You may copy and distribute verbatim copies of the Program's source - code as you receive it, in any medium, provided that you conspicuously - and appropriately publish on each copy an appropriate copyright notice - and disclaimer of warranty; keep intact all the notices that refer to - this License and to the absence of any warranty; and give any other - recipients of the Program a copy of this License along with the Program. - - You may charge a fee for the physical act of transferring a copy, and - you may at your option offer warranty protection in exchange for a fee. - - 2. You may modify your copy or copies of the Program or any portion of - it, thus forming a work based on the Program, and copy and distribute - such modifications or work under the terms of Section 1 above, provided - that you also meet all of these conditions: - - a) You must cause the modified files to carry prominent notices - stating that you changed the files and the date of any change. - - b) You must cause any work that you distribute or publish, that in - whole or in part contains or is derived from the Program or any part - thereof, to be licensed as a whole at no charge to all third parties - under the terms of this License. - - c) If the modified program normally reads commands interactively - when run, you must cause it, when started running for such - interactive use in the most ordinary way, to print or display an - announcement including an appropriate copyright notice and a notice - that there is no warranty (or else, saying that you provide a - warranty) and that users may redistribute the program under these - conditions, and telling the user how to view a copy of this License. - (Exception: if the Program itself is interactive but does not - normally print such an announcement, your work based on the Program - is not required to print an announcement.) - - These requirements apply to the modified work as a whole. If - identifiable sections of that work are not derived from the Program, and - can be reasonably considered independent and separate works in - themselves, then this License, and its terms, do not apply to those - sections when you distribute them as separate works. But when you - distribute the same sections as part of a whole which is a work based on - the Program, the distribution of the whole must be on the terms of this - License, whose permissions for other licensees extend to the entire - whole, and thus to each and every part regardless of who wrote it. - - Thus, it is not the intent of this section to claim rights or contest - your rights to work written entirely by you; rather, the intent is to - exercise the right to control the distribution of derivative or - collective works based on the Program. - - In addition, mere aggregation of another work not based on the Program - with the Program (or with a work based on the Program) on a volume of a - storage or distribution medium does not bring the other work under the - scope of this License. - - 3. You may copy and distribute the Program (or a work based on it, - under Section 2) in object code or executable form under the terms of - Sections 1 and 2 above provided that you also do one of the following: - - a) Accompany it with the complete corresponding machine-readable - source code, which must be distributed under the terms of Sections 1 - and 2 above on a medium customarily used for software interchange; or, - - b) Accompany it with a written offer, valid for at least three - years, to give any third party, for a charge no more than your cost - of physically performing source distribution, a complete - machine-readable copy of the corresponding source code, to be - distributed under the terms of Sections 1 and 2 above on a medium - customarily used for software interchange; or, - - c) Accompany it with the information you received as to the offer to - distribute corresponding source code. (This alternative is allowed - only for noncommercial distribution and only if you received the - program in object code or executable form with such an offer, in - accord with Subsection b above.) - - The source code for a work means the preferred form of the work for - making modifications to it. For an executable work, complete source code - means all the source code for all modules it contains, plus any - associated interface definition files, plus the scripts used to control - compilation and installation of the executable. However, as a special - exception, the source code distributed need not include anything that is - normally distributed (in either source or binary form) with the major - components (compiler, kernel, and so on) of the operating system on - which the executable runs, unless that component itself accompanies the - executable. - - If distribution of executable or object code is made by offering access - to copy from a designated place, then offering equivalent access to copy - the source code from the same place counts as distribution of the source - code, even though third parties are not compelled to copy the source - along with the object code. - - 4. You may not copy, modify, sublicense, or distribute the Program - except as expressly provided under this License. Any attempt otherwise - to copy, modify, sublicense or distribute the Program is void, and will - automatically terminate your rights under this License. However, parties - who have received copies, or rights, from you under this License will - not have their licenses terminated so long as such parties remain in - full compliance. - - 5. You are not required to accept this License, since you have not - signed it. However, nothing else grants you permission to modify or - distribute the Program or its derivative works. These actions are - prohibited by law if you do not accept this License. Therefore, by - modifying or distributing the Program (or any work based on the - Program), you indicate your acceptance of this License to do so, and all - its terms and conditions for copying, distributing or modifying the - Program or works based on it. - - 6. Each time you redistribute the Program (or any work based on the - Program), the recipient automatically receives a license from the - original licensor to copy, distribute or modify the Program subject to - these terms and conditions. You may not impose any further restrictions - on the recipients' exercise of the rights granted herein. You are not - responsible for enforcing compliance by third parties to this License. - - 7. If, as a consequence of a court judgment or allegation of patent - infringement or for any other reason (not limited to patent issues), - conditions are imposed on you (whether by court order, agreement or - otherwise) that contradict the conditions of this License, they do not - excuse you from the conditions of this License. If you cannot distribute - so as to satisfy simultaneously your obligations under this License and - any other pertinent obligations, then as a consequence you may not - distribute the Program at all. For example, if a patent license would - not permit royalty-free redistribution of the Program by all those who - receive copies directly or indirectly through you, then the only way you - could satisfy both it and this License would be to refrain entirely from - distribution of the Program. - - If any portion of this section is held invalid or unenforceable under - any particular circumstance, the balance of the section is intended to - apply and the section as a whole is intended to apply in other - circumstances. - - It is not the purpose of this section to induce you to infringe any - patents or other property right claims or to contest validity of any - such claims; this section has the sole purpose of protecting the - integrity of the free software distribution system, which is implemented - by public license practices. Many people have made generous - contributions to the wide range of software distributed through that - system in reliance on consistent application of that system; it is up to - the author/donor to decide if he or she is willing to distribute - software through any other system and a licensee cannot impose that choice. - - This section is intended to make thoroughly clear what is believed to be - a consequence of the rest of this License. - - 8. If the distribution and/or use of the Program is restricted in - certain countries either by patents or by copyrighted interfaces, the - original copyright holder who places the Program under this License may - add an explicit geographical distribution limitation excluding those - countries, so that distribution is permitted only in or among countries - not thus excluded. In such case, this License incorporates the - limitation as if written in the body of this License. - - 9. The Free Software Foundation may publish revised and/or new - versions of the General Public License from time to time. Such new - versions will be similar in spirit to the present version, but may - differ in detail to address new problems or concerns. - - Each version is given a distinguishing version number. If the Program - specifies a version number of this License which applies to it and "any - later version", you have the option of following the terms and - conditions either of that version or of any later version published by - the Free Software Foundation. If the Program does not specify a version - number of this License, you may choose any version ever published by the - Free Software Foundation. - - 10. If you wish to incorporate parts of the Program into other free - programs whose distribution conditions are different, write to the - author to ask for permission. For software which is copyrighted by the - Free Software Foundation, write to the Free Software Foundation; we - sometimes make exceptions for this. Our decision will be guided by the - two goals of preserving the free status of all derivatives of our free - software and of promoting the sharing and reuse of software generally. - - NO WARRANTY - - 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO - WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. - EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR - OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, - EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED - WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE - ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH - YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL - NECESSARY SERVICING, REPAIR OR CORRECTION. - - 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN - WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY - AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR - DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL - DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM - (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED - INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF - THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR - OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. - - END OF TERMS AND CONDITIONS - - How to Apply These Terms to Your New Programs - - If you develop a new program, and you want it to be of the greatest - possible use to the public, the best way to achieve this is to make it - free software which everyone can redistribute and change under these terms. - - To do so, attach the following notices to the program. It is safest to - attach them to the start of each source file to most effectively convey - the exclusion of warranty; and each file should have at least the - "copyright" line and a pointer to where the full notice is found. - - One line to give the program's name and a brief idea of what it does. - Copyright (C) - - This program is free software; you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2 of the License, or - (at your option) any later version. - - This program is distributed in the hope that it will be useful, but - WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - General Public License for more details. - - You should have received a copy of the GNU General Public License - along with this program; if not, write to the Free Software - Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA - - Also add information on how to contact you by electronic and paper mail. - - If the program is interactive, make it output a short notice like this - when it starts in an interactive mode: - - Gnomovision version 69, Copyright (C) year name of author - Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type - `show w'. This is free software, and you are welcome to redistribute - it under certain conditions; type `show c' for details. - - The hypothetical commands `show w' and `show c' should show the - appropriate parts of the General Public License. Of course, the commands - you use may be called something other than `show w' and `show c'; they - could even be mouse-clicks or menu items--whatever suits your program. - - You should also get your employer (if you work as a programmer) or your - school, if any, to sign a "copyright disclaimer" for the program, if - necessary. Here is a sample; alter the names: - - Yoyodyne, Inc., hereby disclaims all copyright interest in the - program `Gnomovision' (which makes passes at compilers) written by - James Hacker. - - signature of Ty Coon, 1 April 1989 - Ty Coon, President of Vice - - This General Public License does not permit incorporating your program - into proprietary programs. If your program is a subroutine library, you - may consider it more useful to permit linking proprietary applications - with the library. If this is what you want to do, use the GNU Library - General Public License instead of this License. - ---- - -## CLASSPATH EXCEPTION - - Linking this library statically or dynamically with other modules is - making a combined work based on this library. Thus, the terms and - conditions of the GNU General Public License version 2 cover the whole - combination. - - As a special exception, the copyright holders of this library give you - permission to link this library with independent modules to produce an - executable, regardless of the license terms of these independent - modules, and to copy and distribute the resulting executable under - terms of your choice, provided that you also meet, for each linked - independent module, the terms and conditions of the license of that - module. An independent module is a module which is not derived from or - based on this library. If you modify this library, you may extend this - exception to your version of the library, but you are not obligated to - do so. If you do not wish to do so, delete this exception statement - from your version. diff --git a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.zstd-jni b/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.zstd-jni deleted file mode 100644 index 66abb8ae7..000000000 --- a/flink-formats-kafka/flink-sql-avro-confluent-registry/src/main/resources/META-INF/licenses/LICENSE.zstd-jni +++ /dev/null @@ -1,26 +0,0 @@ -Zstd-jni: JNI bindings to Zstd Library - -Copyright (c) 2015-present, Luben Karavelov/ All rights reserved. - -BSD License - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - -* Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. - -* Redistributions in binary form must reproduce the above copyright notice, this - list of conditions and the following disclaimer in the documentation and/or - other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON -ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-formats-kafka/pom.xml b/flink-formats-kafka/pom.xml deleted file mode 100644 index 73fb90939..000000000 --- a/flink-formats-kafka/pom.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-connector-kafka-parent - 3.1-SNAPSHOT - - - pom - - flink-formats-kafka - Flink : Formats : Kafka - - - flink-avro-confluent-registry - flink-sql-avro-confluent-registry - - - diff --git a/pom.xml b/pom.xml index da9142c89..a9e9610ce 100644 --- a/pom.xml +++ b/pom.xml @@ -46,8 +46,6 @@ under the License. flink-connector-kafka flink-sql-connector-kafka flink-connector-kafka-e2e-tests - flink-formats-kafka - flink-confluent-schema-registry-e2e-tests From c1e35275964a7437fc9a32025294a8c3a2523f94 Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Wed, 28 Jun 2023 12:02:11 +0200 Subject: [PATCH 218/322] [FLINK-32462] Pin the used shaded version of Guava and Jackson in the Kafka connector This closes #36. --- pom.xml | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index a9e9610ce..bfc91787a 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,6 @@ under the License. 1.17.0 - 16.1 3.4.0 3.5.9 7.2.2 @@ -83,8 +82,14 @@ under the License. org.apache.flink - flink-shaded-force-shading - ${flink.shaded.version} + flink-shaded-guava + 30.1.1-jre-16.1 + + + + org.apache.flink + flink-shaded-jackson + 2.13.4-16.1 From ad62c133ee32241d9b897543b86a1cc9fe64414b Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Sat, 6 May 2023 11:53:55 +0800 Subject: [PATCH 219/322] [FLINK-32019][Connector/Kafka] EARLIEST offset strategy for partitions discoveried later based on FLIP-288 --- .../connector/kafka/source/KafkaSource.java | 2 +- .../source/enumerator/AssignmentStatus.java | 52 +++++++ .../enumerator/KafkaSourceEnumState.java | 62 +++++++- .../KafkaSourceEnumStateSerializer.java | 133 +++++++++++++----- .../enumerator/KafkaSourceEnumerator.java | 47 +++++-- .../TopicPartitionAndAssignmentStatus.java | 44 ++++++ .../enumerator/KafkaEnumeratorTest.java | 113 +++++++++++++-- .../KafkaSourceEnumStateSerializerTest.java | 42 ++++-- 8 files changed, 420 insertions(+), 75 deletions(-) create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/AssignmentStatus.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/TopicPartitionAndAssignmentStatus.java diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java index 7a17b1ff6..0e764649f 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java @@ -196,7 +196,7 @@ public SplitEnumerator restoreEnumera props, enumContext, boundedness, - checkpoint.assignedPartitions()); + checkpoint); } @Internal diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/AssignmentStatus.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/AssignmentStatus.java new file mode 100644 index 000000000..b7d115386 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/AssignmentStatus.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.source.enumerator; + +import org.apache.flink.annotation.Internal; + +/** status of partition assignment. */ +@Internal +public enum AssignmentStatus { + + /** Partitions that have been assigned to readers. */ + ASSIGNED(0), + /** + * The partitions that have been discovered during initialization but not assigned to readers + * yet. + */ + UNASSIGNED_INITIAL(1); + private final int statusCode; + + AssignmentStatus(int statusCode) { + this.statusCode = statusCode; + } + + public int getStatusCode() { + return statusCode; + } + + public static AssignmentStatus ofStatusCode(int statusCode) { + for (AssignmentStatus statusEnum : AssignmentStatus.values()) { + if (statusEnum.getStatusCode() == statusCode) { + return statusEnum; + } + } + throw new IllegalArgumentException("statusCode is invalid."); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java index 04604abe1..1375d9f60 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java @@ -22,18 +22,72 @@ import org.apache.kafka.common.TopicPartition; +import java.util.HashSet; import java.util.Set; +import java.util.stream.Collectors; /** The state of Kafka source enumerator. */ @Internal public class KafkaSourceEnumState { - private final Set assignedPartitions; + /** Partitions with status: ASSIGNED or UNASSIGNED_INITIAL. */ + private final Set partitions; + /** + * this flag will be marked as true if inital partitions are discovered after enumerator starts. + */ + private final boolean initialDiscoveryFinished; - KafkaSourceEnumState(Set assignedPartitions) { - this.assignedPartitions = assignedPartitions; + public KafkaSourceEnumState( + Set partitions, boolean initialDiscoveryFinished) { + this.partitions = partitions; + this.initialDiscoveryFinished = initialDiscoveryFinished; + } + + KafkaSourceEnumState( + Set assignPartitions, + Set unassignedInitialPartitions, + boolean initialDiscoveryFinished) { + this.partitions = new HashSet<>(); + partitions.addAll( + assignPartitions.stream() + .map( + topicPartition -> + new TopicPartitionAndAssignmentStatus( + topicPartition, AssignmentStatus.ASSIGNED)) + .collect(Collectors.toSet())); + partitions.addAll( + unassignedInitialPartitions.stream() + .map( + topicPartition -> + new TopicPartitionAndAssignmentStatus( + topicPartition, + AssignmentStatus.UNASSIGNED_INITIAL)) + .collect(Collectors.toSet())); + this.initialDiscoveryFinished = initialDiscoveryFinished; + } + + public Set partitions() { + return partitions; } public Set assignedPartitions() { - return assignedPartitions; + return filterPartitionsByAssignmentStatus(AssignmentStatus.ASSIGNED); + } + + public Set unassignedInitialPartitions() { + return filterPartitionsByAssignmentStatus(AssignmentStatus.UNASSIGNED_INITIAL); + } + + public boolean initialDiscoveryFinished() { + return initialDiscoveryFinished; + } + + private Set filterPartitionsByAssignmentStatus( + AssignmentStatus assignmentStatus) { + return partitions.stream() + .filter( + partitionWithStatus -> + partitionWithStatus.assignmentStatus().equals(assignmentStatus)) + .map(TopicPartitionAndAssignmentStatus::topicPartition) + .collect(Collectors.toSet()); } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java index 7428f4205..0ea4d9f65 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java @@ -19,6 +19,7 @@ package org.apache.flink.connector.kafka.source.enumerator; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.connector.base.source.utils.SerdeUtils; import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; @@ -44,10 +45,20 @@ public class KafkaSourceEnumStateSerializer implements SimpleVersionedSerializer { + /** + * state of VERSION_0 contains splitAssignments, which is a mapping from subtask ids to lists of + * assigned splits. + */ private static final int VERSION_0 = 0; + /** state of VERSION_1 only contains assignedPartitions, which is a list of assigned splits. */ private static final int VERSION_1 = 1; + /** + * state of VERSION_2 contains initialDiscoveryFinished and partitions with different assignment + * status. + */ + private static final int VERSION_2 = 2; - private static final int CURRENT_VERSION = VERSION_1; + private static final int CURRENT_VERSION = VERSION_2; @Override public int getVersion() { @@ -56,52 +67,52 @@ public int getVersion() { @Override public byte[] serialize(KafkaSourceEnumState enumState) throws IOException { - return serializeTopicPartitions(enumState.assignedPartitions()); - } - - @Override - public KafkaSourceEnumState deserialize(int version, byte[] serialized) throws IOException { - if (version == CURRENT_VERSION) { - final Set assignedPartitions = deserializeTopicPartitions(serialized); - return new KafkaSourceEnumState(assignedPartitions); - } - - // Backward compatibility - if (version == VERSION_0) { - Map> currentPartitionAssignment = - SerdeUtils.deserializeSplitAssignments( - serialized, new KafkaPartitionSplitSerializer(), HashSet::new); - Set currentAssignedSplits = new HashSet<>(); - currentPartitionAssignment.forEach( - (reader, splits) -> - splits.forEach( - split -> currentAssignedSplits.add(split.getTopicPartition()))); - return new KafkaSourceEnumState(currentAssignedSplits); - } - - throw new IOException( - String.format( - "The bytes are serialized with version %d, " - + "while this deserializer only supports version up to %d", - version, CURRENT_VERSION)); - } - - private static byte[] serializeTopicPartitions(Collection topicPartitions) - throws IOException { + Set partitions = enumState.partitions(); + boolean initialDiscoveryFinished = enumState.initialDiscoveryFinished(); try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream out = new DataOutputStream(baos)) { - - out.writeInt(topicPartitions.size()); - for (TopicPartition tp : topicPartitions) { - out.writeUTF(tp.topic()); - out.writeInt(tp.partition()); + out.writeInt(partitions.size()); + for (TopicPartitionAndAssignmentStatus topicPartitionAndAssignmentStatus : partitions) { + out.writeUTF(topicPartitionAndAssignmentStatus.topicPartition().topic()); + out.writeInt(topicPartitionAndAssignmentStatus.topicPartition().partition()); + out.writeInt(topicPartitionAndAssignmentStatus.assignmentStatus().getStatusCode()); } + out.writeBoolean(initialDiscoveryFinished); out.flush(); - return baos.toByteArray(); } } + @Override + public KafkaSourceEnumState deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case CURRENT_VERSION: + return deserializeTopicPartitionAndAssignmentStatus(serialized); + case VERSION_1: + final Set assignedPartitions = + deserializeTopicPartitions(serialized); + return new KafkaSourceEnumState(assignedPartitions, new HashSet<>(), true); + case VERSION_0: + Map> currentPartitionAssignment = + SerdeUtils.deserializeSplitAssignments( + serialized, new KafkaPartitionSplitSerializer(), HashSet::new); + Set currentAssignedSplits = new HashSet<>(); + currentPartitionAssignment.forEach( + (reader, splits) -> + splits.forEach( + split -> + currentAssignedSplits.add( + split.getTopicPartition()))); + return new KafkaSourceEnumState(currentAssignedSplits, new HashSet<>(), true); + default: + throw new IOException( + String.format( + "The bytes are serialized with version %d, " + + "while this deserializer only supports version up to %d", + version, CURRENT_VERSION)); + } + } + private static Set deserializeTopicPartitions(byte[] serializedTopicPartitions) throws IOException { try (ByteArrayInputStream bais = new ByteArrayInputStream(serializedTopicPartitions); @@ -121,4 +132,48 @@ private static Set deserializeTopicPartitions(byte[] serializedT return topicPartitions; } } + + private static KafkaSourceEnumState deserializeTopicPartitionAndAssignmentStatus( + byte[] serialized) throws IOException { + + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + + final int numPartitions = in.readInt(); + Set partitions = new HashSet<>(numPartitions); + + for (int i = 0; i < numPartitions; i++) { + final String topic = in.readUTF(); + final int partition = in.readInt(); + final int statusCode = in.readInt(); + partitions.add( + new TopicPartitionAndAssignmentStatus( + new TopicPartition(topic, partition), + AssignmentStatus.ofStatusCode(statusCode))); + } + final boolean initialDiscoveryFinished = in.readBoolean(); + if (in.available() > 0) { + throw new IOException("Unexpected trailing bytes in serialized topic partitions"); + } + + return new KafkaSourceEnumState(partitions, initialDiscoveryFinished); + } + } + + @VisibleForTesting + public static byte[] serializeTopicPartitions(Collection topicPartitions) + throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + + out.writeInt(topicPartitions.size()); + for (TopicPartition tp : topicPartitions) { + out.writeUTF(tp.topic()); + out.writeInt(tp.partition()); + } + out.flush(); + + return baos.toByteArray(); + } + } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java index 137f42043..02323a74f 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java @@ -65,6 +65,7 @@ public class KafkaSourceEnumerator private final KafkaSubscriber subscriber; private final OffsetsInitializer startingOffsetInitializer; private final OffsetsInitializer stoppingOffsetInitializer; + private final OffsetsInitializer newDiscoveryOffsetsInitializer; private final Properties properties; private final long partitionDiscoveryIntervalMs; private final SplitEnumeratorContext context; @@ -73,6 +74,12 @@ public class KafkaSourceEnumerator /** Partitions that have been assigned to readers. */ private final Set assignedPartitions; + /** + * The partitions that have been discovered during initialization but not assigned to readers + * yet. + */ + private final Set unassignedInitialPartitions; + /** * The discovered and initialized partition splits that are waiting for owner reader to be * ready. @@ -88,6 +95,8 @@ public class KafkaSourceEnumerator // This flag will be marked as true if periodically partition discovery is disabled AND the // initializing partition discovery has finished. private boolean noMoreNewPartitionSplits = false; + // this flag will be marked as true if initial partitions are discovered after enumerator starts + private boolean initialDiscoveryFinished; public KafkaSourceEnumerator( KafkaSubscriber subscriber, @@ -103,7 +112,7 @@ public KafkaSourceEnumerator( properties, context, boundedness, - Collections.emptySet()); + new KafkaSourceEnumState(Collections.emptySet(), Collections.emptySet(), false)); } public KafkaSourceEnumerator( @@ -113,15 +122,16 @@ public KafkaSourceEnumerator( Properties properties, SplitEnumeratorContext context, Boundedness boundedness, - Set assignedPartitions) { + KafkaSourceEnumState kafkaSourceEnumState) { this.subscriber = subscriber; this.startingOffsetInitializer = startingOffsetInitializer; this.stoppingOffsetInitializer = stoppingOffsetInitializer; + this.newDiscoveryOffsetsInitializer = OffsetsInitializer.earliest(); this.properties = properties; this.context = context; this.boundedness = boundedness; - this.assignedPartitions = new HashSet<>(assignedPartitions); + this.assignedPartitions = new HashSet<>(kafkaSourceEnumState.assignedPartitions()); this.pendingPartitionSplitAssignment = new HashMap<>(); this.partitionDiscoveryIntervalMs = KafkaSourceOptions.getOption( @@ -129,6 +139,9 @@ public KafkaSourceEnumerator( KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS, Long::parseLong); this.consumerGroupId = properties.getProperty(ConsumerConfig.GROUP_ID_CONFIG); + this.unassignedInitialPartitions = + new HashSet<>(kafkaSourceEnumState.unassignedInitialPartitions()); + this.initialDiscoveryFinished = kafkaSourceEnumState.initialDiscoveryFinished(); } /** @@ -195,7 +208,8 @@ public void addReader(int subtaskId) { @Override public KafkaSourceEnumState snapshotState(long checkpointId) throws Exception { - return new KafkaSourceEnumState(assignedPartitions); + return new KafkaSourceEnumState( + assignedPartitions, unassignedInitialPartitions, initialDiscoveryFinished); } @Override @@ -234,6 +248,12 @@ private void checkPartitionChanges(Set fetchedPartitions, Throwa throw new FlinkRuntimeException( "Failed to list subscribed topic partitions due to ", t); } + + if (!initialDiscoveryFinished) { + unassignedInitialPartitions.addAll(fetchedPartitions); + initialDiscoveryFinished = true; + } + final PartitionChange partitionChange = getPartitionChange(fetchedPartitions); if (partitionChange.isEmpty()) { return; @@ -266,10 +286,18 @@ private void checkPartitionChanges(Set fetchedPartitions, Throwa private PartitionSplitChange initializePartitionSplits(PartitionChange partitionChange) { Set newPartitions = Collections.unmodifiableSet(partitionChange.getNewPartitions()); + OffsetsInitializer.PartitionOffsetsRetriever offsetsRetriever = getOffsetsRetriever(); + // initial partitions use OffsetsInitializer specified by the user while new partitions use + // EARLIEST + Map startingOffsets = new HashMap<>(); + startingOffsets.putAll( + newDiscoveryOffsetsInitializer.getPartitionOffsets( + newPartitions, offsetsRetriever)); + startingOffsets.putAll( + startingOffsetInitializer.getPartitionOffsets( + unassignedInitialPartitions, offsetsRetriever)); - Map startingOffsets = - startingOffsetInitializer.getPartitionOffsets(newPartitions, offsetsRetriever); Map stoppingOffsets = stoppingOffsetInitializer.getPartitionOffsets(newPartitions, offsetsRetriever); @@ -344,7 +372,10 @@ private void assignPendingPartitionSplits(Set pendingReaders) { // Mark pending partitions as already assigned pendingAssignmentForReader.forEach( - split -> assignedPartitions.add(split.getTopicPartition())); + split -> { + assignedPartitions.add(split.getTopicPartition()); + unassignedInitialPartitions.remove(split.getTopicPartition()); + }); } } @@ -539,9 +570,9 @@ public Map committedOffsets(Collection par * the beginning offset, end offset as well as the offset matching a timestamp in * partitions. * - * @see KafkaAdminClient#listOffsets(Map) * @param topicPartitionOffsets The mapping from partition to the OffsetSpec to look up. * @return The list offsets result. + * @see KafkaAdminClient#listOffsets(Map) */ private Map listOffsets( Map topicPartitionOffsets) { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/TopicPartitionAndAssignmentStatus.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/TopicPartitionAndAssignmentStatus.java new file mode 100644 index 000000000..2caed99b7 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/TopicPartitionAndAssignmentStatus.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.source.enumerator; + +import org.apache.flink.annotation.Internal; + +import org.apache.kafka.common.TopicPartition; + +/** Kafka partition with assign status. */ +@Internal +public class TopicPartitionAndAssignmentStatus { + private final TopicPartition topicPartition; + private final AssignmentStatus assignmentStatus; + + public TopicPartitionAndAssignmentStatus( + TopicPartition topicPartition, AssignmentStatus assignStatus) { + this.topicPartition = topicPartition; + this.assignmentStatus = assignStatus; + } + + public TopicPartition topicPartition() { + return topicPartition; + } + + public AssignmentStatus assignmentStatus() { + return assignmentStatus; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java index 8d0d3fc11..3ced33f98 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SplitsAssignment; import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.apache.flink.connector.kafka.source.KafkaSourceOptions; import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; @@ -50,6 +51,7 @@ import java.util.Set; import java.util.StringJoiner; import java.util.regex.Pattern; +import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; @@ -64,6 +66,7 @@ public class KafkaEnumeratorTest { private static final int READER0 = 0; private static final int READER1 = 1; + private static final int READER2 = 2; private static final Set PRE_EXISTING_TOPICS = new HashSet<>(Arrays.asList(TOPIC1, TOPIC2)); private static final int PARTITION_DISCOVERY_CALLABLE_INDEX = 0; @@ -254,7 +257,8 @@ public void testDiscoverPartitionsPeriodically() throws Throwable { createEnumerator( context, ENABLE_PERIODIC_PARTITION_DISCOVERY, - INCLUDE_DYNAMIC_TOPIC); + INCLUDE_DYNAMIC_TOPIC, + OffsetsInitializer.latest()); AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) { startEnumeratorAndRegisterReaders(context, enumerator); @@ -290,6 +294,19 @@ public void testDiscoverPartitionsPeriodically() throws Throwable { Arrays.asList(READER0, READER1), Collections.singleton(DYNAMIC_TOPIC_NAME), 3); + + // new partitions use EARLIEST_OFFSET, while initial partitions use LATEST_OFFSET + List initialPartitionAssign = + getAllAssignSplits(context, PRE_EXISTING_TOPICS); + assertThat(initialPartitionAssign) + .extracting(KafkaPartitionSplit::getStartingOffset) + .containsOnly(KafkaPartitionSplit.LATEST_OFFSET); + List newPartitionAssign = + getAllAssignSplits(context, Collections.singleton(DYNAMIC_TOPIC_NAME)); + assertThat(newPartitionAssign) + .extracting(KafkaPartitionSplit::getStartingOffset) + .containsOnly(KafkaPartitionSplit.EARLIEST_OFFSET); + } finally { try (AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) { adminClient.deleteTopics(Collections.singleton(DYNAMIC_TOPIC_NAME)).all().get(); @@ -342,8 +359,11 @@ public void testWorkWithPreexistingAssignments() throws Throwable { createEnumerator( context2, ENABLE_PERIODIC_PARTITION_DISCOVERY ? 1 : -1, + OffsetsInitializer.earliest(), PRE_EXISTING_TOPICS, preexistingAssignments, + Collections.emptySet(), + true, new Properties())) { enumerator.start(); runPeriodicPartitionDiscovery(context2); @@ -371,8 +391,11 @@ public void testKafkaClientProperties() throws Exception { createEnumerator( context, ENABLE_PERIODIC_PARTITION_DISCOVERY ? 1 : -1, + OffsetsInitializer.earliest(), PRE_EXISTING_TOPICS, Collections.emptySet(), + Collections.emptySet(), + false, properties)) { enumerator.start(); @@ -395,20 +418,49 @@ public void testSnapshotState() throws Throwable { KafkaSourceEnumerator enumerator = createEnumerator(context, false)) { enumerator.start(); - // No reader is registered, so the state should be empty + // Step1: Before first discovery, so the state should be empty final KafkaSourceEnumState state1 = enumerator.snapshotState(1L); assertThat(state1.assignedPartitions()).isEmpty(); + assertThat(state1.unassignedInitialPartitions()).isEmpty(); + assertThat(state1.initialDiscoveryFinished()).isFalse(); registerReader(context, enumerator, READER0); registerReader(context, enumerator, READER1); - runOneTimePartitionDiscovery(context); - // The state should contain splits assigned to READER0 and READER1 - final KafkaSourceEnumState state2 = enumerator.snapshotState(1L); + // Step2: First partition discovery after start, but no assignments to readers + context.runNextOneTimeCallable(); + final KafkaSourceEnumState state2 = enumerator.snapshotState(2L); + assertThat(state2.assignedPartitions()).isEmpty(); + assertThat(state2.unassignedInitialPartitions()).isNotEmpty(); + assertThat(state2.initialDiscoveryFinished()).isTrue(); + + // Step3: Assign partials partitions to reader0 and reader1 + context.runNextOneTimeCallable(); + + // The state should contain splits assigned to READER0 and READER1, but no READER2 + // register. + // Thus, both assignedPartitions and unassignedInitialPartitions are not empty. + final KafkaSourceEnumState state3 = enumerator.snapshotState(3L); verifySplitAssignmentWithPartitions( getExpectedAssignments( new HashSet<>(Arrays.asList(READER0, READER1)), PRE_EXISTING_TOPICS), - state2.assignedPartitions()); + state3.assignedPartitions()); + assertThat(state3.unassignedInitialPartitions()).isNotEmpty(); + assertThat(state3.initialDiscoveryFinished()).isTrue(); + // total partitions of state2 and state3 are equal + // state2 only includes unassignedInitialPartitions + // state3 includes unassignedInitialPartitions + assignedPartitions + Set allPartitionOfState3 = new HashSet<>(); + allPartitionOfState3.addAll(state3.unassignedInitialPartitions()); + allPartitionOfState3.addAll(state3.assignedPartitions()); + assertThat(state2.unassignedInitialPartitions()).isEqualTo(allPartitionOfState3); + + // Step4: register READER2, then all partitions are assigned + registerReader(context, enumerator, READER2); + final KafkaSourceEnumState state4 = enumerator.snapshotState(4L); + assertThat(state4.assignedPartitions()).isEqualTo(allPartitionOfState3); + assertThat(state4.unassignedInitialPartitions()).isEmpty(); + assertThat(state4.initialDiscoveryFinished()).isTrue(); } } @@ -480,19 +532,27 @@ private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, boolean enablePeriodicPartitionDiscovery) { return createEnumerator( - enumContext, enablePeriodicPartitionDiscovery, EXCLUDE_DYNAMIC_TOPIC); + enumContext, + enablePeriodicPartitionDiscovery, + EXCLUDE_DYNAMIC_TOPIC, + OffsetsInitializer.earliest()); } private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, long partitionDiscoveryIntervalMs) { - return createEnumerator(enumContext, partitionDiscoveryIntervalMs, EXCLUDE_DYNAMIC_TOPIC); + return createEnumerator( + enumContext, + partitionDiscoveryIntervalMs, + EXCLUDE_DYNAMIC_TOPIC, + OffsetsInitializer.earliest()); } private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, boolean enablePeriodicPartitionDiscovery, - boolean includeDynamicTopic) { + boolean includeDynamicTopic, + OffsetsInitializer startingOffsetsInitializer) { List topics = new ArrayList<>(PRE_EXISTING_TOPICS); if (includeDynamicTopic) { topics.add(DYNAMIC_TOPIC_NAME); @@ -500,15 +560,19 @@ private KafkaSourceEnumerator createEnumerator( return createEnumerator( enumContext, enablePeriodicPartitionDiscovery ? 1 : -1, + startingOffsetsInitializer, topics, Collections.emptySet(), + Collections.emptySet(), + false, new Properties()); } private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, long partitionDiscoveryIntervalMs, - boolean includeDynamicTopic) { + boolean includeDynamicTopic, + OffsetsInitializer startingOffsetsInitializer) { List topics = new ArrayList<>(PRE_EXISTING_TOPICS); if (includeDynamicTopic) { topics.add(DYNAMIC_TOPIC_NAME); @@ -516,8 +580,11 @@ private KafkaSourceEnumerator createEnumerator( return createEnumerator( enumContext, partitionDiscoveryIntervalMs, + startingOffsetsInitializer, topics, Collections.emptySet(), + Collections.emptySet(), + false, new Properties()); } @@ -528,8 +595,11 @@ private KafkaSourceEnumerator createEnumerator( private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, long partitionDiscoveryIntervalMs, + OffsetsInitializer startingOffsetsInitializer, Collection topicsToSubscribe, Set assignedPartitions, + Set unassignedInitialPartitions, + boolean initialDiscoveryFinished, Properties overrideProperties) { // Use a TopicPatternSubscriber so that no exception if a subscribed topic hasn't been // created yet. @@ -538,7 +608,6 @@ private KafkaSourceEnumerator createEnumerator( Pattern topicPattern = Pattern.compile(topicNameJoiner.toString()); KafkaSubscriber subscriber = KafkaSubscriber.getTopicPatternSubscriber(topicPattern); - OffsetsInitializer startingOffsetsInitializer = OffsetsInitializer.earliest(); OffsetsInitializer stoppingOffsetsInitializer = new NoStoppingOffsetsInitializer(); Properties props = @@ -555,7 +624,8 @@ private KafkaSourceEnumerator createEnumerator( props, enumContext, Boundedness.CONTINUOUS_UNBOUNDED, - assignedPartitions); + new KafkaSourceEnumState( + assignedPartitions, unassignedInitialPartitions, initialDiscoveryFinished)); } // --------------------- @@ -632,6 +702,25 @@ private void verifySplitAssignmentWithPartitions( assertThat(actualTopicPartitions).isEqualTo(allTopicPartitionsFromAssignment); } + /** get all assigned partition splits of topics. */ + private List getAllAssignSplits( + MockSplitEnumeratorContext context, Set topics) { + + List allSplits = new ArrayList<>(); + List> splitsAssignmentSequence = + context.getSplitsAssignmentSequence(); + for (SplitsAssignment splitsAssignment : splitsAssignmentSequence) { + List splitsOfOnceAssignment = + splitsAssignment.assignment().values().stream() + .flatMap(splits -> splits.stream()) + .filter(split -> topics.contains(split.getTopic())) + .collect(Collectors.toList()); + allSplits.addAll(splitsOfOnceAssignment); + } + + return allSplits; + } + private Set asEnumState(Map> assignments) { Set enumState = new HashSet<>(); assignments.forEach( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java index 058d2a095..6c172e4a2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java @@ -44,7 +44,11 @@ public class KafkaSourceEnumStateSerializerTest { @Test public void testEnumStateSerde() throws IOException { - final KafkaSourceEnumState state = new KafkaSourceEnumState(constructTopicPartitions()); + final KafkaSourceEnumState state = + new KafkaSourceEnumState( + constructTopicPartitions(0), + constructTopicPartitions(NUM_PARTITIONS_PER_TOPIC), + true); final KafkaSourceEnumStateSerializer serializer = new KafkaSourceEnumStateSerializer(); final byte[] bytes = serializer.serialize(state); @@ -53,37 +57,53 @@ public void testEnumStateSerde() throws IOException { serializer.deserialize(serializer.getVersion(), bytes); assertThat(restoredState.assignedPartitions()).isEqualTo(state.assignedPartitions()); + assertThat(restoredState.unassignedInitialPartitions()) + .isEqualTo(state.unassignedInitialPartitions()); + assertThat(restoredState.initialDiscoveryFinished()).isTrue(); } @Test public void testBackwardCompatibility() throws IOException { - final Set topicPartitions = constructTopicPartitions(); + final Set topicPartitions = constructTopicPartitions(0); final Map> splitAssignments = toSplitAssignments(topicPartitions); // Create bytes in the way of KafkaEnumStateSerializer version 0 doing serialization - final byte[] bytes = + final byte[] bytesV0 = SerdeUtils.serializeSplitAssignments( splitAssignments, new KafkaPartitionSplitSerializer()); + // Create bytes in the way of KafkaEnumStateSerializer version 1 doing serialization + final byte[] bytesV1 = + KafkaSourceEnumStateSerializer.serializeTopicPartitions(topicPartitions); - // Deserialize above bytes with KafkaEnumStateSerializer version 1 to check backward + // Deserialize above bytes with KafkaEnumStateSerializer version 2 to check backward // compatibility - final KafkaSourceEnumState kafkaSourceEnumState = - new KafkaSourceEnumStateSerializer().deserialize(0, bytes); - - assertThat(kafkaSourceEnumState.assignedPartitions()).isEqualTo(topicPartitions); + final KafkaSourceEnumState kafkaSourceEnumStateV0 = + new KafkaSourceEnumStateSerializer().deserialize(0, bytesV0); + final KafkaSourceEnumState kafkaSourceEnumStateV1 = + new KafkaSourceEnumStateSerializer().deserialize(1, bytesV1); + + assertThat(kafkaSourceEnumStateV0.assignedPartitions()).isEqualTo(topicPartitions); + assertThat(kafkaSourceEnumStateV0.unassignedInitialPartitions()).isEmpty(); + assertThat(kafkaSourceEnumStateV0.initialDiscoveryFinished()).isTrue(); + + assertThat(kafkaSourceEnumStateV1.assignedPartitions()).isEqualTo(topicPartitions); + assertThat(kafkaSourceEnumStateV1.unassignedInitialPartitions()).isEmpty(); + assertThat(kafkaSourceEnumStateV1.initialDiscoveryFinished()).isTrue(); } - private Set constructTopicPartitions() { + private Set constructTopicPartitions(int startPartition) { // Create topic partitions for readers. // Reader i will be assigned with NUM_PARTITIONS_PER_TOPIC splits, with topic name // "topic-{i}" and - // NUM_PARTITIONS_PER_TOPIC partitions. + // NUM_PARTITIONS_PER_TOPIC partitions. The starting partition number is startPartition // Totally NUM_READERS * NUM_PARTITIONS_PER_TOPIC partitions will be created. Set topicPartitions = new HashSet<>(); for (int readerId = 0; readerId < NUM_READERS; readerId++) { - for (int partition = 0; partition < NUM_PARTITIONS_PER_TOPIC; partition++) { + for (int partition = startPartition; + partition < startPartition + NUM_PARTITIONS_PER_TOPIC; + partition++) { topicPartitions.add(new TopicPartition(TOPIC_PREFIX + readerId, partition)); } } From 14d8462be33c25031e94da0635105f69bb807641 Mon Sep 17 00:00:00 2001 From: Hongshun Wang <125648852+loserwang1024@users.noreply.github.com> Date: Tue, 11 Jul 2023 10:24:13 +0800 Subject: [PATCH 220/322] [FLINK-32021][kafka] Improve the java docs of SpecifiedOffsetsInitializer and TimestampOffsetsInitializer This closes #29. --- .../enumerator/initializer/SpecifiedOffsetsInitializer.java | 5 +++++ .../enumerator/initializer/TimestampOffsetsInitializer.java | 2 ++ 2 files changed, 7 insertions(+) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java index 5766a5fc1..e402435bd 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java @@ -38,6 +38,11 @@ * An implementation of {@link OffsetsInitializer} which initializes the offsets of the partition * according to the user specified offsets. * + *

Use specified offsets for specified partitions while use commit offsets or offsetResetStrategy + * for unspecified partitions. Specified partition's offset should be less than its latest offset, + * otherwise it will start from the offsetResetStrategy. The default value of offsetResetStrategy is + * earliest. + * *

Package private and should be instantiated via {@link OffsetsInitializer}. */ class SpecifiedOffsetsInitializer implements OffsetsInitializer, OffsetsInitializerValidator { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java index f411307f6..0ad53c9b1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java @@ -28,6 +28,8 @@ /** * An implementation of {@link OffsetsInitializer} to initialize the offsets based on a timestamp. + * If the message meeting the requirement of the timestamp have not been produced to Kafka yet, just + * use the latest offset. * *

Package private and should be instantiated via {@link OffsetsInitializer}. */ From 97dbfcadee982eaae3800f478cf6ba857de1b2db Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 4 Jul 2023 13:04:52 +0800 Subject: [PATCH 221/322] [FLINK-32455] Copy over TypeSerializerUpgradeTestBase due to breaking change in Flink 1.18 This is a temporary workaround for a breaking change that occured on the TypeSerializerUpgradeTestBase due to FLINK-27518. A proper fix would be to properly introduce a public-facing test utility to replace TypeSerializerUpgradeTestBase, and move the Kafka connector test code to use that instead. --- .../kafka/KafkaSerializerUpgradeTest.java | 2 +- .../TypeSerializerUpgradeTestBase.java | 603 ++++++++++++++++++ .../serializer-snapshot | Bin 0 -> 126 bytes .../context-state-serializer-1.17/test-data | Bin 0 -> 19 bytes .../serializer-snapshot | Bin 0 -> 134 bytes .../test-data | Bin 0 -> 17 bytes 6 files changed, 604 insertions(+), 1 deletion(-) create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java create mode 100644 flink-connector-kafka/src/test/resources/context-state-serializer-1.17/serializer-snapshot create mode 100644 flink-connector-kafka/src/test/resources/context-state-serializer-1.17/test-data create mode 100644 flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/serializer-snapshot create mode 100644 flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/test-data diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java index 0d89d39b8..c9e82be5d 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java @@ -22,8 +22,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerMatchers; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; -import org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase; import org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer; +import org.apache.flink.streaming.connectors.kafka.testutils.TypeSerializerUpgradeTestBase; import org.hamcrest.Matcher; import org.mockito.Mockito; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java new file mode 100644 index 000000000..90e218382 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java @@ -0,0 +1,603 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.testutils; + +import org.apache.flink.FlinkVersion; +import org.apache.flink.api.common.typeutils.ClassRelocator; +import org.apache.flink.api.common.typeutils.ThreadContextClassLoader; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerMatchers; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; + +import org.assertj.core.api.AssertionsForClassTypes; +import org.assertj.core.api.HamcrestCondition; +import org.hamcrest.Matcher; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collection; +import java.util.Set; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; +import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.hamcrest.CoreMatchers.not; + +/** + * A test base for testing {@link TypeSerializer} upgrades. + * + *

You can run {@link #generateTestSetupFiles(TestSpecification)} on a Flink branch to + * (re-)generate the test data files. + */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class TypeSerializerUpgradeTestBase { + + public static final FlinkVersion CURRENT_VERSION = FlinkVersion.v1_17; + + public static final Set MIGRATION_VERSIONS = + FlinkVersion.rangeOf(FlinkVersion.v1_11, CURRENT_VERSION); + + // ------------------------------------------------------------------------------ + // APIs + // ------------------------------------------------------------------------------ + + /** + * Creates a collection of {@link TestSpecification} which will be used as input for + * parametrized tests. + */ + public abstract Collection> createTestSpecifications() throws Exception; + + /** + * Setup code for a {@link TestSpecification}. This creates the serializer before upgrade and + * test data, that will be written by the created pre-upgrade {@link TypeSerializer}. + */ + public interface PreUpgradeSetup { + + /** Creates a pre-upgrade {@link TypeSerializer}. */ + TypeSerializer createPriorSerializer(); + + /** Creates test data that will be written using the pre-upgrade {@link TypeSerializer}. */ + PreviousElementT createTestData(); + } + + /** + * Verification code for a {@link TestSpecification}. This creates the "upgraded" {@link + * TypeSerializer} and provides matchers for comparing the deserialized test data and for the + * {@link TypeSerializerSchemaCompatibility}. + */ + public interface UpgradeVerifier { + + /** Creates a post-upgrade {@link TypeSerializer}. */ + TypeSerializer createUpgradedSerializer(); + + /** Returns a {@link Matcher} for asserting the deserialized test data. */ + Matcher testDataMatcher(); + + /** + * Returns a {@link Matcher} for comparing the {@link TypeSerializerSchemaCompatibility} + * that the serializer upgrade produced with an expected {@link + * TypeSerializerSchemaCompatibility}. + */ + Matcher> schemaCompatibilityMatcher( + FlinkVersion version); + } + + private static class ClassLoaderSafePreUpgradeSetup + implements PreUpgradeSetup { + + private final PreUpgradeSetup delegateSetup; + private final ClassLoader setupClassloader; + + ClassLoaderSafePreUpgradeSetup( + Class> delegateSetupClass) + throws Exception { + checkNotNull(delegateSetupClass); + Class> relocatedDelegateSetupClass = + ClassRelocator.relocate(delegateSetupClass); + + this.setupClassloader = relocatedDelegateSetupClass.getClassLoader(); + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(setupClassloader)) { + this.delegateSetup = relocatedDelegateSetupClass.newInstance(); + } + } + + @Override + public TypeSerializer createPriorSerializer() { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(setupClassloader)) { + return delegateSetup.createPriorSerializer(); + } catch (IOException e) { + throw new RuntimeException( + "Error creating prior serializer via ClassLoaderSafePreUpgradeSetup.", e); + } + } + + @Override + public PreviousElementT createTestData() { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(setupClassloader)) { + return delegateSetup.createTestData(); + } catch (IOException e) { + throw new RuntimeException( + "Error creating test data via ThreadContextClassLoader.", e); + } + } + } + + private static class ClassLoaderSafeUpgradeVerifier + implements UpgradeVerifier { + + private final UpgradeVerifier delegateVerifier; + private final ClassLoader verifierClassloader; + + ClassLoaderSafeUpgradeVerifier( + Class> delegateVerifierClass) + throws Exception { + checkNotNull(delegateVerifierClass); + Class> relocatedDelegateVerifierClass = + ClassRelocator.relocate(delegateVerifierClass); + + this.verifierClassloader = relocatedDelegateVerifierClass.getClassLoader(); + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(verifierClassloader)) { + this.delegateVerifier = relocatedDelegateVerifierClass.newInstance(); + } + } + + @Override + public TypeSerializer createUpgradedSerializer() { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(verifierClassloader)) { + return delegateVerifier.createUpgradedSerializer(); + } catch (IOException e) { + throw new RuntimeException( + "Error creating upgraded serializer via ClassLoaderSafeUpgradeVerifier.", + e); + } + } + + @Override + public Matcher testDataMatcher() { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(verifierClassloader)) { + return delegateVerifier.testDataMatcher(); + } catch (IOException e) { + throw new RuntimeException( + "Error creating expected test data via ClassLoaderSafeUpgradeVerifier.", e); + } + } + + @Override + public Matcher> + schemaCompatibilityMatcher(FlinkVersion version) { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(verifierClassloader)) { + return delegateVerifier.schemaCompatibilityMatcher(version); + } catch (IOException e) { + throw new RuntimeException( + "Error creating schema compatibility matcher via ClassLoaderSafeUpgradeVerifier.", + e); + } + } + } + + /** + * Specification of one test scenario. This mainly needs a {@link PreUpgradeSetup} and {@link + * UpgradeVerifier}. + */ + public static class TestSpecification { + private final String name; + private final FlinkVersion flinkVersion; + private final ClassLoaderSafePreUpgradeSetup setup; + private final ClassLoaderSafeUpgradeVerifier verifier; + + public TestSpecification( + String name, + FlinkVersion flinkVersion, + Class> setupClass, + Class> verifierClass) + throws Exception { + this.name = checkNotNull(name); + this.flinkVersion = checkNotNull(flinkVersion); + this.setup = new ClassLoaderSafePreUpgradeSetup<>(setupClass); + this.verifier = new ClassLoaderSafeUpgradeVerifier<>(verifierClass); + } + + @Override + public String toString() { + return name + " / " + flinkVersion; + } + } + + // ------------------------------------------------------------------------------ + // Test file generation + // ------------------------------------------------------------------------------ + + private static final int INITIAL_OUTPUT_BUFFER_SIZE = 64; + + /** + * Execute this test to generate test files. Remember to be using the correct branch when + * generating the test files, e.g. to generate test files for {@link FlinkVersion#v1_8}, you + * should be under the release-1.8 branch. + */ + @Disabled + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void generateTestSetupFiles( + TestSpecification testSpecification) + throws Exception { + Files.createDirectories(getSerializerSnapshotFilePath(testSpecification).getParent()); + + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.setup.setupClassloader)) { + TypeSerializer priorSerializer = + testSpecification.setup.createPriorSerializer(); + + // first, use the serializer to write test data + // NOTE: it is important that we write test data first, because some serializers' + // configuration + // mutates only after being used for serialization (e.g. dynamic type + // registrations for Pojo / Kryo) + DataOutputSerializer testDataOut = new DataOutputSerializer(INITIAL_OUTPUT_BUFFER_SIZE); + priorSerializer.serialize(testSpecification.setup.createTestData(), testDataOut); + writeContentsTo( + getGenerateDataFilePath(testSpecification), testDataOut.getCopyOfBuffer()); + + // ... then write the serializer snapshot + DataOutputSerializer serializerSnapshotOut = + new DataOutputSerializer(INITIAL_OUTPUT_BUFFER_SIZE); + writeSerializerSnapshot(serializerSnapshotOut, priorSerializer, CURRENT_VERSION); + writeContentsTo( + getGenerateSerializerSnapshotFilePath(testSpecification), + serializerSnapshotOut.getCopyOfBuffer()); + } + } + + // ------------------------------------------------------------------------------ + // Tests + // ------------------------------------------------------------------------------ + + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void restoreSerializerIsValid( + TestSpecification testSpecification) + throws Exception { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.verifier.verifierClassloader)) { + assumeThat(TypeSerializerSchemaCompatibility.incompatible()) + .as( + "This test only applies for test specifications that verify an upgraded serializer that is not incompatible.") + .is( + HamcrestCondition.matching( + not( + testSpecification.verifier.schemaCompatibilityMatcher( + testSpecification.flinkVersion)))); + + TypeSerializerSnapshot restoredSerializerSnapshot = + snapshotUnderTest(testSpecification); + + TypeSerializer restoredSerializer = + restoredSerializerSnapshot.restoreSerializer(); + assertSerializerIsValid( + restoredSerializer, + dataUnderTest(testSpecification), + testSpecification.verifier.testDataMatcher()); + } + } + + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void upgradedSerializerHasExpectedSchemaCompatibility( + TestSpecification testSpecification) + throws Exception { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.verifier.verifierClassloader)) { + TypeSerializerSnapshot restoredSerializerSnapshot = + snapshotUnderTest(testSpecification); + TypeSerializer upgradedSerializer = + testSpecification.verifier.createUpgradedSerializer(); + + TypeSerializerSchemaCompatibility upgradeCompatibility = + restoredSerializerSnapshot.resolveSchemaCompatibility(upgradedSerializer); + + AssertionsForClassTypes.assertThat(upgradeCompatibility) + .is( + HamcrestCondition.matching( + testSpecification.verifier.schemaCompatibilityMatcher( + testSpecification.flinkVersion))); + } + } + + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void upgradedSerializerIsValidAfterMigration( + TestSpecification testSpecification) + throws Exception { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.verifier.verifierClassloader)) { + TypeSerializerSnapshot restoredSerializerSnapshot = + snapshotUnderTest(testSpecification); + + TypeSerializer upgradedSerializer = + testSpecification.verifier.createUpgradedSerializer(); + + TypeSerializerSchemaCompatibility upgradeCompatibility = + restoredSerializerSnapshot.resolveSchemaCompatibility(upgradedSerializer); + assumeThat(upgradeCompatibility) + .as( + "This test only applies for test specifications that verify an upgraded serializer that requires migration to be compatible.") + .is( + HamcrestCondition.matching( + TypeSerializerMatchers.isCompatibleAfterMigration())); + + // migrate the previous data schema, + TypeSerializer restoreSerializer = + restoredSerializerSnapshot.restoreSerializer(); + DataInputView migratedData = + readAndThenWriteData( + dataUnderTest(testSpecification), + restoreSerializer, + upgradedSerializer, + testSpecification.verifier.testDataMatcher()); + + // .. and then assert that the upgraded serializer is valid with the migrated data + assertSerializerIsValid( + upgradedSerializer, migratedData, testSpecification.verifier.testDataMatcher()); + } + } + + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void upgradedSerializerIsValidAfterReconfiguration( + TestSpecification testSpecification) + throws Exception { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.verifier.verifierClassloader)) { + TypeSerializerSnapshot restoredSerializerSnapshot = + snapshotUnderTest(testSpecification); + TypeSerializer upgradedSerializer = + testSpecification.verifier.createUpgradedSerializer(); + + TypeSerializerSchemaCompatibility upgradeCompatibility = + restoredSerializerSnapshot.resolveSchemaCompatibility(upgradedSerializer); + assumeThat(upgradeCompatibility) + .as( + "This test only applies for test specifications that verify an upgraded serializer that requires reconfiguration to be compatible.") + .is( + HamcrestCondition.matching( + TypeSerializerMatchers + .isCompatibleWithReconfiguredSerializer())); + + TypeSerializer reconfiguredUpgradedSerializer = + upgradeCompatibility.getReconfiguredSerializer(); + assertSerializerIsValid( + reconfiguredUpgradedSerializer, + dataUnderTest(testSpecification), + testSpecification.verifier.testDataMatcher()); + } + } + + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void upgradedSerializerIsValidWhenCompatibleAsIs( + TestSpecification testSpecification) + throws Exception { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.verifier.verifierClassloader)) { + TypeSerializerSnapshot restoredSerializerSnapshot = + snapshotUnderTest(testSpecification); + TypeSerializer upgradedSerializer = + testSpecification.verifier.createUpgradedSerializer(); + + TypeSerializerSchemaCompatibility upgradeCompatibility = + restoredSerializerSnapshot.resolveSchemaCompatibility(upgradedSerializer); + assumeThat(upgradeCompatibility) + .as( + "This test only applies for test specifications that verify an upgraded serializer that is compatible as is.") + .is(HamcrestCondition.matching(TypeSerializerMatchers.isCompatibleAsIs())); + + assertSerializerIsValid( + upgradedSerializer, + dataUnderTest(testSpecification), + testSpecification.verifier.testDataMatcher()); + } + } + + /** + * Asserts that a given {@link TypeSerializer} is valid, given a {@link DataInputView} of + * serialized data. + * + *

A serializer is valid, iff: + * + *

    + *
  • 1. The serializer can read and then write again the given serialized data. + *
  • 2. The serializer can produce a serializer snapshot which can be written and then read + * back again. + *
  • 3. The serializer's produced snapshot is capable of creating a restore serializer. + *
  • 4. The restore serializer created from the serializer snapshot can read and then write + * again data written by step 1. Given that the serializer is not a restore serializer + * already. + *
+ */ + private static void assertSerializerIsValid( + TypeSerializer serializer, DataInputView dataInput, Matcher testDataMatcher) + throws Exception { + + DataInputView serializedData = + readAndThenWriteData(dataInput, serializer, serializer, testDataMatcher); + TypeSerializerSnapshot snapshot = writeAndThenReadSerializerSnapshot(serializer); + TypeSerializer restoreSerializer = snapshot.restoreSerializer(); + serializedData = + readAndThenWriteData( + serializedData, restoreSerializer, restoreSerializer, testDataMatcher); + + TypeSerializer duplicateSerializer = snapshot.restoreSerializer().duplicate(); + readAndThenWriteData( + serializedData, duplicateSerializer, duplicateSerializer, testDataMatcher); + } + + // ------------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------------ + + /** Paths to use during snapshot generation, which should only use the CURRENT_VERSION. */ + private Path getGenerateSerializerSnapshotFilePath( + TestSpecification testSpecification) { + return Paths.get(getGenerateResourceDirectory(testSpecification) + "/serializer-snapshot"); + } + + /** Paths to use during snapshot generation, which should only use the CURRENT_VERSION. */ + private Path getGenerateDataFilePath( + TestSpecification testSpecification) { + return Paths.get(getGenerateResourceDirectory(testSpecification) + "/test-data"); + } + + /** Paths to use during snapshot generation, which should only use the CURRENT_VERSION. */ + private String getGenerateResourceDirectory( + TestSpecification testSpecification) { + return System.getProperty("user.dir") + + "/src/test/resources/" + + testSpecification.name + + "-" + + CURRENT_VERSION; + } + + private Path getSerializerSnapshotFilePath( + TestSpecification testSpecification) { + return Paths.get(getTestResourceDirectory(testSpecification) + "/serializer-snapshot"); + } + + private Path getTestDataFilePath( + TestSpecification testSpecification) { + return Paths.get(getTestResourceDirectory(testSpecification) + "/test-data"); + } + + private String getTestResourceDirectory( + TestSpecification testSpecification) { + return System.getProperty("user.dir") + + "/src/test/resources/" + + testSpecification.name + + "-" + + testSpecification.flinkVersion; + } + + private TypeSerializerSnapshot snapshotUnderTest( + TestSpecification testSpecification) + throws Exception { + return readSerializerSnapshot( + contentsOf(getSerializerSnapshotFilePath(testSpecification)), + testSpecification.flinkVersion); + } + + private DataInputView dataUnderTest( + TestSpecification testSpecification) { + return contentsOf(getTestDataFilePath(testSpecification)); + } + + private static void writeContentsTo(Path path, byte[] bytes) { + try { + Files.write(path, bytes); + } catch (IOException e) { + throw new RuntimeException("Failed to write to " + path, e); + } + } + + private static DataInputView contentsOf(Path path) { + try { + byte[] bytes = Files.readAllBytes(path); + return new DataInputDeserializer(bytes); + } catch (IOException e) { + throw new RuntimeException("Failed to read contents of " + path, e); + } + } + + private static void writeSerializerSnapshot( + DataOutputView out, TypeSerializer serializer, FlinkVersion flinkVersion) + throws IOException { + + if (flinkVersion.isNewerVersionThan(FlinkVersion.v1_6)) { + writeSerializerSnapshotCurrentFormat(out, serializer); + } else { + throw new UnsupportedOperationException( + "There should be no longer a need to support/use this path since Flink 1.17"); + } + } + + private static void writeSerializerSnapshotCurrentFormat( + DataOutputView out, TypeSerializer serializer) throws IOException { + + TypeSerializerSnapshotSerializationUtil.writeSerializerSnapshot( + out, serializer.snapshotConfiguration()); + } + + private static TypeSerializerSnapshot readSerializerSnapshot( + DataInputView in, FlinkVersion flinkVersion) throws IOException { + + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + checkState(flinkVersion.isNewerVersionThan(FlinkVersion.v1_6)); + return readSerializerSnapshotCurrentFormat(in, classLoader); + } + + private static TypeSerializerSnapshot readSerializerSnapshotCurrentFormat( + DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + + return TypeSerializerSnapshotSerializationUtil.readSerializerSnapshot( + in, userCodeClassLoader); + } + + private static DataInputView readAndThenWriteData( + DataInputView originalDataInput, + TypeSerializer readSerializer, + TypeSerializer writeSerializer, + Matcher testDataMatcher) + throws IOException { + + T data = readSerializer.deserialize(originalDataInput); + assertThat(data).is(HamcrestCondition.matching(testDataMatcher)); + + DataOutputSerializer out = new DataOutputSerializer(INITIAL_OUTPUT_BUFFER_SIZE); + writeSerializer.serialize(data, out); + return new DataInputDeserializer(out.wrapAsByteBuffer()); + } + + private static TypeSerializerSnapshot writeAndThenReadSerializerSnapshot( + TypeSerializer serializer) throws IOException { + + DataOutputSerializer out = new DataOutputSerializer(INITIAL_OUTPUT_BUFFER_SIZE); + writeSerializerSnapshotCurrentFormat(out, serializer); + + DataInputDeserializer in = new DataInputDeserializer(out.wrapAsByteBuffer()); + return readSerializerSnapshotCurrentFormat( + in, Thread.currentThread().getContextClassLoader()); + } +} diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/serializer-snapshot b/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..85a71dc6ba48967d2886e2d23db00f3ace9f3e00 GIT binary patch literal 126 zcmZ|F!3lsc3_#JKt9Z^5f)_7BCkQoaX-y=JAWp9i;O+By0MLM|tw;yOjd3YSGFCUH uJ;{nuML52qVx}b%4j&l5oJ{94vURHgh8=%|}I0JIM-2QB~r literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/test-data new file mode 100644 index 0000000000000000000000000000000000000000..3efe488b18be8cab3ffc86511ae5a20321effd7f GIT binary patch literal 19 acmZQzU|?ooHZ(G3U^X!|V_-J7umk`NKLP3h literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/serializer-snapshot b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..b95eacb633e5812b3ee57980a0dd34732d9d81cc GIT binary patch literal 134 zcmaLNu?@mN3`SuqtU~QEXlRfw3;Z%pVqLH!pJ@=oqbxx8r0)^YMBm7jaX>cZWjF6U yDitsDl_lI&Dsphg(!sZLxS#pRc)msRIwyQU%EDWrv}qc4`~0QmIK~!5M3*1FD=|s{ literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/test-data b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/test-data new file mode 100644 index 0000000000000000000000000000000000000000..0936509e94a5ecf733366d7275336f06d56d70a1 GIT binary patch literal 17 UcmZQ%U@ Date: Tue, 4 Jul 2023 13:17:19 +0800 Subject: [PATCH 222/322] [FLINK-32453] Ambiguous assertThat call after upgrade to Flink 1.18 This closes #39. --- .../connector/kafka/source/enumerator/KafkaEnumeratorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java index 3ced33f98..b89f1b16d 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java @@ -404,7 +404,7 @@ public void testKafkaClientProperties() throws Exception { assertThat(adminClient).isNotNull(); String clientId = (String) Whitebox.getInternalState(adminClient, "clientId"); assertThat(clientId).isNotNull().startsWith(clientIdPrefix); - assertThat(Whitebox.getInternalState(adminClient, "defaultApiTimeoutMs")) + assertThat((int) Whitebox.getInternalState(adminClient, "defaultApiTimeoutMs")) .isEqualTo(defaultTimeoutMs); assertThat(clientId).isNotNull().startsWith(clientIdPrefix); From 9109722c0cf27299b60232c64ae29e00c62934a7 Mon Sep 17 00:00:00 2001 From: Alex Sorokoumov Date: Sun, 12 Mar 2023 13:36:48 -0700 Subject: [PATCH 223/322] [FLINK-31408] Add support for EOS delivery-guarantee in upsert-kafka * add sink.delivery-guarantee and sink.transactional-id-prefix options to upsert-kafka * fix the default isolation.level in kafka connector documentation * let ReducingUpsertSink implement TwoPhaseCommittingSink * add update upsert-kafka connector documentation This closes #7. --- docs/content/docs/connectors/table/kafka.md | 4 +- .../docs/connectors/table/upsert-kafka.md | 29 ++++++++++ .../flink/connector/kafka/sink/KafkaSink.java | 5 +- .../connector/kafka/sink/KafkaWriter.java | 6 +- .../sink/TwoPhaseCommittingStatefulSink.java | 34 +++++++++++ .../kafka/table/KafkaDynamicSink.java | 2 +- .../kafka/table/ReducingUpsertSink.java | 47 ++++++++++----- .../kafka/table/ReducingUpsertWriter.java | 21 +++++-- .../table/UpsertKafkaDynamicTableFactory.java | 14 ++++- .../kafka/table/ReducingUpsertWriterTest.java | 23 +++++--- .../UpsertKafkaDynamicTableFactoryTest.java | 58 +++++++++++++++---- 11 files changed, 190 insertions(+), 53 deletions(-) create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TwoPhaseCommittingStatefulSink.java diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md index 1939c2daf..8866c308f 100644 --- a/docs/content/docs/connectors/table/kafka.md +++ b/docs/content/docs/connectors/table/kafka.md @@ -608,8 +608,8 @@ Besides enabling Flink's checkpointing, you can also choose three different mode * `none`: Flink will not guarantee anything. Produced records can be lost or they can be duplicated. * `at-least-once` (default setting): This guarantees that no records will be lost (although they can be duplicated). * `exactly-once`: Kafka transactions will be used to provide exactly-once semantic. Whenever you write - to Kafka using transactions, do not forget about setting desired `isolation.level` (`read_committed` - or `read_uncommitted` - the latter one is the default value) for any application consuming records + to Kafka using transactions, do not forget about setting desired `isolation.level` (`read_uncommitted` + or `read_committed` - the latter one is the default value) for any application consuming records from Kafka. Please refer to [Kafka documentation]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-producers-and-fault-tolerance) for more caveats about delivery guarantees. diff --git a/docs/content/docs/connectors/table/upsert-kafka.md b/docs/content/docs/connectors/table/upsert-kafka.md index 26a377db4..c3537a764 100644 --- a/docs/content/docs/connectors/table/upsert-kafka.md +++ b/docs/content/docs/connectors/table/upsert-kafka.md @@ -221,6 +221,22 @@ Connector Options By default, this is disabled. Note both 'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' must be set to be greater than zero to enable sink buffer flushing. + +
sink.delivery-guarantee
+ optional + no + at-least-once + String + Defines the delivery semantic for the upsert-kafka sink. Valid enumerationns are 'at-least-once', 'exactly-once' and 'none'. See Consistency guarantees for more details. + + +
sink.transactional-id-prefix
+ optional + yes + (none) + String + If the delivery guarantee is configured as 'exactly-once' this value must be set and is used a prefix for the identifier of all opened Kafka transactions. + @@ -277,6 +293,19 @@ connector is working in the upsert mode, the last record on the same key will ta reading back as a source. Therefore, the upsert-kafka connector achieves idempotent writes just like the [HBase sink]({{< ref "docs/connectors/table/hbase" >}}). +With Flink's checkpointing enabled, the `upsert-kafka` connector can provide exactly-once delivery guarantees. + +Besides enabling Flink's checkpointing, you can also choose three different modes of operating chosen by passing appropriate `sink.delivery-guarantee` option: + +* `none`: Flink will not guarantee anything. Produced records can be lost or they can be duplicated. +* `at-least-once` (default setting): This guarantees that no records will be lost (although they can be duplicated). +* `exactly-once`: Kafka transactions will be used to provide exactly-once semantic. Whenever you write + to Kafka using transactions, do not forget about setting desired `isolation.level` (`read_uncommitted` + or `read_committed` - the latter one is the default value) for any application consuming records + from Kafka. + +Please refer to [Kafka connector documentation]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-producers-and-fault-tolerance) for more caveats about delivery guarantees. + ### Source Per-Partition Watermarks Flink supports to emit per-partition watermarks for Upsert Kafka. Watermarks are generated inside the Kafka diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java index 6f74aaed5..d5b1c3700 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java @@ -21,8 +21,6 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.Committer; -import org.apache.flink.api.connector.sink2.StatefulSink; -import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -56,8 +54,7 @@ */ @PublicEvolving public class KafkaSink - implements StatefulSink, - TwoPhaseCommittingSink { + implements TwoPhaseCommittingStatefulSink { private final DeliveryGuarantee deliveryGuarantee; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java index 1829d5a62..bef76fb6a 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java @@ -22,8 +22,6 @@ import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.connector.sink2.Sink; -import org.apache.flink.api.connector.sink2.StatefulSink; -import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.MetricUtil; import org.apache.flink.metrics.Counter; @@ -72,8 +70,8 @@ * @param The type of the input elements. */ class KafkaWriter - implements StatefulSink.StatefulSinkWriter, - TwoPhaseCommittingSink.PrecommittingSinkWriter { + implements TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + IN, KafkaWriterState, KafkaCommittable> { private static final Logger LOG = LoggerFactory.getLogger(KafkaWriter.class); private static final String KAFKA_PRODUCER_METRIC_NAME = "KafkaProducer"; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TwoPhaseCommittingStatefulSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TwoPhaseCommittingStatefulSink.java new file mode 100644 index 000000000..4e64c8698 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TwoPhaseCommittingStatefulSink.java @@ -0,0 +1,34 @@ +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.sink2.StatefulSink; +import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; + +import java.io.IOException; +import java.util.Collection; + +/** + * A combination of {@link TwoPhaseCommittingSink} and {@link StatefulSink}. + * + *

The purpose of this interface is to be able to pass an interface rather than a {@link + * KafkaSink} implementation into the reducing sink which simplifies unit testing. + * + * @param The type of the sink's input + * @param The type of the sink writer's state + * @param The type of the committables. + */ +@Internal +public interface TwoPhaseCommittingStatefulSink + extends TwoPhaseCommittingSink, StatefulSink { + + PrecommittingStatefulSinkWriter createWriter(InitContext context) + throws IOException; + + PrecommittingStatefulSinkWriter restoreWriter( + InitContext context, Collection recoveredState) throws IOException; + + /** A combination of {@link PrecommittingSinkWriter} and {@link StatefulSinkWriter}. */ + interface PrecommittingStatefulSinkWriter + extends PrecommittingSinkWriter, + StatefulSinkWriter {} +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java index 041846dd7..3f6bc5a27 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java @@ -223,7 +223,7 @@ public DataStreamSink consumeDataStream( ProviderContext providerContext, DataStream dataStream) { final boolean objectReuse = dataStream.getExecutionEnvironment().getConfig().isObjectReuseEnabled(); - final ReducingUpsertSink sink = + final ReducingUpsertSink sink = new ReducingUpsertSink<>( kafkaSink, physicalDataType, diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java index da48e39e4..c88f05456 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java @@ -19,7 +19,8 @@ import org.apache.flink.api.connector.sink.Sink; import org.apache.flink.api.connector.sink.SinkWriter; -import org.apache.flink.api.connector.sink2.StatefulSink; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; @@ -32,19 +33,21 @@ * A wrapper of a {@link Sink}. It will buffer the data emitted by the wrapper {@link SinkWriter} * and only emit it when the buffer is full or a timer is triggered or a checkpoint happens. * - *

The sink provides eventual consistency guarantees without the need of a two-phase protocol - * because the updates are idempotent therefore duplicates have no effect. + *

The sink provides eventual consistency guarantees under {@link + * org.apache.flink.connector.base.DeliveryGuarantee#AT_LEAST_ONCE} because the updates are + * idempotent therefore duplicates have no effect. */ -class ReducingUpsertSink implements StatefulSink { +class ReducingUpsertSink + implements TwoPhaseCommittingStatefulSink { - private final StatefulSink wrappedSink; + private final TwoPhaseCommittingStatefulSink wrappedSink; private final DataType physicalDataType; private final int[] keyProjection; private final SinkBufferFlushMode bufferFlushMode; private final SerializableFunction valueCopyFunction; ReducingUpsertSink( - StatefulSink wrappedSink, + TwoPhaseCommittingStatefulSink wrappedSink, DataType physicalDataType, int[] keyProjection, SinkBufferFlushMode bufferFlushMode, @@ -57,12 +60,11 @@ class ReducingUpsertSink implements StatefulSink createWriter(InitContext context) - throws IOException { - final StatefulSinkWriter wrapperWriter = - wrappedSink.createWriter(context); + public TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> + createWriter(InitContext context) throws IOException { return new ReducingUpsertWriter<>( - wrapperWriter, + wrappedSink.createWriter(context), physicalDataType, keyProjection, bufferFlushMode, @@ -71,12 +73,25 @@ public StatefulSinkWriter createWriter(InitContext context } @Override - public StatefulSinkWriter restoreWriter( - InitContext context, Collection recoveredState) throws IOException { - final StatefulSinkWriter wrapperWriter = - wrappedSink.restoreWriter(context, recoveredState); + public Committer createCommitter() throws IOException { + return wrappedSink.createCommitter(); + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return wrappedSink.getCommittableSerializer(); + } + + @Override + public TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> + restoreWriter(InitContext context, Collection recoveredState) + throws IOException { + final TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> + wrappedWriter = wrappedSink.restoreWriter(context, recoveredState); return new ReducingUpsertWriter<>( - wrapperWriter, + wrappedWriter, physicalDataType, keyProjection, bufferFlushMode, diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java index 914874666..45421df64 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java @@ -19,8 +19,8 @@ import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.api.connector.sink2.SinkWriter; -import org.apache.flink.api.connector.sink2.StatefulSink; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -39,10 +40,13 @@ import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; -class ReducingUpsertWriter - implements StatefulSink.StatefulSinkWriter { +class ReducingUpsertWriter + implements TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> { - private final StatefulSink.StatefulSinkWriter wrappedWriter; + private final TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> + wrappedWriter; private final WrappedContext wrappedContext = new WrappedContext(); private final int batchMaxRowNums; private final Function valueCopyFunction; @@ -55,7 +59,9 @@ class ReducingUpsertWriter private long lastFlush = System.currentTimeMillis(); ReducingUpsertWriter( - StatefulSink.StatefulSinkWriter wrappedWriter, + TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> + wrappedWriter, DataType physicalDataType, int[] keyProjection, SinkBufferFlushMode bufferFlushMode, @@ -150,6 +156,11 @@ private void sinkBuffer() throws IOException, InterruptedException { reduceBuffer.clear(); } + @Override + public Collection prepareCommit() throws IOException, InterruptedException { + return wrappedWriter.prepareCommit(); + } + /** * Wrapper of {@link SinkWriter.Context}. * diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java index b9f2ea71e..cebe27f2e 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java @@ -24,7 +24,6 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.BoundedOptions; import org.apache.flink.table.api.ValidationException; @@ -52,7 +51,10 @@ import java.util.Objects; import java.util.Properties; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.DELIVERY_GUARANTEE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT; @@ -109,9 +111,16 @@ public Set> optionalOptions() { options.add(SCAN_BOUNDED_MODE); options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS); options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS); + options.add(DELIVERY_GUARANTEE); + options.add(TRANSACTIONAL_ID_PREFIX); return options; } + @Override + public Set> forwardOptions() { + return Stream.of(DELIVERY_GUARANTEE, TRANSACTIONAL_ID_PREFIX).collect(Collectors.toSet()); + } + @Override public DynamicTableSource createDynamicTableSource(Context context) { FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); @@ -179,6 +188,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { keyEncodingFormat, valueEncodingFormat, context.getPrimaryKeyIndexes()); + KafkaConnectorOptionsUtil.validateDeliveryGuarantee(tableOptions); Tuple2 keyValueProjections = createKeyValueProjections(context.getCatalogTable()); @@ -205,7 +215,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { tableOptions.get(TOPIC).get(0), properties, null, - DeliveryGuarantee.AT_LEAST_ONCE, + tableOptions.get(DELIVERY_GUARANTEE), true, flushMode, parallelism, diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java index f08ec1d25..0640b9ad7 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.sink2.SinkWriter; -import org.apache.flink.api.connector.sink2.StatefulSink; +import org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; @@ -41,6 +41,7 @@ import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -151,7 +152,7 @@ public ReducingUpsertWriterTest(boolean enableObjectReuse) { @Test public void testWriteData() throws Exception { final MockedSinkWriter writer = new MockedSinkWriter(); - final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); + final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); // write 4 records which doesn't trigger batch size writeData(bufferedWriter, new ReusableIterator(0, 4)); @@ -218,7 +219,7 @@ public void testWriteData() throws Exception { @Test public void testFlushDataWhenCheckpointing() throws Exception { final MockedSinkWriter writer = new MockedSinkWriter(); - final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); + final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); // write all data, there should be 3 records are still buffered writeData(bufferedWriter, new ReusableIterator(0, 4)); // snapshot should flush the buffer @@ -266,7 +267,7 @@ public void testFlushDataWhenCheckpointing() throws Exception { @Test public void testWriteDataWithNullTimestamp() throws Exception { final MockedSinkWriter writer = new MockedSinkWriter(); - final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); + final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); bufferedWriter.write( GenericRowData.ofKind( @@ -322,7 +323,7 @@ private void compareCompactedResult( } } - private void writeData(ReducingUpsertWriter writer, Iterator iterator) + private void writeData(ReducingUpsertWriter writer, Iterator iterator) throws Exception { while (iterator.hasNext()) { RowData next = iterator.next(); @@ -344,7 +345,7 @@ public Long timestamp() { } @SuppressWarnings("unchecked") - private ReducingUpsertWriter createBufferedWriter(MockedSinkWriter sinkWriter) { + private ReducingUpsertWriter createBufferedWriter(MockedSinkWriter sinkWriter) { TypeInformation typeInformation = (TypeInformation) new SinkRuntimeProviderContext(false) @@ -372,7 +373,8 @@ public ScheduledFuture registerTimer( } private static class MockedSinkWriter - implements StatefulSink.StatefulSinkWriter { + implements TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, Void, Void> { boolean flushed = false; @@ -382,7 +384,6 @@ private static class MockedSinkWriter rowDataCollectors = new ArrayList<>(); } - @Override public void write(RowData element, Context context) throws IOException, InterruptedException { // Allow comparison between null timestamps @@ -395,7 +396,6 @@ public void write(RowData element, Context context) rowDataCollectors.add(element); } - @Override public void flush(boolean endOfInput) throws IOException, InterruptedException { flushed = true; } @@ -407,6 +407,11 @@ public void close() throws Exception {} public List snapshotState(long checkpointId) throws IOException { return null; } + + @Override + public Collection prepareCommit() throws IOException, InterruptedException { + return null; + } } private class ReusableIterator implements Iterator { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java index ba424f79e..41d9e7eb0 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java @@ -178,7 +178,14 @@ public void testTableSource() { @Test public void testTableSink() { // Construct table sink using options and table sink factory. - final DynamicTableSink actualSink = createTableSink(SINK_SCHEMA, getFullSinkOptions()); + final Map modifiedOptions = + getModifiedOptions( + getFullSinkOptions(), + options -> { + options.put("sink.delivery-guarantee", "exactly-once"); + options.put("sink.transactional-id-prefix", "kafka-sink"); + }); + final DynamicTableSink actualSink = createTableSink(SINK_SCHEMA, modifiedOptions); final DynamicTableSink expectedSink = createExpectedSink( @@ -190,9 +197,10 @@ public void testTableSink() { null, SINK_TOPIC, UPSERT_KAFKA_SINK_PROPERTIES, - DeliveryGuarantee.AT_LEAST_ONCE, + DeliveryGuarantee.EXACTLY_ONCE, SinkBufferFlushMode.DISABLED, - null); + null, + "kafka-sink"); // Test sink format. final KafkaDynamicSink actualUpsertKafkaSink = (KafkaDynamicSink) actualSink; @@ -219,6 +227,8 @@ public void testBufferedTableSink() { options -> { options.put("sink.buffer-flush.max-rows", "100"); options.put("sink.buffer-flush.interval", "1s"); + options.put("sink.delivery-guarantee", "exactly-once"); + options.put("sink.transactional-id-prefix", "kafka-sink"); })); final DynamicTableSink expectedSink = @@ -231,9 +241,10 @@ public void testBufferedTableSink() { null, SINK_TOPIC, UPSERT_KAFKA_SINK_PROPERTIES, - DeliveryGuarantee.AT_LEAST_ONCE, + DeliveryGuarantee.EXACTLY_ONCE, new SinkBufferFlushMode(100, 1000L), - null); + null, + "kafka-sink"); // Test sink format. final KafkaDynamicSink actualUpsertKafkaSink = (KafkaDynamicSink) actualSink; @@ -266,7 +277,12 @@ public void testBufferedTableSink() { public void testTableSinkWithParallelism() { final Map modifiedOptions = getModifiedOptions( - getFullSinkOptions(), options -> options.put("sink.parallelism", "100")); + getFullSinkOptions(), + options -> { + options.put("sink.parallelism", "100"); + options.put("sink.delivery-guarantee", "exactly-once"); + options.put("sink.transactional-id-prefix", "kafka-sink"); + }); final DynamicTableSink actualSink = createTableSink(SINK_SCHEMA, modifiedOptions); final DynamicTableSink expectedSink = @@ -279,9 +295,10 @@ public void testTableSinkWithParallelism() { null, SINK_TOPIC, UPSERT_KAFKA_SINK_PROPERTIES, - DeliveryGuarantee.AT_LEAST_ONCE, + DeliveryGuarantee.EXACTLY_ONCE, SinkBufferFlushMode.DISABLED, - 100); + 100, + "kafka-sink"); assertThat(actualSink).isEqualTo(expectedSink); final DynamicTableSink.SinkRuntimeProvider provider = @@ -634,6 +651,26 @@ public void testInvalidSinkBufferFlush() { })); } + @Test + public void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "sink.transactional-id-prefix must be specified when using DeliveryGuarantee.EXACTLY_ONCE."))); + + final Map modifiedOptions = + getModifiedOptions( + getFullSinkOptions(), + options -> { + options.remove(KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX.key()); + options.put( + KafkaConnectorOptions.DELIVERY_GUARANTEE.key(), + DeliveryGuarantee.EXACTLY_ONCE.toString()); + }); + createTableSink(SINK_SCHEMA, modifiedOptions); + } + // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- @@ -764,7 +801,8 @@ private static KafkaDynamicSink createExpectedSink( Properties properties, DeliveryGuarantee deliveryGuarantee, SinkBufferFlushMode flushMode, - Integer parallelism) { + Integer parallelism, + String transactionalIdPrefix) { return new KafkaDynamicSink( consumedDataType, consumedDataType, @@ -780,7 +818,7 @@ private static KafkaDynamicSink createExpectedSink( true, flushMode, parallelism, - null); + transactionalIdPrefix); } private KafkaSource assertKafkaSource(ScanTableSource.ScanRuntimeProvider provider) { From 811716c5155e82fa3bfc47ced53daef53bb99cce Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Sat, 6 May 2023 14:31:02 +0800 Subject: [PATCH 224/322] [FLINK-32020] Enable Dynamic Partition Discovery by Default in Kafka Source based on FLIP-288 This closes #40. --- .../kafka/source/KafkaSourceOptions.java | 6 +- .../kafka/table/KafkaConnectorOptions.java | 6 +- .../kafka/table/KafkaDynamicTableFactory.java | 6 +- .../enumerator/KafkaEnumeratorTest.java | 57 +++++++++----- .../table/KafkaDynamicTableFactoryTest.java | 76 +++++++++++++++++++ 5 files changed, 125 insertions(+), 26 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java index 248014c8a..f96cd3ea0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; +import java.time.Duration; import java.util.Properties; import java.util.function.Function; @@ -38,10 +39,11 @@ public class KafkaSourceOptions { public static final ConfigOption PARTITION_DISCOVERY_INTERVAL_MS = ConfigOptions.key("partition.discovery.interval.ms") .longType() - .noDefaultValue() + .defaultValue(Duration.ofMinutes(5).toMillis()) .withDescription( "The interval in milliseconds for the Kafka source to discover " - + "the new partitions. A non-positive value disables the partition discovery."); + + "the new partitions. A non-positive value disables the partition discovery." + + "The default value is 5 minutes, which is equal to the default value of metadata.max.age.ms in Kafka."); public static final ConfigOption REGISTER_KAFKA_CONSUMER_METRICS = ConfigOptions.key("register.consumer.metrics") diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java index a6cdbcedc..81ff13c3c 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java @@ -188,9 +188,11 @@ public class KafkaConnectorOptions { public static final ConfigOption SCAN_TOPIC_PARTITION_DISCOVERY = ConfigOptions.key("scan.topic-partition-discovery.interval") .durationType() - .noDefaultValue() + .defaultValue(Duration.ofMinutes(5)) .withDescription( - "Optional interval for consumer to discover dynamically created Kafka partitions periodically."); + "Optional interval for consumer to discover dynamically created Kafka partitions periodically." + + "The value 0 disables the partition discovery." + + "The default value is 5 minutes, which is equal to the default value of metadata.max.age.ms in Kafka."); // -------------------------------------------------------------------------------------------- // Sink specific options diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java index 48c00918a..89dda61a1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java @@ -201,11 +201,11 @@ public DynamicTableSource createDynamicTableSource(Context context) { final Properties properties = getKafkaProperties(context.getCatalogTable().getOptions()); // add topic-partition discovery - final Optional partitionDiscoveryInterval = - tableOptions.getOptional(SCAN_TOPIC_PARTITION_DISCOVERY).map(Duration::toMillis); + final Duration partitionDiscoveryInterval = + tableOptions.get(SCAN_TOPIC_PARTITION_DISCOVERY); properties.setProperty( KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), - partitionDiscoveryInterval.orElse(-1L).toString()); + Long.toString(partitionDiscoveryInterval.toMillis())); final DataType physicalDataType = context.getPhysicalRowDataType(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java index b89f1b16d..f30c660d7 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java @@ -229,8 +229,8 @@ public void testRunWithPeriodicPartitionDiscoveryOnceToCheckNoMoreSplit() throws public void testRunWithDiscoverPartitionsOnceWithZeroMsToCheckNoMoreSplit() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); - // set partitionDiscoveryIntervalMs = 0 - KafkaSourceEnumerator enumerator = createEnumerator(context, 0L)) { + // Disable periodic partition discovery + KafkaSourceEnumerator enumerator = createEnumerator(context, false)) { // Start the enumerator, and it should schedule a one time task to discover and assign // partitions. @@ -358,7 +358,6 @@ public void testWorkWithPreexistingAssignments() throws Throwable { KafkaSourceEnumerator enumerator = createEnumerator( context2, - ENABLE_PERIODIC_PARTITION_DISCOVERY ? 1 : -1, OffsetsInitializer.earliest(), PRE_EXISTING_TOPICS, preexistingAssignments, @@ -390,7 +389,6 @@ public void testKafkaClientProperties() throws Exception { KafkaSourceEnumerator enumerator = createEnumerator( context, - ENABLE_PERIODIC_PARTITION_DISCOVERY ? 1 : -1, OffsetsInitializer.earliest(), PRE_EXISTING_TOPICS, Collections.emptySet(), @@ -501,6 +499,33 @@ public void testPartitionChangeChecking() throws Throwable { } } + @Test + public void testEnablePartitionDiscoveryByDefault() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + KafkaSourceEnumerator enumerator = createEnumerator(context, new Properties())) { + enumerator.start(); + long partitionDiscoveryIntervalMs = + (long) Whitebox.getInternalState(enumerator, "partitionDiscoveryIntervalMs"); + assertThat(partitionDiscoveryIntervalMs) + .isEqualTo(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.defaultValue()); + assertThat(context.getPeriodicCallables()).isNotEmpty(); + } + } + + @Test + public void testDisablePartitionDiscovery() throws Throwable { + Properties props = new Properties(); + props.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), String.valueOf(0)); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + KafkaSourceEnumerator enumerator = createEnumerator(context, props)) { + enumerator.start(); + assertThat(context.getPeriodicCallables()).isEmpty(); + } + } + // -------------- some common startup sequence --------------- private void startEnumeratorAndRegisterReaders( @@ -539,13 +564,9 @@ private KafkaSourceEnumerator createEnumerator( } private KafkaSourceEnumerator createEnumerator( - MockSplitEnumeratorContext enumContext, - long partitionDiscoveryIntervalMs) { + MockSplitEnumeratorContext enumContext, Properties properties) { return createEnumerator( - enumContext, - partitionDiscoveryIntervalMs, - EXCLUDE_DYNAMIC_TOPIC, - OffsetsInitializer.earliest()); + enumContext, properties, EXCLUDE_DYNAMIC_TOPIC, OffsetsInitializer.earliest()); } private KafkaSourceEnumerator createEnumerator( @@ -557,20 +578,23 @@ private KafkaSourceEnumerator createEnumerator( if (includeDynamicTopic) { topics.add(DYNAMIC_TOPIC_NAME); } + Properties props = new Properties(); + props.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), + enablePeriodicPartitionDiscovery ? "1" : "-1"); return createEnumerator( enumContext, - enablePeriodicPartitionDiscovery ? 1 : -1, startingOffsetsInitializer, topics, Collections.emptySet(), Collections.emptySet(), false, - new Properties()); + props); } private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, - long partitionDiscoveryIntervalMs, + Properties props, boolean includeDynamicTopic, OffsetsInitializer startingOffsetsInitializer) { List topics = new ArrayList<>(PRE_EXISTING_TOPICS); @@ -579,13 +603,12 @@ private KafkaSourceEnumerator createEnumerator( } return createEnumerator( enumContext, - partitionDiscoveryIntervalMs, startingOffsetsInitializer, topics, Collections.emptySet(), Collections.emptySet(), false, - new Properties()); + props); } /** @@ -594,7 +617,6 @@ private KafkaSourceEnumerator createEnumerator( */ private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, - long partitionDiscoveryIntervalMs, OffsetsInitializer startingOffsetsInitializer, Collection topicsToSubscribe, Set assignedPartitions, @@ -613,9 +635,6 @@ private KafkaSourceEnumerator createEnumerator( Properties props = new Properties(KafkaSourceTestEnv.getConsumerProperties(StringDeserializer.class)); KafkaSourceEnumerator.deepCopyProperties(overrideProperties, props); - props.setProperty( - KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), - String.valueOf(partitionDiscoveryIntervalMs)); return new KafkaSourceEnumerator( subscriber, diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java index 7ab050359..bdcbec71f 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java @@ -1095,6 +1095,82 @@ public void testPrimaryKeyValidation() { + " guarantee the semantic of primary key."); } + @Test + public void testDiscoverPartitionByDefault() { + Map tableSourceOptions = + getModifiedOptions( + getBasicSourceOptions(), + options -> options.remove("scan.topic-partition-discovery.interval")); + final KafkaDynamicSource actualSource = + (KafkaDynamicSource) createTableSource(SCHEMA, tableSourceOptions); + Properties props = new Properties(); + props.putAll(KAFKA_SOURCE_PROPERTIES); + // The default partition discovery interval is 5 minutes + props.setProperty("partition.discovery.interval.ms", "300000"); + final Map specificOffsets = new HashMap<>(); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_0), OFFSET_0); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_1), OFFSET_1); + final DecodingFormat> valueDecodingFormat = + new DecodingFormatMock(",", true); + // Test scan source equals + final KafkaDynamicSource expectedKafkaSource = + createExpectedScanSource( + SCHEMA_DATA_TYPE, + null, + valueDecodingFormat, + new int[0], + new int[] {0, 1, 2}, + null, + Collections.singletonList(TOPIC), + null, + props, + StartupMode.SPECIFIC_OFFSETS, + specificOffsets, + 0); + assertThat(actualSource).isEqualTo(expectedKafkaSource); + ScanTableSource.ScanRuntimeProvider provider = + actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertKafkaSource(provider); + } + + @Test + public void testDisableDiscoverPartition() { + Map tableSourceOptions = + getModifiedOptions( + getBasicSourceOptions(), + options -> options.put("scan.topic-partition-discovery.interval", "0")); + final KafkaDynamicSource actualSource = + (KafkaDynamicSource) createTableSource(SCHEMA, tableSourceOptions); + Properties props = new Properties(); + props.putAll(KAFKA_SOURCE_PROPERTIES); + // Disable discovery if the partition discovery interval is 0 minutes + props.setProperty("partition.discovery.interval.ms", "0"); + final Map specificOffsets = new HashMap<>(); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_0), OFFSET_0); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_1), OFFSET_1); + final DecodingFormat> valueDecodingFormat = + new DecodingFormatMock(",", true); + // Test scan source equals + final KafkaDynamicSource expectedKafkaSource = + createExpectedScanSource( + SCHEMA_DATA_TYPE, + null, + valueDecodingFormat, + new int[0], + new int[] {0, 1, 2}, + null, + Collections.singletonList(TOPIC), + null, + props, + StartupMode.SPECIFIC_OFFSETS, + specificOffsets, + 0); + assertThat(actualSource).isEqualTo(expectedKafkaSource); + ScanTableSource.ScanRuntimeProvider provider = + actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertKafkaSource(provider); + } + // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- From 79ae2d70499f81ce489911956c675354657dd44f Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Mon, 17 Jul 2023 14:46:36 +0800 Subject: [PATCH 225/322] [FLINK-32591][Connectors/Kafka] Update document of Kafka Source: Enable Dynamic Partition Discovery by Default in Kafka Source This closes #41. --- docs/content.zh/docs/connectors/datastream/kafka.md | 4 ++-- docs/content.zh/docs/connectors/table/kafka.md | 4 ++-- docs/content/docs/connectors/datastream/kafka.md | 5 ++--- docs/content/docs/connectors/table/kafka.md | 4 ++-- 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/docs/content.zh/docs/connectors/datastream/kafka.md b/docs/content.zh/docs/connectors/datastream/kafka.md index 039bdeca8..35fd281f8 100644 --- a/docs/content.zh/docs/connectors/datastream/kafka.md +++ b/docs/content.zh/docs/connectors/datastream/kafka.md @@ -229,7 +229,7 @@ Kafka consumer 的配置可以参考 [Apache Kafka 文档](http://kafka.apache.o ### 动态分区检查 为了在不重启 Flink 作业的情况下处理 Topic 扩容或新建 Topic 等场景,可以将 Kafka Source 配置为在提供的 Topic / Partition -订阅模式下定期检查新分区。要启用动态分区检查,请将 ```partition.discovery.interval.ms``` 设置为非负值: +订阅模式下定期检查新分区。要启用动态分区检查,请将 ```partition.discovery.interval.ms``` 设置为正值: {{< tabs "KafkaSource#PartitionDiscovery" >}} @@ -248,7 +248,7 @@ KafkaSource.builder() \ {{< /tabs >}} {{< hint warning >}} -分区检查功能默认**不开启**。需要显式地设置分区检查间隔才能启用此功能。 +分区检查间隔默认为5分钟。需要显式地设置分区检查间隔为非正数才能关闭此功能。 {{< /hint >}} ### 事件时间和水印 diff --git a/docs/content.zh/docs/connectors/table/kafka.md b/docs/content.zh/docs/connectors/table/kafka.md index f9ea66f86..ce26e0728 100644 --- a/docs/content.zh/docs/connectors/table/kafka.md +++ b/docs/content.zh/docs/connectors/table/kafka.md @@ -338,9 +338,9 @@ CREATE TABLE KafkaTable (

scan.topic-partition-discovery.interval
可选 - (无) + 5分钟 Duration - Consumer 定期探测动态创建的 Kafka topic 和 partition 的时间间隔。 + Consumer 定期探测动态创建的 Kafka topic 和 partition 的时间间隔。需要显式地设置'scan.topic-partition-discovery.interval'为0才能关闭此功能
sink.partitioner
diff --git a/docs/content/docs/connectors/datastream/kafka.md b/docs/content/docs/connectors/datastream/kafka.md index 7589b8d3c..acbbbc799 100644 --- a/docs/content/docs/connectors/datastream/kafka.md +++ b/docs/content/docs/connectors/datastream/kafka.md @@ -245,7 +245,7 @@ it is configured: ### Dynamic Partition Discovery In order to handle scenarios like topic scaling-out or topic creation without restarting the Flink job, Kafka source can be configured to periodically discover new partitions under provided -topic-partition subscribing pattern. To enable partition discovery, set a non-negative value for +topic-partition subscribing pattern. To enable partition discovery, set a positive value for property ```partition.discovery.interval.ms```: {{< tabs "KafkaSource#PartitionDiscovery" >}} @@ -264,8 +264,7 @@ KafkaSource.builder() \ {{< /tabs >}} {{< hint warning >}} -Partition discovery is **disabled** by default. You need to explicitly set the partition discovery -interval to enable this feature. +The partition discovery interval is 5 minutes by default. To **disable** this feature, you need to explicitly set the partition discovery interval to a non-positive value. {{< /hint >}} ### Event Time and Watermarks diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md index 8866c308f..93123d89c 100644 --- a/docs/content/docs/connectors/table/kafka.md +++ b/docs/content/docs/connectors/table/kafka.md @@ -364,9 +364,9 @@ Connector Options
scan.topic-partition-discovery.interval
optional yes - (none) + 5 minutes Duration - Interval for consumer to discover dynamically created Kafka topics and partitions periodically. + Interval for consumer to discover dynamically created Kafka topics and partitions periodically. To disable this feature, you need to explicitly set the 'scan.topic-partition-discovery.interval' value to 0.
sink.partitioner
From d6525c1481fc2d2821f361d2d5ce48f97221dd74 Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Thu, 4 May 2023 17:28:35 +0800 Subject: [PATCH 226/322] [FLINK-31208][Connectors/Kafka] KafkaSourceReader overrides meaninglessly a method(pauseOrResumeSplits) --- .../connector/kafka/source/reader/KafkaSourceReader.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java index e8fe3d776..a3aa555f1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java @@ -38,7 +38,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -186,12 +185,6 @@ protected KafkaPartitionSplit toSplitType(String splitId, KafkaPartitionSplitSta return splitState.toKafkaPartitionSplit(); } - @Override - public void pauseOrResumeSplits( - Collection splitsToPause, Collection splitsToResume) { - splitFetcherManager.pauseOrResumeSplits(splitsToPause, splitsToResume); - } - // ------------------------ @VisibleForTesting From 818d1fdedaad63631eab5d44ec90c748cfcf299f Mon Sep 17 00:00:00 2001 From: Alex Sorokoumov Date: Tue, 5 Sep 2023 16:28:14 -0700 Subject: [PATCH 227/322] [FLINK-33017] Remove dependency on shaded guava The bump in shaded guava in Flink 1.18 changed import paths and caused the class loader fail when loading ManagedMemoryUtils. Looking at the root cause of the issue, shading was used as a technique to avoid dependency hell. As flink-connector-kafka should work with both flink 1.17 and 1.18 that use different guava versions (and hence library import paths), shading did not really solve the problem it was introduced for in the first place. There are several several options to work around the problem. First, we could introduce our own shading for guava. Second, we could see if the dependency on guava is necessary at all and maybe remove it completely. This patch takes the latter route and removes dependency on guava from this connector. --- .../tests/util/kafka/SmokeKafkaITCase.java | 5 +- .../connector/kafka/sink/KafkaWriter.java | 31 ++++++------- .../connectors/kafka/FlinkKafkaProducer.java | 13 +++--- .../internals/FlinkKafkaInternalProducer.java | 8 ++-- .../FlinkKafkaInternalProducerITCase.java | 5 +- ...aRecordSerializationSchemaBuilderTest.java | 17 ++++--- .../connector/kafka/sink/KafkaSinkITCase.java | 9 ++-- .../kafka/sink/KafkaWriterITCase.java | 7 ++- .../sink/TransactionToAbortCheckerTest.java | 46 ++++++++++++------- .../KafkaRecordDeserializationSchemaTest.java | 6 +-- .../FlinkKafkaInternalProducerITCase.java | 7 +-- .../kafka/KafkaConsumerTestBase.java | 13 ++++-- .../kafka/shuffle/KafkaShuffleITCase.java | 26 +++++------ .../table/KafkaDynamicTableFactoryTest.java | 4 +- pom.xml | 6 --- 15 files changed, 105 insertions(+), 98 deletions(-) diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java index 726eceea9..a4d0c002e 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java @@ -29,8 +29,6 @@ import org.apache.flink.test.util.JobSubmission; import org.apache.flink.util.TestLoggerExtension; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.NewTopic; @@ -56,6 +54,7 @@ import java.nio.ByteBuffer; import java.nio.file.Path; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -137,7 +136,7 @@ public void testKafka() throws Exception { // create the required topics final short replicationFactor = 1; admin.createTopics( - Lists.newArrayList( + Arrays.asList( new NewTopic(inputTopic, 1, replicationFactor), new NewTopic(outputTopic, 1, replicationFactor))) .all() diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java index bef76fb6a..0cc16b219 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java @@ -31,10 +31,6 @@ import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricMutableWrapper; import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; -import org.apache.flink.shaded.guava30.com.google.common.io.Closer; - import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; @@ -49,6 +45,7 @@ import java.io.IOException; import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Deque; @@ -104,9 +101,10 @@ class KafkaWriter // producer pool only used for exactly once private final Deque> producerPool = new ArrayDeque<>(); - private final Closer closer = Closer.create(); private long lastCheckpointId; + private final Deque producerCloseables = new ArrayDeque<>(); + private boolean closed = false; private long lastSync = System.currentTimeMillis(); @@ -178,7 +176,7 @@ class KafkaWriter } else if (deliveryGuarantee == DeliveryGuarantee.AT_LEAST_ONCE || deliveryGuarantee == DeliveryGuarantee.NONE) { this.currentProducer = new FlinkKafkaInternalProducer<>(this.kafkaProducerConfig, null); - closer.register(this.currentProducer); + producerCloseables.add(this.currentProducer); initKafkaMetrics(this.currentProducer); } else { throw new UnsupportedOperationException( @@ -236,21 +234,18 @@ public List snapshotState(long checkpointId) throws IOExceptio currentProducer = getTransactionalProducer(checkpointId + 1); currentProducer.beginTransaction(); } - return ImmutableList.of(kafkaWriterState); + return Collections.singletonList(kafkaWriterState); } @Override public void close() throws Exception { closed = true; LOG.debug("Closing writer with {}", currentProducer); - closeAll( - this::abortCurrentProducer, - closer, - producerPool::clear, - () -> { - checkState(currentProducer.isClosed()); - currentProducer = null; - }); + closeAll(this::abortCurrentProducer, producerPool::clear); + closeAll(producerCloseables); + checkState( + currentProducer.isClosed(), "Could not close current producer " + currentProducer); + currentProducer = null; // Rethrow exception for the case in which close is called before writer() and flush(). checkAsyncException(); @@ -279,7 +274,8 @@ FlinkKafkaInternalProducer getCurrentProducer() { void abortLingeringTransactions( Collection recoveredStates, long startCheckpointId) { - List prefixesToAbort = Lists.newArrayList(transactionalIdPrefix); + List prefixesToAbort = new ArrayList<>(); + prefixesToAbort.add(transactionalIdPrefix); final Optional lastStateOpt = recoveredStates.stream().findFirst(); if (lastStateOpt.isPresent()) { @@ -337,7 +333,7 @@ private FlinkKafkaInternalProducer getOrCreateTransactionalProdu FlinkKafkaInternalProducer producer = producerPool.poll(); if (producer == null) { producer = new FlinkKafkaInternalProducer<>(kafkaProducerConfig, transactionalId); - closer.register(producer); + producerCloseables.add(producer); producer.initTransactions(); initKafkaMetrics(producer); } else { @@ -452,6 +448,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { asyncProducerException = decorateException(metadata, exception, producer); } + // Checking for exceptions from previous writes mailboxExecutor.submit( () -> { // Checking for exceptions from previous writes diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java index ca81cc8e3..d6cbe2e7d 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java @@ -52,8 +52,6 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.TemporaryClassLoaderContext; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import org.apache.commons.lang3.StringUtils; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; @@ -1200,8 +1198,10 @@ public void initializeState(FunctionInitializationContext context) throws Except if (semantic != FlinkKafkaProducer.Semantic.EXACTLY_ONCE) { nextTransactionalIdHint = null; } else { - ArrayList transactionalIdHints = - Lists.newArrayList(nextTransactionalIdHintState.get()); + List transactionalIdHints = + new ArrayList<>(); + nextTransactionalIdHintState.get().forEach(transactionalIdHints::add); + if (transactionalIdHints.size() > 1) { throw new IllegalStateException( "There should be at most one next transactional id hint written by the first subtask"); @@ -1444,8 +1444,9 @@ private void migrateNextTransactionalIdHindState(FunctionInitializationContext c context.getOperatorStateStore() .getUnionListState(NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR_V2); - ArrayList oldTransactionalIdHints = - Lists.newArrayList(oldNextTransactionalIdHintState.get()); + List oldTransactionalIdHints = new ArrayList<>(); + oldNextTransactionalIdHintState.get().forEach(oldTransactionalIdHints::add); + if (!oldTransactionalIdHints.isEmpty()) { nextTransactionalIdHintState.addAll(oldTransactionalIdHints); // clear old state diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java index 4b270b0b7..12dad9fba 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java @@ -22,8 +22,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.util.Preconditions; -import org.apache.flink.shaded.guava30.com.google.common.base.Joiner; - import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.Callback; @@ -51,10 +49,12 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.time.Duration; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.concurrent.Future; +import java.util.stream.Collectors; /** Internal flink kafka producer. */ @PublicEvolving @@ -169,7 +169,9 @@ public void close(Duration duration) { LOG.debug( "Closed internal KafkaProducer {}. Stacktrace: {}", System.identityHashCode(this), - Joiner.on("\n").join(Thread.currentThread().getStackTrace())); + Arrays.stream(Thread.currentThread().getStackTrace()) + .map(StackTraceElement::toString) + .collect(Collectors.joining("\n"))); } closed = true; } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java index dd15ec226..69e9f19f7 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java @@ -19,8 +19,6 @@ import org.apache.flink.util.TestLoggerExtension; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -43,6 +41,7 @@ import org.testcontainers.junit.jupiter.Testcontainers; import java.time.Duration; +import java.util.Arrays; import java.util.List; import java.util.Properties; import java.util.function.Consumer; @@ -173,7 +172,7 @@ private static Properties getProperties() { } private static List>> provideTransactionsFinalizer() { - return Lists.newArrayList( + return Arrays.asList( FlinkKafkaInternalProducer::commitTransaction, FlinkKafkaInternalProducer::abortTransaction); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java index 6dd5baed3..701f9c8aa 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java @@ -23,9 +23,6 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.util.TestLogger; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap; - import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.header.Header; @@ -38,6 +35,7 @@ import org.junit.Test; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -153,7 +151,8 @@ public void testSerializeRecordWithPartitioner() throws Exception { public void testSerializeRecordWithHeaderProvider() throws Exception { final HeaderProvider headerProvider = (ignored) -> - new RecordHeaders(ImmutableList.of(new RecordHeader("a", "a".getBytes()))); + new RecordHeaders( + Collections.singletonList(new RecordHeader("a", "a".getBytes()))); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() @@ -186,7 +185,7 @@ public void testSerializeRecordWithKey() { @Test public void testKafkaKeySerializerWrapperWithoutConfigurable() throws Exception { - final Map config = ImmutableMap.of("simpleKey", "simpleValue"); + final Map config = Collections.singletonMap("simpleKey", "simpleValue"); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() .setTopic(DEFAULT_TOPIC) @@ -203,7 +202,7 @@ public void testKafkaKeySerializerWrapperWithoutConfigurable() throws Exception @Test public void testKafkaValueSerializerWrapperWithoutConfigurable() throws Exception { - final Map config = ImmutableMap.of("simpleKey", "simpleValue"); + final Map config = Collections.singletonMap("simpleKey", "simpleValue"); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() .setTopic(DEFAULT_TOPIC) @@ -217,7 +216,7 @@ public void testKafkaValueSerializerWrapperWithoutConfigurable() throws Exceptio @Test public void testSerializeRecordWithKafkaSerializer() throws Exception { - final Map config = ImmutableMap.of("configKey", "configValue"); + final Map config = Collections.singletonMap("configKey", "configValue"); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() .setTopic(DEFAULT_TOPIC) @@ -285,7 +284,7 @@ private static void assertOnlyOneSerializerAllowed( KafkaRecordSerializationSchemaBuilder, KafkaRecordSerializationSchemaBuilder>> valueSerializationSetter() { - return ImmutableList.of( + return Arrays.asList( (b) -> b.setKafkaValueSerializer(StringSerializer.class), (b) -> b.setValueSerializationSchema(new SimpleStringSchema()), (b) -> @@ -298,7 +297,7 @@ private static void assertOnlyOneSerializerAllowed( KafkaRecordSerializationSchemaBuilder, KafkaRecordSerializationSchemaBuilder>> keySerializationSetter() { - return ImmutableList.of( + return Arrays.asList( (b) -> b.setKafkaKeySerializer(StringSerializer.class), (b) -> b.setKeySerializationSchema(new SimpleStringSchema()), (b) -> diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java index 942902f04..fda9d6fa3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java @@ -60,8 +60,6 @@ import org.apache.flink.util.DockerImageVersions; import org.apache.flink.util.TestLogger; -import org.apache.flink.shaded.guava30.com.google.common.base.Joiner; - import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.CreateTopicsResult; @@ -88,6 +86,7 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -624,7 +623,11 @@ private void checkProducerLeak() throws InterruptedException { } private String format(Map.Entry leak) { - return leak.getKey().getName() + ":\n" + Joiner.on("\n").join(leak.getValue()); + String stackTrace = + Arrays.stream(leak.getValue()) + .map(StackTraceElement::toString) + .collect(Collectors.joining("\n")); + return leak.getKey().getName() + ":\n" + stackTrace; } private boolean findAliveKafkaThread(Map.Entry threadStackTrace) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index c1b022dce..811ffa207 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -34,8 +34,6 @@ import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.UserCodeClassLoader; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; @@ -60,6 +58,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Optional; @@ -545,7 +544,7 @@ private KafkaWriter createWriterWithConfiguration( new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer), new DummyRecordSerializer(), new DummySchemaContext(), - ImmutableList.of()); + Collections.emptyList()); } private KafkaWriter createWriterWithConfiguration( @@ -557,7 +556,7 @@ private KafkaWriter createWriterWithConfiguration( sinkInitContext, new DummyRecordSerializer(), new DummySchemaContext(), - ImmutableList.of()); + Collections.emptyList()); } private static Properties getKafkaClientConfiguration() { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java index 1b65c9d33..897a85911 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java @@ -19,10 +19,10 @@ import org.apache.flink.util.TestLogger; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap; - import org.junit.Test; +import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -32,18 +32,30 @@ public class TransactionToAbortCheckerTest extends TestLogger { public static final String ABORT = "abort"; + public static final String KEEP = "keep"; @Test public void testMustAbortTransactionsWithSameSubtaskIdAndHigherCheckpointOffset() { + Map offsetMapping = new HashMap<>(2); + offsetMapping.put(0, 1L); + offsetMapping.put(2, 3L); final TransactionsToAbortChecker checker = - new TransactionsToAbortChecker(2, ImmutableMap.of(0, 1L, 2, 3L), 0); + new TransactionsToAbortChecker(2, offsetMapping, 0); // abort recovered subtasksId with equal or higher checkpoint offset - final Map> openTransactions = - ImmutableMap.of( - 0, ImmutableMap.of(2L, ABORT, 1L, ABORT), - 2, ImmutableMap.of(3L, ABORT, 4L, ABORT), - 3, ImmutableMap.of(3L, "keep", 4L, "keep")); + final Map> openTransactions = new HashMap<>(3); + final Map subtask0 = new HashMap<>(); + subtask0.put(1L, ABORT); + subtask0.put(2L, ABORT); + openTransactions.put(0, subtask0); + final Map subtask2 = new HashMap<>(); + subtask2.put(3L, ABORT); + subtask2.put(4L, ABORT); + openTransactions.put(2, subtask2); + final Map subtask3 = new HashMap<>(); + subtask3.put(3L, KEEP); + subtask3.put(4L, KEEP); + openTransactions.put(3, subtask3); final List transactionsToAbort = checker.getTransactionsToAbort(openTransactions); assertThat(transactionsToAbort).hasSize(4); @@ -53,16 +65,18 @@ public void testMustAbortTransactionsWithSameSubtaskIdAndHigherCheckpointOffset( @Test public void testMustAbortTransactionsIfLowestCheckpointOffsetIsMinimumOffset() { final TransactionsToAbortChecker checker = - new TransactionsToAbortChecker(2, ImmutableMap.of(0, 1L), 0); + new TransactionsToAbortChecker(2, Collections.singletonMap(0, 1L), 0); // abort recovered subtasksId with equal or higher checkpoint offset - final Map> openTransactions = - ImmutableMap.of( - 0, ImmutableMap.of(2L, ABORT, 1L, ABORT), - 2, ImmutableMap.of(1L, ABORT), - 3, ImmutableMap.of(1L, "keep"), - 4, ImmutableMap.of(1L, ABORT), - 5, ImmutableMap.of(1L, "keep")); + final Map> openTransactions = new HashMap<>(5); + final Map subtask0 = new HashMap<>(); + subtask0.put(1L, ABORT); + subtask0.put(2L, ABORT); + openTransactions.put(0, subtask0); + openTransactions.put(2, Collections.singletonMap(1L, ABORT)); + openTransactions.put(3, Collections.singletonMap(1L, KEEP)); + openTransactions.put(4, Collections.singletonMap(1L, ABORT)); + openTransactions.put(5, Collections.singletonMap(1L, KEEP)); final List transactionsToAbort = checker.getTransactionsToAbort(openTransactions); assertThat(transactionsToAbort).hasSize(4); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java index 8766719a0..e764c8609 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java @@ -25,7 +25,6 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.jackson.JacksonMapperFactory; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; @@ -38,6 +37,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -116,7 +116,7 @@ public void testKafkaValueDeserializerWrapper() throws Exception { @Test public void testKafkaValueDeserializerWrapperWithoutConfigurable() throws Exception { - final Map config = ImmutableMap.of("simpleKey", "simpleValue"); + final Map config = Collections.singletonMap("simpleKey", "simpleValue"); KafkaRecordDeserializationSchema schema = KafkaRecordDeserializationSchema.valueOnly(SimpleStringSerializer.class, config); schema.open(new TestingDeserializationContext()); @@ -127,7 +127,7 @@ public void testKafkaValueDeserializerWrapperWithoutConfigurable() throws Except @Test public void testKafkaValueDeserializerWrapperWithConfigurable() throws Exception { - final Map config = ImmutableMap.of("configKey", "configValue"); + final Map config = Collections.singletonMap("configKey", "configValue"); KafkaRecordDeserializationSchema schema = KafkaRecordDeserializationSchema.valueOnly( ConfigurableStringSerializer.class, config); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java index 2f393374a..15729a8c8 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java @@ -20,8 +20,6 @@ import org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer; -import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -35,6 +33,7 @@ import java.time.Duration; import java.util.Collections; +import java.util.Iterator; import java.util.Properties; import java.util.UUID; @@ -241,9 +240,11 @@ private void assertRecord(String topicName, String expectedKey, String expectedV records = kafkaConsumer.poll(Duration.ofMillis(10000)); } - ConsumerRecord record = Iterables.getOnlyElement(records); + final Iterator> it = records.iterator(); + ConsumerRecord record = it.next(); assertThat(record.key()).isEqualTo(expectedKey); assertThat(record.value()).isEqualTo(expectedValue); + assertThat(it.hasNext()).isFalse(); } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index a8596ee7c..88f6ac607 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -80,8 +80,6 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; - import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetResetStrategy; @@ -104,6 +102,7 @@ import java.util.Collections; import java.util.Date; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -258,7 +257,10 @@ public void run() { } while (System.nanoTime() < deadline); // cancel the job & wait for the job to finish - client.cancel(Iterables.getOnlyElement(getRunningJobs(client))).get(); + final Iterator it = getRunningJobs(client).iterator(); + final JobID jobId = it.next(); + client.cancel(jobId).get(); + assertThat(it.hasNext()).isFalse(); runner.join(); final Throwable t = errorRef.get(); @@ -349,7 +351,10 @@ public void run() { } while (System.nanoTime() < deadline); // cancel the job & wait for the job to finish - client.cancel(Iterables.getOnlyElement(getRunningJobs(client))).get(); + final Iterator it = getRunningJobs(client).iterator(); + final JobID jobId = it.next(); + client.cancel(jobId).get(); + assertThat(it.hasNext()).isFalse(); runner.join(); final Throwable t = errorRef.get(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java index 9a0a14da3..5505bdde3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java @@ -34,9 +34,6 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaShuffleFetcher.KafkaShuffleWatermark; import org.apache.flink.util.PropertiesUtil; -import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.Rule; import org.junit.Test; @@ -374,17 +371,16 @@ private void testRecordSerDe(TimeCharacteristic timeCharacteristic) throws Excep // Records in a single partition are kept in order Collection> records = - Iterables.getOnlyElement( - testKafkaShuffleProducer( - topic( - "test_serde-" + UUID.randomUUID(), - timeCharacteristic), - env, - 1, - 1, - numElementsPerProducer, - timeCharacteristic) - .values()); + testKafkaShuffleProducer( + topic("test_serde-" + UUID.randomUUID(), timeCharacteristic), + env, + 1, + 1, + numElementsPerProducer, + timeCharacteristic) + .values() + .iterator() + .next(); switch (timeCharacteristic) { case ProcessingTime: @@ -516,7 +512,7 @@ private Map>> testKafkaShuffl r -> { final int partition = r.partition(); if (!results.containsKey(partition)) { - results.put(partition, Lists.newArrayList()); + results.put(partition, new ArrayList<>()); } results.get(partition).add(r); }); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java index bdcbec71f..2c82fc152 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java @@ -74,8 +74,6 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLoggerExtension; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; - import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; @@ -618,7 +616,7 @@ public void testTableSink() { @Test public void testTableSinkSemanticTranslation() { - final List semantics = ImmutableList.of("exactly-once", "at-least-once", "none"); + final List semantics = Arrays.asList("exactly-once", "at-least-once", "none"); final EncodingFormat> valueEncodingFormat = new EncodingFormatMock(","); for (final String semantic : semantics) { diff --git a/pom.xml b/pom.xml index bfc91787a..76ea59d9c 100644 --- a/pom.xml +++ b/pom.xml @@ -80,12 +80,6 @@ under the License.
- - org.apache.flink - flink-shaded-guava - 30.1.1-jre-16.1 - - org.apache.flink flink-shaded-jackson From 27a546551b677330bf3cc38a02b569097619d8a5 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 13 Sep 2023 13:48:01 +0200 Subject: [PATCH 228/322] [FLINK-28758] Fix stop-with-savepoint for FlinkKafkaConsumer --- .../kafka/internals/KafkaFetcher.java | 7 + .../kafka/FlinkKafkaConsumerITCase.java | 129 ++++++++++++++++++ 2 files changed, 136 insertions(+) create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java index bee233975..9c4d8387c 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java @@ -24,6 +24,7 @@ import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.util.Collector; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.SerializedValue; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -142,6 +143,12 @@ public void runFetchLoop() throws Exception { partitionConsumerRecordsHandler(partitionRecords, partition); } } + } catch (Handover.ClosedException ex) { + if (running) { + // rethrow, only if we are running, if fetcher is not running we should not throw + // the ClosedException, as we are stopping gracefully + ExceptionUtils.rethrowException(ex); + } } finally { // this signals the consumer thread that no more work is to be done consumerThread.shutdown(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java new file mode 100644 index 000000000..90c773730 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.test.util.MiniClusterWithClientResource; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.junit.ClassRule; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.TestInstance.Lifecycle; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.Path; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; + +/** ITCase tests class for {@link FlinkKafkaConsumer}. */ +@TestInstance(Lifecycle.PER_CLASS) +public class FlinkKafkaConsumerITCase { + private static final String TOPIC1 = "FlinkKafkaConsumerITCase_topic1"; + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(new Configuration()) + .build()); + + @BeforeAll + public void setup() throws Throwable { + KafkaSourceTestEnv.setup(); + KafkaSourceTestEnv.setupTopic( + TOPIC1, true, true, KafkaSourceTestEnv::getRecordsForTopicWithoutTimestamp); + } + + @AfterAll + public void tearDown() throws Exception { + KafkaSourceTestEnv.tearDown(); + } + + @Test + public void testStopWithSavepoint(@TempDir Path savepointsDir) throws Exception { + Configuration config = + new Configuration() + .set( + CheckpointingOptions.SAVEPOINT_DIRECTORY, + savepointsDir.toUri().toString()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(1); + + Properties properties = new Properties(); + properties.setProperty( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, + KafkaSourceTestEnv.brokerConnectionStrings); + properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testStopWithSavepoint"); + properties.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + + FlinkKafkaConsumer kafkaConsumer = + new FlinkKafkaConsumer<>( + TOPIC1, + new TypeInformationSerializationSchema<>( + BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig()), + properties); + DataStreamSource stream = env.addSource(kafkaConsumer); + + ProgressLatchingIdentityFunction.resetBeforeUse(); + stream.map(new ProgressLatchingIdentityFunction()).addSink(new DiscardingSink<>()); + + JobClient jobClient = env.executeAsync(); + + ProgressLatchingIdentityFunction.getProgressLatch().await(); + + // Check that stopWithSavepoint completes successfully + jobClient.stopWithSavepoint(false, null, SavepointFormatType.CANONICAL).get(); + // TODO: ideally we should test recovery, that there were no data losses etc, but this + // is already a deprecated class, so I'm not adding new tests for that now. + } + + private static class ProgressLatchingIdentityFunction implements MapFunction { + + static CountDownLatch progressLatch; + + static void resetBeforeUse() { + progressLatch = new CountDownLatch(1); + } + + public static CountDownLatch getProgressLatch() { + return progressLatch; + } + + @Override + public Integer map(Integer integer) throws Exception { + progressLatch.countDown(); + return integer; + } + } +} From 4c03d60e0fcbaf4ff0d659f39a132fae93191c71 Mon Sep 17 00:00:00 2001 From: Hang Ruan Date: Mon, 11 Sep 2023 17:08:44 +0800 Subject: [PATCH 229/322] [hotfix][docs] Optimize the Security part in docs --- docs/content.zh/docs/connectors/table/kafka.md | 16 ++++++++-------- docs/content/docs/connectors/table/kafka.md | 9 +++++---- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/kafka.md b/docs/content.zh/docs/connectors/table/kafka.md index ce26e0728..27c7c9b19 100644 --- a/docs/content.zh/docs/connectors/table/kafka.md +++ b/docs/content.zh/docs/connectors/table/kafka.md @@ -572,8 +572,8 @@ Source 输出的 watermark 由读取的分区中最小的 watermark 决定。 请参阅 [Kafka watermark 策略]({{< ref "docs/dev/datastream/event-time/generating_watermarks" >}}#watermark-策略和-kafka-连接器) 以获取更多细节。 ### 安全 -要启用加密和认证相关的安全配置,只需将安全配置加上 "properties." 前缀配置在 Kafka 表上即可。下面的代码片段展示了如何配置 Kafka 表以使用 -PLAIN 作为 SASL 机制并提供 JAAS 配置: +要启用加密和认证相关的安全配置,只需将安全配置加上 "properties." 前缀配置在 Kafka 表上即可。下面的代码片段展示了当依赖 SQL client JAR 时, 如何配置 Kafka 表 +以使用 PLAIN 作为 SASL 机制并提供 JAAS 配置: ```sql CREATE TABLE KafkaTable ( `user_id` BIGINT, @@ -585,10 +585,10 @@ CREATE TABLE KafkaTable ( ... 'properties.security.protocol' = 'SASL_PLAINTEXT', 'properties.sasl.mechanism' = 'PLAIN', - 'properties.sasl.jaas.config' = 'org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";' + 'properties.sasl.jaas.config' = 'org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";' ) ``` -另一个更复杂的例子,使用 SASL_SSL 作为安全协议并使用 SCRAM-SHA-256 作为 SASL 机制: +另一个更复杂的例子,当依赖 SQL client JAR 时,使用 SASL_SSL 作为安全协议并使用 SCRAM-SHA-256 作为 SASL 机制: ```sql CREATE TABLE KafkaTable ( `user_id` BIGINT, @@ -610,13 +610,13 @@ CREATE TABLE KafkaTable ( /* 将 SASL 机制配置为 as SCRAM-SHA-256 */ 'properties.sasl.mechanism' = 'SCRAM-SHA-256', /* 配置 JAAS */ - 'properties.sasl.jaas.config' = 'org.apache.kafka.common.security.scram.ScramLoginModule required username=\"username\" password=\"password\";' + 'properties.sasl.jaas.config' = 'org.apache.flink.kafka.shaded.org.apache.kafka.common.security.scram.ScramLoginModule required username=\"username\" password=\"password\";' ) ``` -如果在作业 JAR 中 Kafka 客户端依赖的类路径被重置了(relocate class),登录模块(login module)的类路径可能会不同,因此请根据登录模块在 -JAR 中实际的类路径来改写以上配置。例如在 SQL client JAR 中,Kafka client 依赖被重置在了 `org.apache.flink.kafka.shaded.org.apache.kafka` 路径下, -因此 plain 登录模块的类路径应写为 `org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule`。 +在作业 JAR 中 Kafka 客户端依赖的类路径被重置了(relocate class),登录模块(login module)的类路径可能会不同,因此需要根据登录模块在 +JAR 中实际的类路径来改写以上配置。在 SQL client JAR 中,Kafka client 依赖被重置在了 `org.apache.flink.kafka.shaded.org.apache.kafka` +路径下,因此以上的代码片段中 plain 登录模块的类路径写为 `org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule`。 关于安全配置的详细描述,请参阅 Apache Kafka 文档中的"安全"一节。 diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md index 93123d89c..18578569a 100644 --- a/docs/content/docs/connectors/table/kafka.md +++ b/docs/content/docs/connectors/table/kafka.md @@ -629,7 +629,7 @@ for more details. ### Security In order to enable security configurations including encryption and authentication, you just need to setup security configurations with "properties." prefix in table options. The code snippet below shows configuring Kafka table to -use PLAIN as SASL mechanism and provide JAAS configuration: +use PLAIN as SASL mechanism and provide JAAS configuration when using SQL client JAR : ```sql CREATE TABLE KafkaTable ( `user_id` BIGINT, @@ -644,7 +644,7 @@ CREATE TABLE KafkaTable ( 'properties.sasl.jaas.config' = 'org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";' ) ``` -For a more complex example, use SASL_SSL as the security protocol and use SCRAM-SHA-256 as SASL mechanism: +For a more complex example, use SASL_SSL as the security protocol and use SCRAM-SHA-256 as SASL mechanism when using SQL client JAR : ```sql CREATE TABLE KafkaTable ( `user_id` BIGINT, @@ -672,8 +672,9 @@ CREATE TABLE KafkaTable ( Please note that the class path of the login module in `sasl.jaas.config` might be different if you relocate Kafka client dependencies, so you may need to rewrite it with the actual class path of the module in the JAR. -For example if you are using SQL client JAR, which has relocate Kafka client dependencies to `org.apache.flink.kafka.shaded.org.apache.kafka`, -the path of plain login module should be `org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule` instead. +SQL client JAR has relocated Kafka client dependencies to `org.apache.flink.kafka.shaded.org.apache.kafka`, +then the path of plain login module in code snippets above need to be +`org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule` when using SQL client JAR. For detailed explanations of security configurations, please refer to the "Security" section in Apache Kafka documentation. From d89a082180232bb79e3c764228c4e7dbb9eb6b8b Mon Sep 17 00:00:00 2001 From: Ethan Gouty Date: Wed, 8 Feb 2023 12:11:40 -0900 Subject: [PATCH 230/322] [FLINK-29398] Provide rack ID to KafkaSource to take advantage of Rack Awareness This closes #53. This closes #20. Co-authored-by: Jeremy DeGroot Co-authored-by: jcmejias1 Co-authored-by: Mason Chen Co-authored-by: Ethan Gouty Co-authored-by: Siva Venkat Gogineni --- .../docs/connectors/datastream/kafka.md | 19 +++++++ .../connector/kafka/source/KafkaSource.java | 17 +++++- .../kafka/source/KafkaSourceBuilder.java | 19 ++++++- .../reader/KafkaPartitionSplitReader.java | 23 ++++++++ .../reader/KafkaPartitionSplitReaderTest.java | 43 ++++++++++++++- .../source/reader/KafkaSourceReaderTest.java | 54 +++++++++++++++++-- 6 files changed, 168 insertions(+), 7 deletions(-) diff --git a/docs/content/docs/connectors/datastream/kafka.md b/docs/content/docs/connectors/datastream/kafka.md index acbbbc799..422ed9e36 100644 --- a/docs/content/docs/connectors/datastream/kafka.md +++ b/docs/content/docs/connectors/datastream/kafka.md @@ -465,6 +465,25 @@ client dependencies in the job JAR, so you may need to rewrite it with the actua For detailed explanations of security configurations, please refer to the "Security" section in Apache Kafka documentation. +## Kafka Rack Awareness + +Kafka rack awareness allows Flink to select and control the cloud region and availability zone that Kafka consumers read from, based on the Rack ID. This feature reduces network costs and latency since it allows consumers to connect to the closest Kafka brokers, possibly colocated in the same cloud region and availability zone. +A client's rack is indicated using the `client.rack` config, and should correspond to a broker's `broker.rack` config. + +https://kafka.apache.org/documentation/#consumerconfigs_client.rack + +### RackId + +setRackIdSupplier() is the Builder method allows us to determine the consumer's rack. If provided, the Supplier will be run when the consumer is set up on the Task Manager, and the consumer's `client.rack` configuration will be set to the value. + +One of the ways this can be implemented is by making setRackId equal to an environment variable within your taskManager, for instance: + +``` +.setRackIdSupplier(() -> System.getenv("TM_NODE_AZ")) +``` + +The "TM_NODE_AZ" is the name of the environment variable in the TaskManager container that contains the zone we want to use. + ### Behind the Scene {{< hint info >}} If you are interested in how Kafka source works under the design of new data source API, you may diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java index 0e764649f..54f5f856c 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java @@ -49,6 +49,7 @@ import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.util.UserCodeClassLoader; +import org.apache.flink.util.function.SerializableSupplier; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -56,6 +57,7 @@ import java.io.IOException; import java.util.Collection; +import java.util.Optional; import java.util.Properties; import java.util.function.Consumer; import java.util.function.Supplier; @@ -98,6 +100,8 @@ public class KafkaSource private final KafkaRecordDeserializationSchema deserializationSchema; // The configurations. private final Properties props; + // Client rackId callback + private final SerializableSupplier rackIdSupplier; KafkaSource( KafkaSubscriber subscriber, @@ -105,13 +109,15 @@ public class KafkaSource @Nullable OffsetsInitializer stoppingOffsetsInitializer, Boundedness boundedness, KafkaRecordDeserializationSchema deserializationSchema, - Properties props) { + Properties props, + SerializableSupplier rackIdSupplier) { this.subscriber = subscriber; this.startingOffsetsInitializer = startingOffsetsInitializer; this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; this.boundedness = boundedness; this.deserializationSchema = deserializationSchema; this.props = props; + this.rackIdSupplier = rackIdSupplier; } /** @@ -157,7 +163,14 @@ public UserCodeClassLoader getUserCodeClassLoader() { new KafkaSourceReaderMetrics(readerContext.metricGroup()); Supplier splitReaderSupplier = - () -> new KafkaPartitionSplitReader(props, readerContext, kafkaSourceReaderMetrics); + () -> + new KafkaPartitionSplitReader( + props, + readerContext, + kafkaSourceReaderMetrics, + Optional.ofNullable(rackIdSupplier) + .map(Supplier::get) + .orElse(null)); KafkaRecordEmitter recordEmitter = new KafkaRecordEmitter<>(deserializationSchema); return new KafkaSourceReader<>( diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java index afaa72db7..dcad476ba 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java @@ -26,6 +26,7 @@ import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializerValidator; import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.util.function.SerializableSupplier; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.TopicPartition; @@ -80,6 +81,7 @@ * .setTopics(Arrays.asList(TOPIC1, TOPIC2)) * .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) * .setUnbounded(OffsetsInitializer.latest()) + * .setRackId(() -> MY_RACK_ID) * .build(); * } * @@ -100,6 +102,8 @@ public class KafkaSourceBuilder { private KafkaRecordDeserializationSchema deserializationSchema; // The configurations. protected Properties props; + // Client rackId supplier + private SerializableSupplier rackIdSupplier; KafkaSourceBuilder() { this.subscriber = null; @@ -108,6 +112,7 @@ public class KafkaSourceBuilder { this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; this.deserializationSchema = null; this.props = new Properties(); + this.rackIdSupplier = null; } /** @@ -355,6 +360,17 @@ public KafkaSourceBuilder setClientIdPrefix(String prefix) { return setProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key(), prefix); } + /** + * Set the clientRackId supplier to be passed down to the KafkaPartitionSplitReader. + * + * @param rackIdCallback callback to provide Kafka consumer client.rack + * @return this KafkaSourceBuilder + */ + public KafkaSourceBuilder setRackIdSupplier(SerializableSupplier rackIdCallback) { + this.rackIdSupplier = rackIdCallback; + return this; + } + /** * Set an arbitrary property for the KafkaSource and KafkaConsumer. The valid keys can be found * in {@link ConsumerConfig} and {@link KafkaSourceOptions}. @@ -422,7 +438,8 @@ public KafkaSource build() { stoppingOffsetsInitializer, boundedness, deserializationSchema, - props); + props, + rackIdSupplier); } // ------------- private helpers -------------- diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java index f52940c4b..94940b8e7 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java @@ -80,11 +80,20 @@ public KafkaPartitionSplitReader( Properties props, SourceReaderContext context, KafkaSourceReaderMetrics kafkaSourceReaderMetrics) { + this(props, context, kafkaSourceReaderMetrics, null); + } + + public KafkaPartitionSplitReader( + Properties props, + SourceReaderContext context, + KafkaSourceReaderMetrics kafkaSourceReaderMetrics, + String rackIdSupplier) { this.subtaskId = context.getIndexOfSubtask(); this.kafkaSourceReaderMetrics = kafkaSourceReaderMetrics; Properties consumerProps = new Properties(); consumerProps.putAll(props); consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, createConsumerClientId(props)); + setConsumerClientRack(consumerProps, rackIdSupplier); this.consumer = new KafkaConsumer<>(consumerProps); this.stoppingOffsets = new HashMap<>(); this.groupId = consumerProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG); @@ -256,6 +265,20 @@ KafkaConsumer consumer() { // --------------- private helper method ---------------------- + /** + * This Method performs Null and empty Rack Id validation and sets the rack id to the + * client.rack Consumer Config. + * + * @param consumerProps Consumer Property. + * @param rackId Rack Id's. + */ + @VisibleForTesting + void setConsumerClientRack(Properties consumerProps, String rackId) { + if (rackId != null && !rackId.isEmpty()) { + consumerProps.setProperty(ConsumerConfig.CLIENT_RACK_CONFIG, rackId); + } + } + private void parseStartingOffsets( KafkaPartitionSplit split, List partitionsStartingFromEarliest, diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java index 7263bd028..edd41326a 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java @@ -48,6 +48,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.EmptySource; +import org.junit.jupiter.params.provider.NullAndEmptySource; import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; @@ -319,6 +320,38 @@ public void testUsingCommittedOffsetsWithEarliestOrLatestOffsetResetStrategy( assertThat(reader.consumer().position(partition)).isEqualTo(expectedOffset); } + @Test + public void testConsumerClientRackSupplier() { + String rackId = "use1-az1"; + Properties properties = new Properties(); + KafkaPartitionSplitReader reader = + createReader( + properties, + UnregisteredMetricsGroup.createSourceReaderMetricGroup(), + rackId); + + // Here we call the helper function directly, because the KafkaPartitionSplitReader + // doesn't allow us to examine the final ConsumerConfig object + reader.setConsumerClientRack(properties, rackId); + assertThat(properties.get(ConsumerConfig.CLIENT_RACK_CONFIG)).isEqualTo(rackId); + } + + @ParameterizedTest + @NullAndEmptySource + public void testSetConsumerClientRackIgnoresNullAndEmpty(String rackId) { + Properties properties = new Properties(); + KafkaPartitionSplitReader reader = + createReader( + properties, + UnregisteredMetricsGroup.createSourceReaderMetricGroup(), + rackId); + + // Here we call the helper function directly, because the KafkaPartitionSplitReader + // doesn't allow us to examine the final ConsumerConfig object + reader.setConsumerClientRack(properties, rackId); + assertThat(properties.containsKey(ConsumerConfig.CLIENT_RACK_CONFIG)).isFalse(); + } + // ------------------ private void assignSplitsAndFetchUntilFinish(KafkaPartitionSplitReader reader, int readerId) @@ -383,6 +416,13 @@ private KafkaPartitionSplitReader createReader() { private KafkaPartitionSplitReader createReader( Properties additionalProperties, SourceReaderMetricGroup sourceReaderMetricGroup) { + return createReader(additionalProperties, sourceReaderMetricGroup, null); + } + + private KafkaPartitionSplitReader createReader( + Properties additionalProperties, + SourceReaderMetricGroup sourceReaderMetricGroup, + String rackId) { Properties props = new Properties(); props.putAll(KafkaSourceTestEnv.getConsumerProperties(ByteArrayDeserializer.class)); props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none"); @@ -394,7 +434,8 @@ private KafkaPartitionSplitReader createReader( return new KafkaPartitionSplitReader( props, new TestingReaderContext(new Configuration(), sourceReaderMetricGroup), - kafkaSourceReaderMetrics); + kafkaSourceReaderMetrics, + rackId); } private Map assignSplits( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java index e4ee39d23..7f879f21c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java @@ -40,6 +40,7 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.testutils.MetricListener; import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup; +import org.apache.flink.util.function.SerializableSupplier; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.NewTopic; @@ -53,6 +54,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import java.time.Duration; import java.util.ArrayList; @@ -79,6 +81,8 @@ import static org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv.NUM_PARTITIONS; import static org.apache.flink.core.testutils.CommonTestUtils.waitUtil; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; /** Unit tests for {@link KafkaSourceReader}. */ public class KafkaSourceReaderTest extends SourceReaderTestBase { @@ -271,7 +275,8 @@ void testDisableOffsetCommit() throws Exception { Boundedness.CONTINUOUS_UNBOUNDED, new TestingReaderContext(), (ignore) -> {}, - properties)) { + properties, + null)) { reader.addSplits( getSplits(numSplits, NUM_RECORDS_PER_SPLIT, Boundedness.CONTINUOUS_UNBOUNDED)); ValidatingSourceOutput output = new ValidatingSourceOutput(); @@ -479,6 +484,45 @@ public void testSupportsPausingOrResumingSplits() throws Exception { } } + @Test + public void testThatReaderDoesNotCallRackIdSupplierOnInit() throws Exception { + SerializableSupplier rackIdSupplier = Mockito.mock(SerializableSupplier.class); + + try (KafkaSourceReader reader = + (KafkaSourceReader) + createReader( + Boundedness.CONTINUOUS_UNBOUNDED, + new TestingReaderContext(), + (ignore) -> {}, + new Properties(), + rackIdSupplier)) { + // Do nothing here + } + + verify(rackIdSupplier, never()).get(); + } + + @Test + public void testThatReaderDoesCallRackIdSupplierOnSplitAssignment() throws Exception { + SerializableSupplier rackIdSupplier = Mockito.mock(SerializableSupplier.class); + Mockito.when(rackIdSupplier.get()).thenReturn("use1-az1"); + + try (KafkaSourceReader reader = + (KafkaSourceReader) + createReader( + Boundedness.CONTINUOUS_UNBOUNDED, + new TestingReaderContext(), + (ignore) -> {}, + new Properties(), + rackIdSupplier)) { + reader.addSplits( + Collections.singletonList( + new KafkaPartitionSplit(new TopicPartition(TOPIC, 1), 1L))); + } + + verify(rackIdSupplier).get(); + } + // ------------------------------------------ @Override @@ -535,14 +579,15 @@ private SourceReader createReader( throws Exception { Properties properties = new Properties(); properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId); - return createReader(boundedness, context, splitFinishedHook, properties); + return createReader(boundedness, context, splitFinishedHook, properties, null); } private SourceReader createReader( Boundedness boundedness, SourceReaderContext context, Consumer> splitFinishedHook, - Properties props) + Properties props, + SerializableSupplier rackIdSupplier) throws Exception { KafkaSourceBuilder builder = KafkaSource.builder() @@ -559,6 +604,9 @@ private SourceReader createReader( if (boundedness == Boundedness.BOUNDED) { builder.setBounded(OffsetsInitializer.latest()); } + if (rackIdSupplier != null) { + builder.setRackIdSupplier(rackIdSupplier); + } return KafkaSourceTestUtils.createReaderWithFinishedSplitHook( builder.build(), context, splitFinishedHook); From b09928d5ef290f2a046dc1fe40b4c5cebe76f997 Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Mon, 9 Oct 2023 20:46:35 +0800 Subject: [PATCH 231/322] [FLINK-33219][connector/kafka] Add new archunit violation messages introduced by FLINK-31804. This closes #56 * [FLINK-33219][connector/kafka] Add new archunit violation messages introduced by FLINK-31804 The reason we add new violation messages instead of update existing ones is that the patch of FLINK-31804 is only applied after Flink 1.18. We need to make sure the CI could run successfully for Flink versions before and after that. If Kafka connector decides to drop support for versions before 1.18 in the future, please re-freeze the violations then. Co-authored-by: Martijn Visser --- .github/workflows/push_pr.yml | 5 +- .../86dfd459-67a9-4b26-9b5c-0b0bbf22681a | 78 +++++++++++++++++++ 2 files changed, 82 insertions(+), 1 deletion(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 335f108f6..3e54984ca 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -23,6 +23,9 @@ concurrency: cancel-in-progress: true jobs: compile_and_test: + strategy: + matrix: + flink: [ 1.17.1 ] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: - flink_version: 1.17.0 + flink_version: ${{ matrix.flink }} diff --git a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a index c1e656185..d952f0267 100644 --- a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a +++ b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a @@ -74,5 +74,83 @@ org.apache.flink.streaming.connectors.kafka.shuffle.KafkaShuffleITCase does not * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.KafkaSinkITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.KafkaTransactionLogITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.source.KafkaSourceITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.source.KafkaSourceLegacyITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.kafka.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.kafka.KafkaITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.kafka.KafkaProducerAtLeastOnceITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.kafka.KafkaProducerExactlyOnceITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.kafka.shuffle.KafkaShuffleExactlyOnceITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.kafka.shuffle.KafkaShuffleITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file From 73f761fa73d4200d18f628eef7c79cf91dd1a0bc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 15 Jun 2023 16:21:31 +0000 Subject: [PATCH 232/322] [FLINK-33149][Connector/Kafka] Bump snappy-java from 1.1.8.3 to 1.1.10.5 Bumps [snappy-java](https://github.com/xerial/snappy-java) from 1.1.8.3 to 1.1.10.5. - [Release notes](https://github.com/xerial/snappy-java/releases) - [Commits](https://github.com/xerial/snappy-java/compare/1.1.8.3...v1.1.10.5) --- updated-dependencies: - dependency-name: org.xerial.snappy:snappy-java dependency-type: direct:production ... Signed-off-by: dependabot[bot] --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 76ea59d9c..96e4521bf 100644 --- a/pom.xml +++ b/pom.xml @@ -67,7 +67,7 @@ under the License. 2.12 2.12.7 2.12.7 - 1.1.8.3 + 1.1.10.5 1.11.1 false From b0f15f279915a3862d353c9d2726a52741c248ba Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 10 Oct 2023 12:18:19 -0700 Subject: [PATCH 233/322] [FLINK-33231] [source] Properly evict offsetsToCommit cache on checkpoint complete if no offsets exist Prior to this fix, if the offsets to commit for a given checkpoint is empty, which can be the case if no starting offsets were retrieved from Kafka yet, then on checkpoint completion the cache is not properly evicted up to the given checkpoint. This change fixes this such that in notifyOnCheckpointComplete, we shortcut the method execution to not need to try to commit the offsets since its empty anyways, and always remember to evict the cache up to the completed checkpoint. --- .../source/reader/KafkaSourceReader.java | 21 ++++++++++++------ .../source/reader/KafkaSourceReaderTest.java | 22 ++++++++++++++++++- 2 files changed, 35 insertions(+), 8 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java index a3aa555f1..cb85fc89d 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java @@ -132,9 +132,13 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { Map committedPartitions = offsetsToCommit.get(checkpointId); if (committedPartitions == null) { - LOG.debug( - "Offsets for checkpoint {} either do not exist or have already been committed.", - checkpointId); + LOG.debug("Offsets for checkpoint {} have already been committed.", checkpointId); + return; + } + + if (committedPartitions.isEmpty()) { + LOG.debug("There are no offsets to commit for checkpoint {}.", checkpointId); + removeAllOffsetsToCommitUpToCheckpoint(checkpointId); return; } @@ -167,14 +171,17 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { entry -> committedPartitions.containsKey( entry.getKey())); - while (!offsetsToCommit.isEmpty() - && offsetsToCommit.firstKey() <= checkpointId) { - offsetsToCommit.remove(offsetsToCommit.firstKey()); - } + removeAllOffsetsToCommitUpToCheckpoint(checkpointId); } }); } + private void removeAllOffsetsToCommitUpToCheckpoint(long checkpointId) { + while (!offsetsToCommit.isEmpty() && offsetsToCommit.firstKey() <= checkpointId) { + offsetsToCommit.remove(offsetsToCommit.firstKey()); + } + } + @Override protected KafkaPartitionSplitState initializedState(KafkaPartitionSplit split) { return new KafkaPartitionSplitState(split); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java index 7f879f21c..b350d8c6b 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java @@ -61,6 +61,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -187,7 +188,26 @@ void testCommitEmptyOffsets() throws Exception { (KafkaSourceReader) createReader(Boundedness.CONTINUOUS_UNBOUNDED, groupId)) { reader.snapshotState(100L); - reader.notifyCheckpointComplete(100L); + reader.snapshotState(101L); + reader.snapshotState(102L); + + // After each snapshot, a new entry should have been added to the offsets-to-commit + // cache for the checkpoint + final Map> expectedOffsetsToCommit = + new HashMap<>(); + expectedOffsetsToCommit.put(100L, new HashMap<>()); + expectedOffsetsToCommit.put(101L, new HashMap<>()); + expectedOffsetsToCommit.put(102L, new HashMap<>()); + assertThat(reader.getOffsetsToCommit()).isEqualTo(expectedOffsetsToCommit); + + // only notify up to checkpoint 101L; all offsets prior to 101L should be evicted from + // cache, leaving only 102L + reader.notifyCheckpointComplete(101L); + + final Map> + expectedOffsetsToCommitAfterNotify = new HashMap<>(); + expectedOffsetsToCommitAfterNotify.put(102L, new HashMap<>()); + assertThat(reader.getOffsetsToCommit()).isEqualTo(expectedOffsetsToCommitAfterNotify); } // Verify the committed offsets. try (AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) { From 89343bb5d226e7e6d52e0ff18b172cf2156087a7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Oct 2023 13:27:17 +0000 Subject: [PATCH 234/322] [FLINK-33234][Connector/Kafka] Bump Guava to 32.1.2-jre Bumps [guava](https://github.com/google/guava) from 30.1.1-jre to 32.1.2-jre. - [Release notes](https://github.com/google/guava/releases) - [Commits](https://github.com/google/guava/commits) --- updated-dependencies: - dependency-name: com.google.guava:guava dependency-type: direct:production ... Signed-off-by: dependabot[bot] --- .../flink-end-to-end-tests-common-kafka/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index be24c03e4..ffd724209 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -155,7 +155,7 @@ under the License. com.google.guava guava - 30.1.1-jre + 32.1.2-jre @@ -234,7 +234,7 @@ under the License. com.google.guava guava - 30.1.1-jre + 32.1.2-jre guava.jar jar ${project.build.directory}/dependencies From 26ab532b376d143f721d3ee89f64500577dfb84a Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Wed, 11 Oct 2023 09:55:48 +0200 Subject: [PATCH 235/322] [FLINK-33234][Connector/Kafka] Exclude bundled Guava versions from Kafka AVRO serializer and Kafka Schema Registry Client --- .../flink-end-to-end-tests-common-kafka/pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index ffd724209..4b08f59fa 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -106,6 +106,12 @@ under the License. kafka-avro-serializer ${confluent.version} test + + + com.google.guava + guava + +
@@ -118,6 +124,10 @@ under the License. org.apache.kafka kafka-clients + + com.google.guava + guava + From e9d3089791391cf3fcd632abbf7ea4e6cf8e8ada Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Mon, 9 Oct 2023 15:23:15 +0200 Subject: [PATCH 236/322] [FLINK-33191][Connector/Kafka] Remove dependency on Flink Shaded --- .../kafka/util/JacksonMapperFactory.java | 49 +++++++++++++++++++ .../JSONKeyValueDeserializationSchema.java | 9 ++-- .../KafkaRecordDeserializationSchemaTest.java | 9 ++-- ...JSONKeyValueDeserializationSchemaTest.java | 7 ++- pom.xml | 24 ++++++--- 5 files changed, 78 insertions(+), 20 deletions(-) create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/util/JacksonMapperFactory.java diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/util/JacksonMapperFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/util/JacksonMapperFactory.java new file mode 100644 index 000000000..c9301c79c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/util/JacksonMapperFactory.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.util; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; + +/** Factory for Jackson mappers. */ +public final class JacksonMapperFactory { + + public static ObjectMapper createObjectMapper() { + final ObjectMapper objectMapper = new ObjectMapper(); + registerModules(objectMapper); + return objectMapper; + } + + public static ObjectMapper createObjectMapper(JsonFactory jsonFactory) { + final ObjectMapper objectMapper = new ObjectMapper(jsonFactory); + registerModules(objectMapper); + return objectMapper; + } + + private static void registerModules(ObjectMapper mapper) { + mapper.registerModule(new JavaTimeModule()) + .registerModule(new Jdk8Module().configureAbsentsAsNulls(true)) + .disable(SerializationFeature.WRITE_DURATIONS_AS_TIMESTAMPS) + .disable(SerializationFeature.WRITE_DATES_AS_TIMESTAMPS); + } + + private JacksonMapperFactory() {} +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java index e2b428eec..970bad1c3 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java @@ -20,13 +20,12 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.kafka.util.JacksonMapperFactory; import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.clients.consumer.ConsumerRecord; import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java index e764c8609..d61b7f834 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java @@ -18,17 +18,16 @@ package org.apache.flink.connector.kafka.source.reader.deserializer; +import org.apache.flink.connector.kafka.util.JacksonMapperFactory; import org.apache.flink.connector.testutils.formats.DummyInitializationContext; import org.apache.flink.connector.testutils.source.deserialization.TestingDeserializationContext; import org.apache.flink.formats.json.JsonDeserializationSchema; import org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema; import org.apache.flink.util.Collector; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java index ddbcf1c94..a5abb5e6d 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java @@ -17,13 +17,12 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.connector.kafka.util.JacksonMapperFactory; import org.apache.flink.connector.testutils.formats.DummyInitializationContext; import org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.Test; diff --git a/pom.xml b/pom.xml index 96e4521bf..607916de5 100644 --- a/pom.xml +++ b/pom.xml @@ -54,7 +54,7 @@ under the License. 3.5.9 7.2.2 - 2.13.4.20221013 + 2.15.2 4.13.2 5.9.1 3.23.1 @@ -80,13 +80,25 @@ under the License. + - org.apache.flink - flink-shaded-jackson - 2.13.4-16.1 + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + + com.fasterxml.jackson.datatype + jackson-datatype-jsr310 + + + + com.fasterxml.jackson.datatype + jackson-datatype-jdk8 - - From bd260f14dd280f464e5dcea76b3735782312b4ae Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Tue, 10 Oct 2023 13:11:38 +0200 Subject: [PATCH 237/322] [FLINK-33191][Connector/Kafka] Make the `testKafkaValueDeserializationSchemaWrapper` test use `flink-shaded-jackson` since it tests `flink-shaded-jackson` ObjectNodes. Co-authored-by: zentol This closes #57. --- .../KafkaRecordDeserializationSchemaTest.java | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java index d61b7f834..b0ca63161 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java @@ -81,16 +81,24 @@ public void testKafkaDeserializationSchemaWrapper() throws Exception { @Test public void testKafkaValueDeserializationSchemaWrapper() throws Exception { final ConsumerRecord consumerRecord = getConsumerRecord(); - KafkaRecordDeserializationSchema schema = - KafkaRecordDeserializationSchema.valueOnly( - new JsonDeserializationSchema<>(ObjectNode.class)); + KafkaRecordDeserializationSchema< + org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node + .ObjectNode> + schema = + KafkaRecordDeserializationSchema.valueOnly( + new JsonDeserializationSchema<>( + org.apache.flink.shaded.jackson2.com.fasterxml.jackson + .databind.node.ObjectNode.class)); schema.open(new DummyInitializationContext()); - SimpleCollector collector = new SimpleCollector<>(); + SimpleCollector< + org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node + .ObjectNode> + collector = new SimpleCollector<>(); schema.deserialize(consumerRecord, collector); assertThat(collector.list).hasSize(1); - ObjectNode deserializedValue = collector.list.get(0); - + org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode + deserializedValue = collector.list.get(0); assertThat(deserializedValue.get("word").asText()).isEqualTo("world"); assertThat(deserializedValue.get("key")).isNull(); assertThat(deserializedValue.get("metadata")).isNull(); From c829d7a81ae0792af19525d257226c84fab7370b Mon Sep 17 00:00:00 2001 From: MartijnVisser Date: Wed, 11 Oct 2023 21:54:24 +0200 Subject: [PATCH 238/322] [FLINK-33238][Formats/Avro] Upgrade used AVRO version to 1.11.3. This closes #59 * [FLINK-33238][Formats/Avro] Upgrade used AVRO version to 1.11.3 to mitigate scanners flagging Flink or the Flink Kafka connector as vulnerable for CVE-2023-39410 * [FLINK-33238][Formats/Avro] Pin transitive dependency org.apache.commons:commons-compress to 1.22 to address dependency convergence --- pom.xml | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 607916de5..ff8efac36 100644 --- a/pom.xml +++ b/pom.xml @@ -68,7 +68,7 @@ under the License. 2.12.7 2.12.7 1.1.10.5 - 1.11.1 + 1.11.3 false 1.17.0 @@ -405,6 +405,13 @@ under the License. 2.1 + + + org.apache.commons + commons-compress + 1.22 + + org.testcontainers testcontainers-bom From 398054aeb50c63368c81e4acf7a3193f02cdfedf Mon Sep 17 00:00:00 2001 From: MartijnVisser Date: Mon, 16 Oct 2023 11:37:41 +0200 Subject: [PATCH 239/322] [hotfix] Update CI strategy. This closes #61 1. Test all PRs for `main` against all supported versions, meaning 1.17.x and 1.18.x. That's because only PRs run the dependency convergence check, and not nightly builds. 2. Make sure that we test nightlies against all supported versions (currently 1.17.x for the `v3.0` branch plus 1.17.x and 1.18.x against `main`) --- .github/workflows/push_pr.yml | 2 +- .github/workflows/weekly.yml | 14 ++++++++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 3e54984ca..be8e32702 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -25,7 +25,7 @@ jobs: compile_and_test: strategy: matrix: - flink: [ 1.17.1 ] + flink: [ 1.17.1, 1.18-SNAPSHOT ] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index b4daceb73..a9de998c9 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -26,8 +26,18 @@ jobs: if: github.repository_owner == 'apache' strategy: matrix: - flink: [1.17-SNAPSHOT, 1.18-SNAPSHOT] + flink_branches: [{ + flink: 1.17.1, + branch: v3.0 + }, { + flink: 1.17.1, + branch: main + }, { + flink: 1.18-SNAPSHOT, + branch: main + }] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: - flink_version: ${{ matrix.flink }} + flink_version: ${{ matrix.flink_branches.flink }} + connector_branch: ${{ matrix.flink_branches.branch }} run_dependency_convergence: false From 37cbb83f55e9d6f0b8dc35bb8da867086dfa4d9e Mon Sep 17 00:00:00 2001 From: Hang Ruan Date: Mon, 16 Oct 2023 23:58:23 +0800 Subject: [PATCH 240/322] [FLINK-30400][build] Stop bundling flink-connector-base. This closes #50 * [FLINK-30400][build] Stop bundling flink-connector-base --------- Co-authored-by: Martijn Visser --- .../flink-end-to-end-tests-common-kafka/pom.xml | 6 ++++++ flink-connector-kafka/pom.xml | 1 + flink-sql-connector-kafka/pom.xml | 1 - 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index 4b08f59fa..94227a4b4 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -67,6 +67,12 @@ under the License. + + org.apache.flink + flink-connector-base + ${flink.version} + test + org.apache.kafka diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index fe097505b..7c549fdac 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -51,6 +51,7 @@ under the License. org.apache.flink flink-connector-base ${flink.version} + provided diff --git a/flink-sql-connector-kafka/pom.xml b/flink-sql-connector-kafka/pom.xml index ca877f3eb..d22899e4f 100644 --- a/flink-sql-connector-kafka/pom.xml +++ b/flink-sql-connector-kafka/pom.xml @@ -67,7 +67,6 @@ under the License. - org.apache.flink:flink-connector-base org.apache.flink:flink-connector-kafka org.apache.kafka:* From 946df1ed8b03bc41f3aaf8851912e76c239ba550 Mon Sep 17 00:00:00 2001 From: tanjialiang Date: Tue, 26 Sep 2023 19:10:17 +0800 Subject: [PATCH 241/322] [FLINK-28303] Support LatestOffsetsInitializer to avoid latest-offset strategy lose data This closes #52. --- .../initializer/LatestOffsetsInitializer.java | 47 ++++++ .../initializer/OffsetsInitializer.java | 3 +- .../ReaderHandledOffsetsInitializer.java | 9 +- .../source/split/KafkaPartitionSplit.java | 19 +-- .../enumerator/KafkaEnumeratorTest.java | 2 +- .../initializer/OffsetsInitializerTest.java | 2 +- .../reader/KafkaPartitionSplitReaderTest.java | 4 +- .../source/reader/KafkaSourceReaderTest.java | 4 +- .../KafkaPartitionSplitSerializerTest.java | 6 +- .../table/KafkaDynamicTableFactoryTest.java | 8 +- .../kafka/table/KafkaTableITCase.java | 139 ++++++++++++++++++ .../UpsertKafkaDynamicTableFactoryTest.java | 8 +- .../MockPartitionOffsetsRetriever.java | 11 ++ 13 files changed, 230 insertions(+), 32 deletions(-) create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/LatestOffsetsInitializer.java diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/LatestOffsetsInitializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/LatestOffsetsInitializer.java new file mode 100644 index 000000000..b6c95a65e --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/LatestOffsetsInitializer.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.source.enumerator.initializer; + +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; + +import java.util.Collection; +import java.util.Map; + +/** + * An implementation of {@link OffsetsInitializer} to initialize the offsets based on a + * latest-offset. + * + *

Package private and should be instantiated via {@link OffsetsInitializer}. + */ +class LatestOffsetsInitializer implements OffsetsInitializer { + private static final long serialVersionUID = 3014700244733286989L; + + @Override + public Map getPartitionOffsets( + Collection partitions, + PartitionOffsetsRetriever partitionOffsetsRetriever) { + return partitionOffsetsRetriever.endOffsets(partitions); + } + + @Override + public OffsetResetStrategy getAutoOffsetResetStrategy() { + return OffsetResetStrategy.LATEST; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java index db682c6b0..0f0c5d25b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java @@ -155,8 +155,7 @@ static OffsetsInitializer earliest() { * @return an {@link OffsetsInitializer} which initializes the offsets to the latest offsets. */ static OffsetsInitializer latest() { - return new ReaderHandledOffsetsInitializer( - KafkaPartitionSplit.LATEST_OFFSET, OffsetResetStrategy.LATEST); + return new LatestOffsetsInitializer(); } /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java index 026320d93..42abd5778 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java @@ -32,9 +32,9 @@ import static org.apache.flink.util.Preconditions.checkState; /** - * A initializer that initialize the partitions to the earliest / latest / last-committed offsets. - * The offsets initialization are taken care of by the {@code KafkaPartitionSplitReader} instead of - * by the {@code KafkaSourceEnumerator}. + * A initializer that initialize the partitions to the earliest / last-committed offsets. The + * offsets initialization are taken care of by the {@code KafkaPartitionSplitReader} instead of by + * the {@code KafkaSourceEnumerator}. * *

Package private and should be instantiated via {@link OffsetsInitializer}. */ @@ -46,8 +46,7 @@ class ReaderHandledOffsetsInitializer implements OffsetsInitializer, OffsetsInit /** * The only valid value for startingOffset is following. {@link * KafkaPartitionSplit#EARLIEST_OFFSET EARLIEST_OFFSET}, {@link - * KafkaPartitionSplit#LATEST_OFFSET LATEST_OFFSET}, {@link KafkaPartitionSplit#COMMITTED_OFFSET - * COMMITTED_OFFSET} + * KafkaPartitionSplit#COMMITTED_OFFSET COMMITTED_OFFSET} */ ReaderHandledOffsetsInitializer(long startingOffset, OffsetResetStrategy offsetResetStrategy) { this.startingOffset = startingOffset; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java index 8c2a1fd1f..ef1b8b88f 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java @@ -35,7 +35,8 @@ public class KafkaPartitionSplit implements SourceSplit { public static final long NO_STOPPING_OFFSET = Long.MIN_VALUE; // Indicating the split should consume from the latest. - public static final long LATEST_OFFSET = -1; + // @deprecated Only be used for compatibility with the history state, see FLINK-28303 + @Deprecated public static final long LATEST_OFFSET = -1; // Indicating the split should consume from the earliest. public static final long EARLIEST_OFFSET = -2; // Indicating the split should consume from the last committed offset. @@ -43,9 +44,9 @@ public class KafkaPartitionSplit implements SourceSplit { // Valid special starting offsets public static final Set VALID_STARTING_OFFSET_MARKERS = - new HashSet<>(Arrays.asList(EARLIEST_OFFSET, LATEST_OFFSET, COMMITTED_OFFSET)); + new HashSet<>(Arrays.asList(EARLIEST_OFFSET, COMMITTED_OFFSET)); public static final Set VALID_STOPPING_OFFSET_MARKERS = - new HashSet<>(Arrays.asList(LATEST_OFFSET, COMMITTED_OFFSET, NO_STOPPING_OFFSET)); + new HashSet<>(Arrays.asList(COMMITTED_OFFSET, NO_STOPPING_OFFSET)); private final TopicPartition tp; private final long startingOffset; @@ -132,8 +133,8 @@ private static void verifyInitialOffset( String.format( "Invalid starting offset %d is specified for partition %s. " + "It should either be non-negative or be one of the " - + "[%d(earliest), %d(latest), %d(committed)].", - startingOffset, tp, LATEST_OFFSET, EARLIEST_OFFSET, COMMITTED_OFFSET)); + + "[%d(earliest), %d(committed)].", + startingOffset, tp, EARLIEST_OFFSET, COMMITTED_OFFSET)); } if (stoppingOffset < 0 && !VALID_STOPPING_OFFSET_MARKERS.contains(stoppingOffset)) { @@ -141,12 +142,8 @@ private static void verifyInitialOffset( String.format( "Illegal stopping offset %d is specified for partition %s. " + "It should either be non-negative or be one of the " - + "[%d(latest), %d(committed), %d(Long.MIN_VALUE, no_stopping_offset)].", - stoppingOffset, - tp, - LATEST_OFFSET, - COMMITTED_OFFSET, - NO_STOPPING_OFFSET)); + + "[%d(committed), %d(Long.MIN_VALUE, no_stopping_offset)].", + stoppingOffset, tp, COMMITTED_OFFSET, NO_STOPPING_OFFSET)); } } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java index f30c660d7..8b308af16 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java @@ -300,7 +300,7 @@ public void testDiscoverPartitionsPeriodically() throws Throwable { getAllAssignSplits(context, PRE_EXISTING_TOPICS); assertThat(initialPartitionAssign) .extracting(KafkaPartitionSplit::getStartingOffset) - .containsOnly(KafkaPartitionSplit.LATEST_OFFSET); + .containsOnly((long) KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION); List newPartitionAssign = getAllAssignSplits(context, Collections.singleton(DYNAMIC_TOPIC_NAME)); assertThat(newPartitionAssign) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java index e0cd8506d..46dd61a6f 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java @@ -84,7 +84,7 @@ public void testLatestOffsetsInitializer() { assertThat(offsets).hasSameSizeAs(partitions); assertThat(offsets.keySet()).containsAll(partitions); for (long offset : offsets.values()) { - assertThat(offset).isEqualTo(KafkaPartitionSplit.LATEST_OFFSET); + assertThat(offset).isEqualTo(KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION); } assertThat(initializer.getAutoOffsetResetStrategy()).isEqualTo(OffsetResetStrategy.LATEST); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java index edd41326a..b592a6917 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java @@ -246,8 +246,8 @@ public void testAssignEmptySplit() throws Exception { final KafkaPartitionSplit emptySplit = new KafkaPartitionSplit( new TopicPartition(TOPIC2, 0), - KafkaPartitionSplit.LATEST_OFFSET, - KafkaPartitionSplit.LATEST_OFFSET); + KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION, + KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION); final KafkaPartitionSplit emptySplitWithZeroStoppingOffset = new KafkaPartitionSplit(new TopicPartition(TOPIC3, 0), 0, 0); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java index b350d8c6b..f5aa7f5fd 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java @@ -397,7 +397,9 @@ void testAssigningEmptySplits() throws Exception { // Normal split with NUM_RECORDS_PER_SPLIT records final KafkaPartitionSplit normalSplit = new KafkaPartitionSplit( - new TopicPartition(TOPIC, 0), 0, KafkaPartitionSplit.LATEST_OFFSET); + new TopicPartition(TOPIC, 0), + 0, + KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION); // Empty split with no record final KafkaPartitionSplit emptySplit = new KafkaPartitionSplit( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java index 4ca5c9cb6..db7647242 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java @@ -36,11 +36,7 @@ public void testSerializer() throws IOException { Long normalOffset = 1L; TopicPartition topicPartition = new TopicPartition(topic, 1); List stoppingOffsets = - Lists.newArrayList( - KafkaPartitionSplit.COMMITTED_OFFSET, - KafkaPartitionSplit.LATEST_OFFSET, - offsetZero, - normalOffset); + Lists.newArrayList(KafkaPartitionSplit.COMMITTED_OFFSET, offsetZero, normalOffset); KafkaPartitionSplitSerializer splitSerializer = new KafkaPartitionSplitSerializer(); for (Long stoppingOffset : stoppingOffsets) { KafkaPartitionSplit kafkaPartitionSplit = diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java index 2c82fc152..1246d53a3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java @@ -477,13 +477,17 @@ public void testBoundedLatestOffset() { OffsetsInitializer offsetsInitializer = KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); TopicPartition partition = new TopicPartition(TOPIC, 0); + long endOffsets = 123L; Map partitionOffsets = offsetsInitializer.getPartitionOffsets( Collections.singletonList(partition), - MockPartitionOffsetsRetriever.noInteractions()); + MockPartitionOffsetsRetriever.latest( + (tps) -> + Collections.singletonMap( + partition, endOffsets))); assertThat(partitionOffsets) .containsOnlyKeys(partition) - .containsEntry(partition, KafkaPartitionSplit.LATEST_OFFSET); + .containsEntry(partition, endOffsets); }); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java index 2674183fb..409acd977 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java @@ -18,12 +18,19 @@ package org.apache.flink.streaming.connectors.kafka.table; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.core.testutils.FlinkAssertions; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.data.RowData; import org.apache.flink.table.utils.EncodingUtils; @@ -944,6 +951,138 @@ public void testPerPartitionWatermarkWithIdleSource() throws Exception { deleteTestTopic(topic); } + @Test + public void testLatestOffsetStrategyResume() throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "latest_offset_resume_topic_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic, 6, 1); + env.setParallelism(1); + + // ---------- Produce data into Kafka's partition 0-6 ------------------- + + String groupId = getStandardProps().getProperty("group.id"); + String bootstraps = getBootstrapServers(); + + final String createTable = + String.format( + "CREATE TABLE kafka (\n" + + " `partition_id` INT,\n" + + " `value` INT\n" + + ") WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'properties.group.id' = '%s',\n" + + " 'scan.startup.mode' = 'latest-offset',\n" + + " 'sink.partitioner' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + topic, bootstraps, groupId, TestPartitioner.class.getName(), format); + + tEnv.executeSql(createTable); + + String initialValues = + "INSERT INTO kafka VALUES (0, 0), (1, 0), (2, 0), (3, 0), (4, 0), (5, 0)"; + tEnv.executeSql(initialValues).await(); + + // ---------- Consume stream from Kafka ------------------- + + String createSink = + "CREATE TABLE MySink(\n" + + " `id` INT,\n" + + " `value` INT\n" + + ") WITH (\n" + + " 'connector' = 'values'\n" + + ")"; + tEnv.executeSql(createSink); + + String executeInsert = "INSERT INTO MySink SELECT `partition_id`, `value` FROM kafka"; + TableResult tableResult = tEnv.executeSql(executeInsert); + + // ---------- Produce data into Kafka's partition 0-2 ------------------- + + String moreValues = "INSERT INTO kafka VALUES (0, 1), (1, 1), (2, 1)"; + tEnv.executeSql(moreValues).await(); + + final List expected = Arrays.asList("+I[0, 1]", "+I[1, 1]", "+I[2, 1]"); + KafkaTableTestUtils.waitingExpectedResults("MySink", expected, Duration.ofSeconds(5)); + + // ---------- Stop the consume job with savepoint ------------------- + + String savepointBasePath = getTempDirPath(topic + "-savepoint"); + assert tableResult.getJobClient().isPresent(); + JobClient client = tableResult.getJobClient().get(); + String savepointPath = + client.stopWithSavepoint(false, savepointBasePath, SavepointFormatType.DEFAULT) + .get(); + + // ---------- Produce data into Kafka's partition 0-5 ------------------- + + String produceValuesBeforeResume = + "INSERT INTO kafka VALUES (0, 2), (1, 2), (2, 2), (3, 1), (4, 1), (5, 1)"; + tEnv.executeSql(produceValuesBeforeResume).await(); + + // ---------- Resume the consume job from savepoint ------------------- + + Configuration configuration = new Configuration(); + configuration.set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + + tEnv.executeSql(createTable); + tEnv.executeSql(createSink); + tableResult = tEnv.executeSql(executeInsert); + + final List afterResumeExpected = + Arrays.asList( + "+I[0, 1]", + "+I[1, 1]", + "+I[2, 1]", + "+I[0, 2]", + "+I[1, 2]", + "+I[2, 2]", + "+I[3, 1]", + "+I[4, 1]", + "+I[5, 1]"); + KafkaTableTestUtils.waitingExpectedResults( + "MySink", afterResumeExpected, Duration.ofSeconds(5)); + + // ---------- Produce data into Kafka's partition 0-5 ------------------- + + String produceValuesAfterResume = + "INSERT INTO kafka VALUES (0, 3), (1, 3), (2, 3), (3, 2), (4, 2), (5, 2)"; + this.tEnv.executeSql(produceValuesAfterResume).await(); + + final List afterProduceExpected = + Arrays.asList( + "+I[0, 1]", + "+I[1, 1]", + "+I[2, 1]", + "+I[0, 2]", + "+I[1, 2]", + "+I[2, 2]", + "+I[3, 1]", + "+I[4, 1]", + "+I[5, 1]", + "+I[0, 3]", + "+I[1, 3]", + "+I[2, 3]", + "+I[3, 2]", + "+I[4, 2]", + "+I[5, 2]"); + KafkaTableTestUtils.waitingExpectedResults( + "MySink", afterProduceExpected, Duration.ofSeconds(5)); + + // ------------- cleanup ------------------- + + tableResult.getJobClient().ifPresent(JobClient::cancel); + deleteTestTopic(topic); + } + @Test public void testStartFromGroupOffsetsLatest() throws Exception { testStartFromGroupOffsets("latest"); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java index 41d9e7eb0..15c740d21 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java @@ -465,13 +465,17 @@ public void testBoundedLatestOffset() { OffsetsInitializer offsetsInitializer = KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); TopicPartition partition = new TopicPartition(SOURCE_TOPIC, 0); + long endOffsets = 123L; Map partitionOffsets = offsetsInitializer.getPartitionOffsets( Collections.singletonList(partition), - MockPartitionOffsetsRetriever.noInteractions()); + MockPartitionOffsetsRetriever.latest( + (tps) -> + Collections.singletonMap( + partition, endOffsets))); assertThat(partitionOffsets) .containsOnlyKeys(partition) - .containsEntry(partition, KafkaPartitionSplit.LATEST_OFFSET); + .containsEntry(partition, endOffsets); }); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java index 175bddd6a..9947bc5b6 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java @@ -68,6 +68,17 @@ public static MockPartitionOffsetsRetriever timestampAndEnd( UNSUPPORTED_RETRIEVAL, endOffsets, UNSUPPORTED_RETRIEVAL, retriever); } + public static MockPartitionOffsetsRetriever latest(OffsetsRetriever endOffsets) { + return new MockPartitionOffsetsRetriever( + UNSUPPORTED_RETRIEVAL, + endOffsets, + UNSUPPORTED_RETRIEVAL, + partitions -> { + throw new UnsupportedOperationException( + "The method was not supposed to be called"); + }); + } + private MockPartitionOffsetsRetriever( OffsetsRetriever committedOffsets, OffsetsRetriever endOffsets, From 54e3b70deb349538edba1ec2b051ed9d9f79b563 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 23 Oct 2023 10:56:53 -0700 Subject: [PATCH 242/322] [FLINK-28303] Allow LATEST_OFFSET marker when restoring from old checkpoints --- .../kafka/source/split/KafkaPartitionSplit.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java index ef1b8b88f..7c04600d1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java @@ -44,9 +44,9 @@ public class KafkaPartitionSplit implements SourceSplit { // Valid special starting offsets public static final Set VALID_STARTING_OFFSET_MARKERS = - new HashSet<>(Arrays.asList(EARLIEST_OFFSET, COMMITTED_OFFSET)); + new HashSet<>(Arrays.asList(EARLIEST_OFFSET, LATEST_OFFSET, COMMITTED_OFFSET)); public static final Set VALID_STOPPING_OFFSET_MARKERS = - new HashSet<>(Arrays.asList(COMMITTED_OFFSET, NO_STOPPING_OFFSET)); + new HashSet<>(Arrays.asList(LATEST_OFFSET, COMMITTED_OFFSET, NO_STOPPING_OFFSET)); private final TopicPartition tp; private final long startingOffset; @@ -133,8 +133,8 @@ private static void verifyInitialOffset( String.format( "Invalid starting offset %d is specified for partition %s. " + "It should either be non-negative or be one of the " - + "[%d(earliest), %d(committed)].", - startingOffset, tp, EARLIEST_OFFSET, COMMITTED_OFFSET)); + + "[%d(earliest), %d(latest), %d(committed)].", + startingOffset, tp, LATEST_OFFSET, EARLIEST_OFFSET, COMMITTED_OFFSET)); } if (stoppingOffset < 0 && !VALID_STOPPING_OFFSET_MARKERS.contains(stoppingOffset)) { @@ -142,8 +142,12 @@ private static void verifyInitialOffset( String.format( "Illegal stopping offset %d is specified for partition %s. " + "It should either be non-negative or be one of the " - + "[%d(committed), %d(Long.MIN_VALUE, no_stopping_offset)].", - stoppingOffset, tp, COMMITTED_OFFSET, NO_STOPPING_OFFSET)); + + "[%d(latest), %d(committed), %d(Long.MIN_VALUE, no_stopping_offset)].", + stoppingOffset, + tp, + LATEST_OFFSET, + COMMITTED_OFFSET, + NO_STOPPING_OFFSET)); } } } From 40da49e73b956b1a6fa0c1351b695b520c91919e Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Wed, 25 Oct 2023 13:45:32 +0200 Subject: [PATCH 243/322] [hotfix] Test connector against Flink 1.18.0 and Flink 1.19-SNAPSHOT --- .github/workflows/push_pr.yml | 2 +- .github/workflows/weekly.yml | 8 +++++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index be8e32702..f40a0320c 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -25,7 +25,7 @@ jobs: compile_and_test: strategy: matrix: - flink: [ 1.17.1, 1.18-SNAPSHOT ] + flink: [ 1.17.1, 1.18.0, 1.19-SNAPSHOT ] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index a9de998c9..890e80659 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -29,11 +29,17 @@ jobs: flink_branches: [{ flink: 1.17.1, branch: v3.0 + }, { + flink: 1.18.0, + branch: v3.0 }, { flink: 1.17.1, branch: main }, { - flink: 1.18-SNAPSHOT, + flink: 1.18.0, + branch: main + }, { + flink: 1.19-SNAPSHOT, branch: main }] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils From b732ce307364bd8fc9d2183677a0f6a8de3d6410 Mon Sep 17 00:00:00 2001 From: Martijn Visser Date: Wed, 25 Oct 2023 14:33:19 +0200 Subject: [PATCH 244/322] [hotfix] Partially revert 40da49e73b956b1a6fa0c1351b695b520c91919e and don't test against Flink 1.19-SNAPSHOT yet --- .github/workflows/push_pr.yml | 2 +- .github/workflows/weekly.yml | 3 --- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index f40a0320c..ddc50ab8d 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -25,7 +25,7 @@ jobs: compile_and_test: strategy: matrix: - flink: [ 1.17.1, 1.18.0, 1.19-SNAPSHOT ] + flink: [ 1.17.1, 1.18.0 ] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index 890e80659..b363b7fb9 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -38,9 +38,6 @@ jobs: }, { flink: 1.18.0, branch: main - }, { - flink: 1.19-SNAPSHOT, - branch: main }] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: From 979791c4c71e944c16c51419cf9a84aa1f8fea4c Mon Sep 17 00:00:00 2001 From: mas-chen Date: Mon, 30 Oct 2023 12:29:27 -0700 Subject: [PATCH 245/322] [hotfix] refer to sql_connector_download_table shortcode in the docs to adhere to new connector versioning format --- docs/content/docs/connectors/table/kafka.md | 2 +- docs/content/docs/connectors/table/upsert-kafka.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md index 18578569a..0256301ba 100644 --- a/docs/content/docs/connectors/table/kafka.md +++ b/docs/content/docs/connectors/table/kafka.md @@ -35,7 +35,7 @@ The Kafka connector allows for reading data from and writing data into Kafka top Dependencies ------------ -{{< sql_download_table "kafka" >}} +{{< sql_connector_download_table "kafka" >}} The Kafka connector is not part of the binary distribution. See how to link with it for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). diff --git a/docs/content/docs/connectors/table/upsert-kafka.md b/docs/content/docs/connectors/table/upsert-kafka.md index c3537a764..612376402 100644 --- a/docs/content/docs/connectors/table/upsert-kafka.md +++ b/docs/content/docs/connectors/table/upsert-kafka.md @@ -47,7 +47,7 @@ key will fall into the same partition. Dependencies ------------ -{{< sql_download_table "upsert-kafka" >}} +{{< sql_connector_download_table "upsert-kafka" >}} The Upsert Kafka connector is not part of the binary distribution. See how to link with it for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). From c38a0406104646a7ea8199bb64244310e344ce2b Mon Sep 17 00:00:00 2001 From: pvary Date: Mon, 11 Dec 2023 09:40:50 +0100 Subject: [PATCH 246/322] [FLINK-33559] Externalize Kafka Python connector code --- .github/workflows/push_pr.yml | 8 + .gitignore | 18 +- flink-python/MANIFEST.in | 21 + flink-python/README.txt | 14 + flink-python/dev/integration_test.sh | 54 + flink-python/pom.xml | 222 ++++ .../pyflink/datastream/connectors/kafka.py | 1163 +++++++++++++++++ .../datastream/connectors/tests/test_kafka.py | 669 ++++++++++ .../pyflink/pyflink_gateway_server.py | 288 ++++ flink-python/setup.py | 158 +++ flink-python/tox.ini | 51 + pom.xml | 1 + 12 files changed, 2666 insertions(+), 1 deletion(-) create mode 100644 flink-python/MANIFEST.in create mode 100644 flink-python/README.txt create mode 100755 flink-python/dev/integration_test.sh create mode 100644 flink-python/pom.xml create mode 100644 flink-python/pyflink/datastream/connectors/kafka.py create mode 100644 flink-python/pyflink/datastream/connectors/tests/test_kafka.py create mode 100644 flink-python/pyflink/pyflink_gateway_server.py create mode 100644 flink-python/setup.py create mode 100644 flink-python/tox.ini diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index ddc50ab8d..8f53a5bde 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -29,3 +29,11 @@ jobs: uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} + + python_test: + strategy: + matrix: + flink: [ 1.17.1, 1.18.0 ] + uses: apache/flink-connector-shared-utils/.github/workflows/python_ci.yml@ci_utils + with: + flink_version: ${{ matrix.flink }} diff --git a/.gitignore b/.gitignore index 5f0068cda..901fd674f 100644 --- a/.gitignore +++ b/.gitignore @@ -35,4 +35,20 @@ out/ tools/flink tools/flink-* tools/releasing/release -tools/japicmp-output \ No newline at end of file +tools/japicmp-output + +# Generated file, do not store in git +flink-python/pyflink/datastream/connectors/kafka_connector_version.py +flink-python/apache_flink_connectors_kafka.egg-info/ +flink-python/.tox/ +flink-python/build +flink-python/dist +flink-python/dev/download +flink-python/dev/.conda/ +flink-python/dev/log/ +flink-python/dev/.stage.txt +flink-python/dev/install_command.sh +flink-python/dev/lint-python.sh +flink-python/dev/build-wheels.sh +flink-python/dev/glibc_version_fix.h +flink-python/dev/dev-requirements.txt diff --git a/flink-python/MANIFEST.in b/flink-python/MANIFEST.in new file mode 100644 index 000000000..3578d2dfc --- /dev/null +++ b/flink-python/MANIFEST.in @@ -0,0 +1,21 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +graft pyflink +global-exclude *.py[cod] __pycache__ .DS_Store + diff --git a/flink-python/README.txt b/flink-python/README.txt new file mode 100644 index 000000000..a12c13e5e --- /dev/null +++ b/flink-python/README.txt @@ -0,0 +1,14 @@ +This is official Apache Flink Kafka Python connector. + +For the latest information about Flink connector, please visit our website at: + + https://flink.apache.org + +and our GitHub Account for Kafka connector + + https://github.com/apache/flink-connector-kafka + +If you have any questions, ask on our Mailing lists: + + user@flink.apache.org + dev@flink.apache.org diff --git a/flink-python/dev/integration_test.sh b/flink-python/dev/integration_test.sh new file mode 100755 index 000000000..19816725a --- /dev/null +++ b/flink-python/dev/integration_test.sh @@ -0,0 +1,54 @@ +#!/usr/bin/env bash +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +function test_module() { + module="$FLINK_PYTHON_DIR/pyflink/$1" + echo "test module $module" + pytest --durations=20 ${module} $2 + if [[ $? -ne 0 ]]; then + echo "test module $module failed" + exit 1 + fi +} + +function test_all_modules() { + # test datastream module + test_module "datastream" +} + +# CURRENT_DIR is "flink-connector-kafka/flink-python/dev/" +CURRENT_DIR="$(cd "$( dirname "$0" )" && pwd)" + +# FLINK_PYTHON_DIR is "flink-connector-kafka/flink-python" +FLINK_PYTHON_DIR=$(dirname "$CURRENT_DIR") + +# FLINK_SOURCE_DIR is "flink-connector-kafka" +FLINK_SOURCE_DIR=$(dirname "$FLINK_PYTHON_DIR") + +# set the FLINK_TEST_LIB_DIR to "flink-connector-kafka" +export FLINK_TEST_LIBS="${FLINK_SOURCE_DIR}/flink-python/target/test-dependencies/*" + +# Temporarily update the installed 'pyflink_gateway_server.py' files with the new one +# Needed only until Flink 1.19 release +echo "Checking ${FLINK_SOURCE_DIR} for 'pyflink_gateway_server.py'" +find "${FLINK_SOURCE_DIR}/flink-python" -name pyflink_gateway_server.py +find "${FLINK_SOURCE_DIR}/flink-python/.tox" -name pyflink_gateway_server.py -exec cp "${FLINK_SOURCE_DIR}/flink-python/pyflink/pyflink_gateway_server.py" {} \; + +# python test +test_all_modules diff --git a/flink-python/pom.xml b/flink-python/pom.xml new file mode 100644 index 000000000..cb3f70caa --- /dev/null +++ b/flink-python/pom.xml @@ -0,0 +1,222 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connector-kafka-parent + 3.1-SNAPSHOT + + + flink-connector-kafka-python + Flink : Connectors : SQL : Kafka : Python + + pom + + + + + + + org.apache.flink + flink-sql-connector-kafka + ${project.version} + + + org.apache.flink + flink-runtime + ${flink.version} + + + org.apache.flink + flink-streaming-java + ${flink.version} + + + org.apache.flink + flink-connector-test-utils + ${flink.version} + + + org.apache.flink + flink-sql-avro + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + clean + clean + + run + + + + + + + + + + + + + + + + + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + package + + copy + + + + + org.apache.flink + flink-sql-connector-kafka + + + org.apache.flink + flink-runtime + + tests + + + org.apache.flink + flink-streaming-java + + tests + + + org.apache.flink + flink-connector-test-utils + + + org.apache.flink + flink-test-utils + + + org.apache.flink + flink-sql-avro + + + ${project.build.directory}/test-dependencies + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + package + + copy-dependencies + + + junit + ${project.build.directory}/test-dependencies + + + + + + + org.codehaus.mojo + wagon-maven-plugin + 2.0.2 + + + download-install + validate + + download-single + + + https://raw.githubusercontent.com/apache/flink-connector-shared-utils/ci_utils/python/install_command.sh + ${project.basedir}/dev + ${python.infra.download.skip} + + + + download-lint + validate + + download-single + + + https://raw.githubusercontent.com/apache/flink-connector-shared-utils/ci_utils/python/lint-python.sh + ${project.basedir}/dev + ${python.infra.download.skip} + + + + download-build-wheels + validate + + download-single + + + https://raw.githubusercontent.com/apache/flink-connector-shared-utils/ci_utils/python/build-wheels.sh + ${project.basedir}/dev + ${python.infra.download.skip} + + + + download-build-version-header + validate + + download-single + + + https://raw.githubusercontent.com/apache/flink-connector-shared-utils/ci_utils/python/glibc_version_fix.h + ${project.basedir}/dev + ${python.infra.download.skip} + + + + + + + diff --git a/flink-python/pyflink/datastream/connectors/kafka.py b/flink-python/pyflink/datastream/connectors/kafka.py new file mode 100644 index 000000000..0e0a12893 --- /dev/null +++ b/flink-python/pyflink/datastream/connectors/kafka.py @@ -0,0 +1,1163 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import warnings +from abc import ABC, abstractmethod +from enum import Enum +from typing import Dict, Union, List, Set, Callable, Any, Optional + +from py4j.java_gateway import JavaObject, get_java_class + +from pyflink.common import DeserializationSchema, TypeInformation, typeinfo, SerializationSchema, \ + Types, Row +from pyflink.datastream.connectors import Source, Sink +from pyflink.datastream.connectors.base import DeliveryGuarantee, SupportsPreprocessing, \ + StreamTransformer +from pyflink.datastream.functions import SinkFunction, SourceFunction +from pyflink.java_gateway import get_gateway +from pyflink.util.java_utils import to_jarray, get_field, get_field_value + +__all__ = [ + 'FlinkKafkaConsumer', + 'FlinkKafkaProducer', + 'KafkaSource', + 'KafkaSourceBuilder', + 'KafkaSink', + 'KafkaSinkBuilder', + 'Semantic', + 'KafkaTopicPartition', + 'KafkaOffsetsInitializer', + 'KafkaOffsetResetStrategy', + 'KafkaRecordSerializationSchema', + 'KafkaRecordSerializationSchemaBuilder', + 'KafkaTopicSelector' +] + + +# ---- FlinkKafkaConsumer ---- + +class FlinkKafkaConsumerBase(SourceFunction, ABC): + """ + Base class of all Flink Kafka Consumer data sources. This implements the common behavior across + all kafka versions. + + The Kafka version specific behavior is defined mainly in the specific subclasses. + """ + + def __init__(self, j_flink_kafka_consumer): + super(FlinkKafkaConsumerBase, self).__init__(source_func=j_flink_kafka_consumer) + + def set_commit_offsets_on_checkpoints(self, + commit_on_checkpoints: bool) -> 'FlinkKafkaConsumerBase': + """ + Specifies whether or not the consumer should commit offsets back to kafka on checkpoints. + This setting will only have effect if checkpointing is enabled for the job. If checkpointing + isn't enabled, only the "auto.commit.enable" (for 0.8) / "enable.auto.commit" (for 0.9+) + property settings will be used. + """ + self._j_function = self._j_function \ + .setCommitOffsetsOnCheckpoints(commit_on_checkpoints) + return self + + def set_start_from_earliest(self) -> 'FlinkKafkaConsumerBase': + """ + Specifies the consumer to start reading from the earliest offset for all partitions. This + lets the consumer ignore any committed group offsets in Zookeeper/ Kafka brokers. + + This method does not affect where partitions are read from when the consumer is restored + from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + savepoint, only the offsets in the restored state will be used. + """ + self._j_function = self._j_function.setStartFromEarliest() + return self + + def set_start_from_latest(self) -> 'FlinkKafkaConsumerBase': + """ + Specifies the consuer to start reading from the latest offset for all partitions. This lets + the consumer ignore any committed group offsets in Zookeeper / Kafka brokers. + + This method does not affect where partitions are read from when the consumer is restored + from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + savepoint, only the offsets in the restored state will be used. + """ + self._j_function = self._j_function.setStartFromLatest() + return self + + def set_start_from_timestamp(self, startup_offsets_timestamp: int) -> 'FlinkKafkaConsumerBase': + """ + Specifies the consumer to start reading partitions from a specified timestamp. The specified + timestamp must be before the current timestamp. This lets the consumer ignore any committed + group offsets in Zookeeper / Kafka brokers. + + The consumer will look up the earliest offset whose timestamp is greater than or equal to + the specific timestamp from Kafka. If there's no such offset, the consumer will use the + latest offset to read data from Kafka. + + This method does not affect where partitions are read from when the consumer is restored + from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + savepoint, only the offsets in the restored state will be used. + + :param startup_offsets_timestamp: timestamp for the startup offsets, as milliseconds for + epoch. + """ + self._j_function = self._j_function.setStartFromTimestamp( + startup_offsets_timestamp) + return self + + def set_start_from_group_offsets(self) -> 'FlinkKafkaConsumerBase': + """ + Specifies the consumer to start reading from any committed group offsets found in Zookeeper/ + Kafka brokers. The 'group.id' property must be set in the configuration properties. If no + offset can be found for a partition, the behaviour in 'auto.offset.reset' set in the + configuration properties will be used for the partition. + + This method does not affect where partitions are read from when the consumer is restored + from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + savepoint, only the offsets in the restored state will be used. + """ + self._j_function = self._j_function.setStartFromGroupOffsets() + return self + + def disable_filter_restored_partitions_with_subscribed_topics(self) -> 'FlinkKafkaConsumerBase': + """ + By default, when restoring from a checkpoint / savepoint, the consumer always ignores + restored partitions that are no longer associated with the current specified topics or topic + pattern to subscribe to. + + This method does not affect where partitions are read from when the consumer is restored + from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + savepoint, only the offsets in the restored state will be used. + """ + self._j_function = self._j_function \ + .disableFilterRestoredPartitionsWithSubscribedTopics() + return self + + def get_produced_type(self) -> TypeInformation: + return typeinfo._from_java_type(self._j_function.getProducedType()) + + +def _get_kafka_consumer(topics, properties, deserialization_schema, j_consumer_clz): + if not isinstance(topics, list): + topics = [topics] + gateway = get_gateway() + j_properties = gateway.jvm.java.util.Properties() + for key, value in properties.items(): + j_properties.setProperty(key, value) + + j_flink_kafka_consumer = j_consumer_clz(topics, + deserialization_schema._j_deserialization_schema, + j_properties) + return j_flink_kafka_consumer + + +class FlinkKafkaConsumer(FlinkKafkaConsumerBase): + """ + The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from + Apache Kafka. The consumer can run in multiple parallel instances, each of which will + pull data from one or more Kafka partitions. + + The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost + during a failure, and that the computation processes elements 'exactly once. (These guarantees + naturally assume that Kafka itself does not lose any data.) + + Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. + The offsets committed to Kafka / Zookeeper are only to bring the outside view of progress in + sync with Flink's view of the progress. That way, monitoring and other jobs can get a view of + how far the Flink Kafka consumer has consumed a topic. + + Please refer to Kafka's documentation for the available configuration properties: + http://kafka.apache.org/documentation.html#newconsumerconfigs + """ + + def __init__(self, topics: Union[str, List[str]], deserialization_schema: DeserializationSchema, + properties: Dict): + """ + Creates a new Kafka streaming source consumer for Kafka 0.10.x. + + This constructor allows passing multiple topics to the consumer. + + :param topics: The Kafka topics to read from. + :param deserialization_schema: The de-/serializer used to convert between Kafka's byte + messages and Flink's objects. + :param properties: The properties that are used to configure both the fetcher and the offset + handler. + """ + + warnings.warn("Deprecated in 1.16. Use KafkaSource instead.", DeprecationWarning) + JFlinkKafkaConsumer = get_gateway().jvm \ + .org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer + j_flink_kafka_consumer = _get_kafka_consumer(topics, properties, deserialization_schema, + JFlinkKafkaConsumer) + super(FlinkKafkaConsumer, self).__init__(j_flink_kafka_consumer=j_flink_kafka_consumer) + + +# ---- FlinkKafkaProducer ---- + + +class Semantic(Enum): + """ + Semantics that can be chosen. + + :data: `EXACTLY_ONCE`: + + The Flink producer will write all messages in a Kafka transaction that will be committed to + the Kafka on a checkpoint. In this mode FlinkKafkaProducer sets up a pool of + FlinkKafkaProducer. Between each checkpoint there is created new Kafka transaction, which is + being committed on FlinkKafkaProducer.notifyCheckpointComplete(long). If checkpoint + complete notifications are running late, FlinkKafkaProducer can run out of + FlinkKafkaProducers in the pool. In that case any subsequent FlinkKafkaProducer.snapshot- + State() requests will fail and the FlinkKafkaProducer will keep using the + FlinkKafkaProducer from previous checkpoint. To decrease chances of failing checkpoints + there are four options: + + 1. decrease number of max concurrent checkpoints + 2. make checkpoints mre reliable (so that they complete faster) + 3. increase delay between checkpoints + 4. increase size of FlinkKafkaProducers pool + + :data: `AT_LEAST_ONCE`: + + The Flink producer will wait for all outstanding messages in the Kafka buffers to be + acknowledged by the Kafka producer on a checkpoint. + + :data: `NONE`: + + Means that nothing will be guaranteed. Messages can be lost and/or duplicated in case of + failure. + + """ + + EXACTLY_ONCE = 0, + AT_LEAST_ONCE = 1, + NONE = 2 + + def _to_j_semantic(self): + JSemantic = get_gateway().jvm \ + .org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.Semantic + return getattr(JSemantic, self.name) + + +class FlinkKafkaProducerBase(SinkFunction, ABC): + """ + Flink Sink to produce data into a Kafka topic. + + Please note that this producer provides at-least-once reliability guarantees when checkpoints + are enabled and set_flush_on_checkpoint(True) is set. Otherwise, the producer doesn;t provid any + reliability guarantees. + """ + + def __init__(self, j_flink_kafka_producer): + super(FlinkKafkaProducerBase, self).__init__(sink_func=j_flink_kafka_producer) + + def set_log_failures_only(self, log_failures_only: bool) -> 'FlinkKafkaProducerBase': + """ + Defines whether the producer should fail on errors, or only log them. If this is set to + true, then exceptions will be only logged, if set to false, exceptions will be eventually + thrown and cause the streaming program to fail (and enter recovery). + + :param log_failures_only: The flag to indicate logging-only on exceptions. + """ + self._j_function.setLogFailuresOnly(log_failures_only) + return self + + def set_flush_on_checkpoint(self, flush_on_checkpoint: bool) -> 'FlinkKafkaProducerBase': + """ + If set to true, the Flink producer will wait for all outstanding messages in the Kafka + buffers to be acknowledged by the Kafka producer on a checkpoint. + + This way, the producer can guarantee that messages in the Kafka buffers are part of the + checkpoint. + + :param flush_on_checkpoint: Flag indicating the flush mode (true = flush on checkpoint) + """ + self._j_function.setFlushOnCheckpoint(flush_on_checkpoint) + return self + + def set_write_timestamp_to_kafka(self, + write_timestamp_to_kafka: bool) -> 'FlinkKafkaProducerBase': + """ + If set to true, Flink will write the (event time) timestamp attached to each record into + Kafka. Timestamps must be positive for Kafka to accept them. + + :param write_timestamp_to_kafka: Flag indicating if Flink's internal timestamps are written + to Kafka. + """ + self._j_function.setWriteTimestampToKafka(write_timestamp_to_kafka) + return self + + +class FlinkKafkaProducer(FlinkKafkaProducerBase): + """ + Flink Sink to produce data into a Kafka topic. By + default producer will use AT_LEAST_ONCE semantic. Before using EXACTLY_ONCE please refer to + Flink's Kafka connector documentation. + """ + + def __init__(self, topic: str, serialization_schema: SerializationSchema, + producer_config: Dict, kafka_producer_pool_size: int = 5, + semantic=Semantic.AT_LEAST_ONCE): + """ + Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to the topic. + + Using this constructor, the default FlinkFixedPartitioner will be used as the partitioner. + This default partitioner maps each sink subtask to a single Kafka partition (i.e. all + records received by a sink subtask will end up in the same Kafka partition). + + :param topic: ID of the Kafka topic. + :param serialization_schema: User defined key-less serialization schema. + :param producer_config: Properties with the producer configuration. + """ + gateway = get_gateway() + j_properties = gateway.jvm.java.util.Properties() + for key, value in producer_config.items(): + j_properties.setProperty(key, value) + + JFlinkKafkaProducer = gateway.jvm \ + .org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer + + j_flink_kafka_producer = JFlinkKafkaProducer( + topic, serialization_schema._j_serialization_schema, j_properties, None, + semantic._to_j_semantic(), kafka_producer_pool_size) + super(FlinkKafkaProducer, self).__init__(j_flink_kafka_producer=j_flink_kafka_producer) + + def ignore_failures_after_transaction_timeout(self) -> 'FlinkKafkaProducer': + """ + Disables the propagation of exceptions thrown when committing presumably timed out Kafka + transactions during recovery of the job. If a Kafka transaction is timed out, a commit will + never be successful. Hence, use this feature to avoid recovery loops of the Job. Exceptions + will still be logged to inform the user that data loss might have occurred. + + Note that we use the System.currentTimeMillis() to track the age of a transaction. Moreover, + only exceptions thrown during the recovery are caught, i.e., the producer will attempt at + least one commit of the transaction before giving up. + + :return: This FlinkKafkaProducer. + """ + self._j_function.ignoreFailuresAfterTransactionTimeout() + return self + + +# ---- KafkaSource ---- + + +class KafkaSource(Source): + """ + The Source implementation of Kafka. Please use a :class:`KafkaSourceBuilder` to construct a + :class:`KafkaSource`. The following example shows how to create a KafkaSource emitting records + of String type. + + :: + + >>> source = KafkaSource \\ + ... .builder() \\ + ... .set_bootstrap_servers('MY_BOOTSTRAP_SERVERS') \\ + ... .set_group_id('MY_GROUP') \\ + ... .set_topics('TOPIC1', 'TOPIC2') \\ + ... .set_value_only_deserializer(SimpleStringSchema()) \\ + ... .set_starting_offsets(KafkaOffsetsInitializer.earliest()) \\ + ... .build() + + .. versionadded:: 1.16.0 + """ + + def __init__(self, j_kafka_source: JavaObject): + super().__init__(j_kafka_source) + + @staticmethod + def builder() -> 'KafkaSourceBuilder': + """ + Get a kafkaSourceBuilder to build a :class:`KafkaSource`. + + :return: a Kafka source builder. + """ + return KafkaSourceBuilder() + + +class KafkaSourceBuilder(object): + """ + The builder class for :class:`KafkaSource` to make it easier for the users to construct a + :class:`KafkaSource`. + + The following example shows the minimum setup to create a KafkaSource that reads the String + values from a Kafka topic. + + :: + + >>> source = KafkaSource.builder() \\ + ... .set_bootstrap_servers('MY_BOOTSTRAP_SERVERS') \\ + ... .set_topics('TOPIC1', 'TOPIC2') \\ + ... .set_value_only_deserializer(SimpleStringSchema()) \\ + ... .build() + + The bootstrap servers, topics/partitions to consume, and the record deserializer are required + fields that must be set. + + To specify the starting offsets of the KafkaSource, one can call :meth:`set_starting_offsets`. + + By default, the KafkaSource runs in an CONTINUOUS_UNBOUNDED mode and never stops until the Flink + job is canceled or fails. To let the KafkaSource run in CONTINUOUS_UNBOUNDED but stops at some + given offsets, one can call :meth:`set_stopping_offsets`. For example the following KafkaSource + stops after it consumes up to the latest partition offsets at the point when the Flink started. + + :: + + >>> source = KafkaSource.builder() \\ + ... .set_bootstrap_servers('MY_BOOTSTRAP_SERVERS') \\ + ... .set_topics('TOPIC1', 'TOPIC2') \\ + ... .set_value_only_deserializer(SimpleStringSchema()) \\ + ... .set_unbounded(KafkaOffsetsInitializer.latest()) \\ + ... .build() + + .. versionadded:: 1.16.0 + """ + + def __init__(self): + self._j_builder = get_gateway().jvm.org.apache.flink.connector.kafka.source \ + .KafkaSource.builder() + + def build(self) -> 'KafkaSource': + return KafkaSource(self._j_builder.build()) + + def set_bootstrap_servers(self, bootstrap_servers: str) -> 'KafkaSourceBuilder': + """ + Sets the bootstrap servers for the KafkaConsumer of the KafkaSource. + + :param bootstrap_servers: the bootstrap servers of the Kafka cluster. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setBootstrapServers(bootstrap_servers) + return self + + def set_group_id(self, group_id: str) -> 'KafkaSourceBuilder': + """ + Sets the consumer group id of the KafkaSource. + + :param group_id: the group id of the KafkaSource. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setGroupId(group_id) + return self + + def set_topics(self, *topics: str) -> 'KafkaSourceBuilder': + """ + Set a list of topics the KafkaSource should consume from. All the topics in the list should + have existed in the Kafka cluster. Otherwise, an exception will be thrown. To allow some + topics to be created lazily, please use :meth:`set_topic_pattern` instead. + + :param topics: the list of topics to consume from. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setTopics(to_jarray(get_gateway().jvm.java.lang.String, topics)) + return self + + def set_topic_pattern(self, topic_pattern: str) -> 'KafkaSourceBuilder': + """ + Set a topic pattern to consume from use the java Pattern. For grammar, check out + `JavaDoc `_ . + + :param topic_pattern: the pattern of the topic name to consume from. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setTopicPattern(get_gateway().jvm.java.util.regex + .Pattern.compile(topic_pattern)) + return self + + def set_partitions(self, partitions: Set['KafkaTopicPartition']) -> 'KafkaSourceBuilder': + """ + Set a set of partitions to consume from. + + Example: + :: + + >>> KafkaSource.builder().set_partitions({ + ... KafkaTopicPartition('TOPIC1', 0), + ... KafkaTopicPartition('TOPIC1', 1), + ... }) + + :param partitions: the set of partitions to consume from. + :return: this KafkaSourceBuilder. + """ + j_set = get_gateway().jvm.java.util.HashSet() + for tp in partitions: + j_set.add(tp._to_j_topic_partition()) + self._j_builder.setPartitions(j_set) + return self + + def set_starting_offsets(self, starting_offsets_initializer: 'KafkaOffsetsInitializer') \ + -> 'KafkaSourceBuilder': + """ + Specify from which offsets the KafkaSource should start consume from by providing an + :class:`KafkaOffsetsInitializer`. + + The following :class:`KafkaOffsetsInitializer` s are commonly used and provided out of the + box. Currently, customized offset initializer is not supported in PyFlink. + + * :meth:`KafkaOffsetsInitializer.earliest` - starting from the earliest offsets. This is + also the default offset initializer of the KafkaSource for starting offsets. + * :meth:`KafkaOffsetsInitializer.latest` - starting from the latest offsets. + * :meth:`KafkaOffsetsInitializer.committedOffsets` - starting from the committed offsets of + the consumer group. If there is no committed offsets, starting from the offsets + specified by the :class:`KafkaOffsetResetStrategy`. + * :meth:`KafkaOffsetsInitializer.offsets` - starting from the specified offsets for each + partition. + * :meth:`KafkaOffsetsInitializer.timestamp` - starting from the specified timestamp for each + partition. Note that the guarantee here is that all the records in Kafka whose timestamp + is greater than the given starting timestamp will be consumed. However, it is possible + that some consumer records whose timestamp is smaller than the given starting timestamp + are also consumed. + + :param starting_offsets_initializer: the :class:`KafkaOffsetsInitializer` setting the + starting offsets for the Source. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setStartingOffsets(starting_offsets_initializer._j_initializer) + return self + + def set_unbounded(self, stopping_offsets_initializer: 'KafkaOffsetsInitializer') \ + -> 'KafkaSourceBuilder': + """ + By default, the KafkaSource is set to run in CONTINUOUS_UNBOUNDED manner and thus never + stops until the Flink job fails or is canceled. To let the KafkaSource run as a streaming + source but still stops at some point, one can set an :class:`KafkaOffsetsInitializer` + to specify the stopping offsets for each partition. When all the partitions have reached + their stopping offsets, the KafkaSource will then exit. + + This method is different from :meth:`set_bounded` that after setting the stopping offsets + with this method, KafkaSource will still be CONTINUOUS_UNBOUNDED even though it will stop at + the stopping offsets specified by the stopping offset initializer. + + The following :class:`KafkaOffsetsInitializer` s are commonly used and provided out of the + box. Currently, customized offset initializer is not supported in PyFlink. + + * :meth:`KafkaOffsetsInitializer.latest` - starting from the latest offsets. + * :meth:`KafkaOffsetsInitializer.committedOffsets` - starting from the committed offsets of + the consumer group. If there is no committed offsets, starting from the offsets + specified by the :class:`KafkaOffsetResetStrategy`. + * :meth:`KafkaOffsetsInitializer.offsets` - starting from the specified offsets for each + partition. + * :meth:`KafkaOffsetsInitializer.timestamp` - starting from the specified timestamp for each + partition. Note that the guarantee here is that all the records in Kafka whose timestamp + is greater than the given starting timestamp will be consumed. However, it is possible + that some consumer records whose timestamp is smaller than the given starting timestamp + are also consumed. + + :param stopping_offsets_initializer: the :class:`KafkaOffsetsInitializer` to specify the + stopping offsets. + :return: this KafkaSourceBuilder + """ + self._j_builder.setUnbounded(stopping_offsets_initializer._j_initializer) + return self + + def set_bounded(self, stopping_offsets_initializer: 'KafkaOffsetsInitializer') \ + -> 'KafkaSourceBuilder': + """ + By default, the KafkaSource is set to run in CONTINUOUS_UNBOUNDED manner and thus never + stops until the Flink job fails or is canceled. To let the KafkaSource run in BOUNDED manner + and stop at some point, one can set an :class:`KafkaOffsetsInitializer` to specify the + stopping offsets for each partition. When all the partitions have reached their stopping + offsets, the KafkaSource will then exit. + + This method is different from :meth:`set_unbounded` that after setting the stopping offsets + with this method, :meth:`KafkaSource.get_boundedness` will return BOUNDED instead of + CONTINUOUS_UNBOUNDED. + + The following :class:`KafkaOffsetsInitializer` s are commonly used and provided out of the + box. Currently, customized offset initializer is not supported in PyFlink. + + * :meth:`KafkaOffsetsInitializer.latest` - starting from the latest offsets. + * :meth:`KafkaOffsetsInitializer.committedOffsets` - starting from the committed offsets of + the consumer group. If there is no committed offsets, starting from the offsets + specified by the :class:`KafkaOffsetResetStrategy`. + * :meth:`KafkaOffsetsInitializer.offsets` - starting from the specified offsets for each + partition. + * :meth:`KafkaOffsetsInitializer.timestamp` - starting from the specified timestamp for each + partition. Note that the guarantee here is that all the records in Kafka whose timestamp + is greater than the given starting timestamp will be consumed. However, it is possible + that some consumer records whose timestamp is smaller than the given starting timestamp + are also consumed. + + :param stopping_offsets_initializer: the :class:`KafkaOffsetsInitializer` to specify the + stopping offsets. + :return: this KafkaSourceBuilder + """ + self._j_builder.setBounded(stopping_offsets_initializer._j_initializer) + return self + + def set_value_only_deserializer(self, deserialization_schema: DeserializationSchema) \ + -> 'KafkaSourceBuilder': + """ + Sets the :class:`~pyflink.common.serialization.DeserializationSchema` for deserializing the + value of Kafka's ConsumerRecord. The other information (e.g. key) in a ConsumerRecord will + be ignored. + + :param deserialization_schema: the :class:`DeserializationSchema` to use for + deserialization. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setValueOnlyDeserializer(deserialization_schema._j_deserialization_schema) + return self + + def set_client_id_prefix(self, prefix: str) -> 'KafkaSourceBuilder': + """ + Sets the client id prefix of this KafkaSource. + + :param prefix: the client id prefix to use for this KafkaSource. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setClientIdPrefix(prefix) + return self + + def set_property(self, key: str, value: str) -> 'KafkaSourceBuilder': + """ + Set an arbitrary property for the KafkaSource and KafkaConsumer. The valid keys can be found + in ConsumerConfig and KafkaSourceOptions. + + Note that the following keys will be overridden by the builder when the KafkaSource is + created. + + * ``key.deserializer`` is always set to ByteArrayDeserializer. + * ``value.deserializer`` is always set to ByteArrayDeserializer. + * ``auto.offset.reset.strategy`` is overridden by AutoOffsetResetStrategy returned by + :class:`KafkaOffsetsInitializer` for the starting offsets, which is by default + :meth:`KafkaOffsetsInitializer.earliest`. + * ``partition.discovery.interval.ms`` is overridden to -1 when :meth:`set_bounded` has been + invoked. + + :param key: the key of the property. + :param value: the value of the property. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setProperty(key, value) + return self + + def set_properties(self, props: Dict) -> 'KafkaSourceBuilder': + """ + Set arbitrary properties for the KafkaSource and KafkaConsumer. The valid keys can be found + in ConsumerConfig and KafkaSourceOptions. + + Note that the following keys will be overridden by the builder when the KafkaSource is + created. + + * ``key.deserializer`` is always set to ByteArrayDeserializer. + * ``value.deserializer`` is always set to ByteArrayDeserializer. + * ``auto.offset.reset.strategy`` is overridden by AutoOffsetResetStrategy returned by + :class:`KafkaOffsetsInitializer` for the starting offsets, which is by default + :meth:`KafkaOffsetsInitializer.earliest`. + * ``partition.discovery.interval.ms`` is overridden to -1 when :meth:`set_bounded` has been + invoked. + * ``client.id`` is overridden to "client.id.prefix-RANDOM_LONG", or "group.id-RANDOM_LONG" + if the client id prefix is not set. + + :param props: the properties to set for the KafkaSource. + :return: this KafkaSourceBuilder. + """ + gateway = get_gateway() + j_properties = gateway.jvm.java.util.Properties() + for key, value in props.items(): + j_properties.setProperty(key, value) + self._j_builder.setProperties(j_properties) + return self + + +class KafkaTopicPartition(object): + """ + Corresponding to Java ``org.apache.kafka.common.TopicPartition`` class. + + Example: + :: + + >>> topic_partition = KafkaTopicPartition('TOPIC1', 0) + + .. versionadded:: 1.16.0 + """ + + def __init__(self, topic: str, partition: int): + self._topic = topic + self._partition = partition + + def _to_j_topic_partition(self): + jvm = get_gateway().jvm + return jvm.org.apache.flink.kafka.shaded.org.apache.kafka.common.TopicPartition( + self._topic, self._partition) + + def __eq__(self, other): + if not isinstance(other, KafkaTopicPartition): + return False + return self._topic == other._topic and self._partition == other._partition + + def __hash__(self): + return 31 * (31 + self._partition) + hash(self._topic) + + +class KafkaOffsetResetStrategy(Enum): + """ + Corresponding to Java ``org.apache.kafka.client.consumer.OffsetResetStrategy`` class. + + .. versionadded:: 1.16.0 + """ + + LATEST = 0 + EARLIEST = 1 + NONE = 2 + + def _to_j_offset_reset_strategy(self): + JOffsetResetStrategy = get_gateway().jvm.org.apache.flink.kafka.shaded.org.apache.kafka.\ + clients.consumer.OffsetResetStrategy + return getattr(JOffsetResetStrategy, self.name) + + +class KafkaOffsetsInitializer(object): + """ + An interface for users to specify the starting / stopping offset of a KafkaPartitionSplit. + + .. versionadded:: 1.16.0 + """ + + def __init__(self, j_initializer: JavaObject): + self._j_initializer = j_initializer + + @staticmethod + def committed_offsets( + offset_reset_strategy: 'KafkaOffsetResetStrategy' = KafkaOffsetResetStrategy.NONE) -> \ + 'KafkaOffsetsInitializer': + """ + Get an :class:`KafkaOffsetsInitializer` which initializes the offsets to the committed + offsets. An exception will be thrown at runtime if there is no committed offsets. + + An optional :class:`KafkaOffsetResetStrategy` can be specified to initialize the offsets if + the committed offsets does not exist. + + :param offset_reset_strategy: the offset reset strategy to use when the committed offsets do + not exist. + :return: an offset initializer which initialize the offsets to the committed offsets. + """ + JOffsetsInitializer = get_gateway().jvm.org.apache.flink.connector.kafka.source.\ + enumerator.initializer.OffsetsInitializer + return KafkaOffsetsInitializer(JOffsetsInitializer.committedOffsets( + offset_reset_strategy._to_j_offset_reset_strategy())) + + @staticmethod + def timestamp(timestamp: int) -> 'KafkaOffsetsInitializer': + """ + Get an :class:`KafkaOffsetsInitializer` which initializes the offsets in each partition so + that the initialized offset is the offset of the first record whose record timestamp is + greater than or equals the give timestamp. + + :param timestamp: the timestamp to start the consumption. + :return: an :class:`OffsetsInitializer` which initializes the offsets based on the given + timestamp. + """ + JOffsetsInitializer = get_gateway().jvm.org.apache.flink.connector.kafka.source. \ + enumerator.initializer.OffsetsInitializer + return KafkaOffsetsInitializer(JOffsetsInitializer.timestamp(timestamp)) + + @staticmethod + def earliest() -> 'KafkaOffsetsInitializer': + """ + Get an :class:`KafkaOffsetsInitializer` which initializes the offsets to the earliest + available offsets of each partition. + + :return: an :class:`KafkaOffsetsInitializer` which initializes the offsets to the earliest + available offsets. + """ + JOffsetsInitializer = get_gateway().jvm.org.apache.flink.connector.kafka.source. \ + enumerator.initializer.OffsetsInitializer + return KafkaOffsetsInitializer(JOffsetsInitializer.earliest()) + + @staticmethod + def latest() -> 'KafkaOffsetsInitializer': + """ + Get an :class:`KafkaOffsetsInitializer` which initializes the offsets to the latest offsets + of each partition. + + :return: an :class:`KafkaOffsetsInitializer` which initializes the offsets to the latest + offsets. + """ + JOffsetsInitializer = get_gateway().jvm.org.apache.flink.connector.kafka.source. \ + enumerator.initializer.OffsetsInitializer + return KafkaOffsetsInitializer(JOffsetsInitializer.latest()) + + @staticmethod + def offsets(offsets: Dict['KafkaTopicPartition', int], + offset_reset_strategy: 'KafkaOffsetResetStrategy' = + KafkaOffsetResetStrategy.EARLIEST) -> 'KafkaOffsetsInitializer': + """ + Get an :class:`KafkaOffsetsInitializer` which initializes the offsets to the specified + offsets. + + An optional :class:`KafkaOffsetResetStrategy` can be specified to initialize the offsets in + case the specified offset is out of range. + + Example: + :: + + >>> KafkaOffsetsInitializer.offsets({ + ... KafkaTopicPartition('TOPIC1', 0): 0, + ... KafkaTopicPartition('TOPIC1', 1): 10000 + ... }, KafkaOffsetResetStrategy.EARLIEST) + + :param offsets: the specified offsets for each partition. + :param offset_reset_strategy: the :class:`KafkaOffsetResetStrategy` to use when the + specified offset is out of range. + :return: an :class:`KafkaOffsetsInitializer` which initializes the offsets to the specified + offsets. + """ + jvm = get_gateway().jvm + j_map_wrapper = jvm.org.apache.flink.python.util.HashMapWrapper( + None, get_java_class(jvm.Long)) + for tp, offset in offsets.items(): + j_map_wrapper.put(tp._to_j_topic_partition(), offset) + + JOffsetsInitializer = get_gateway().jvm.org.apache.flink.connector.kafka.source. \ + enumerator.initializer.OffsetsInitializer + return KafkaOffsetsInitializer(JOffsetsInitializer.offsets( + j_map_wrapper.asMap(), offset_reset_strategy._to_j_offset_reset_strategy())) + + +class KafkaSink(Sink, SupportsPreprocessing): + """ + Flink Sink to produce data into a Kafka topic. The sink supports all delivery guarantees + described by :class:`DeliveryGuarantee`. + + * :attr:`DeliveryGuarantee.NONE` does not provide any guarantees: messages may be lost in case + of issues on the Kafka broker and messages may be duplicated in case of a Flink failure. + * :attr:`DeliveryGuarantee.AT_LEAST_ONCE` the sink will wait for all outstanding records in the + Kafka buffers to be acknowledged by the Kafka producer on a checkpoint. No messages will be + lost in case of any issue with the Kafka brokers but messages may be duplicated when Flink + restarts. + * :attr:`DeliveryGuarantee.EXACTLY_ONCE`: In this mode the KafkaSink will write all messages in + a Kafka transaction that will be committed to Kafka on a checkpoint. Thus, if the consumer + reads only committed data (see Kafka consumer config ``isolation.level``), no duplicates + will be seen in case of a Flink restart. However, this delays record writing effectively + until a checkpoint is written, so adjust the checkpoint duration accordingly. Please ensure + that you use unique transactional id prefixes across your applications running on the same + Kafka cluster such that multiple running jobs do not interfere in their transactions! + Additionally, it is highly recommended to tweak Kafka transaction timeout (link) >> maximum + checkpoint duration + maximum restart duration or data loss may happen when Kafka expires an + uncommitted transaction. + + .. versionadded:: 1.16.0 + """ + + def __init__(self, j_kafka_sink, transformer: Optional[StreamTransformer] = None): + super().__init__(j_kafka_sink) + self._transformer = transformer + + @staticmethod + def builder() -> 'KafkaSinkBuilder': + """ + Create a :class:`KafkaSinkBuilder` to construct :class:`KafkaSink`. + """ + return KafkaSinkBuilder() + + def get_transformer(self) -> Optional[StreamTransformer]: + return self._transformer + + +class KafkaSinkBuilder(object): + """ + Builder to construct :class:`KafkaSink`. + + The following example shows the minimum setup to create a KafkaSink that writes String values + to a Kafka topic. + + :: + + >>> record_serializer = KafkaRecordSerializationSchema.builder() \\ + ... .set_topic(MY_SINK_TOPIC) \\ + ... .set_value_serialization_schema(SimpleStringSchema()) \\ + ... .build() + >>> sink = KafkaSink.builder() \\ + ... .set_bootstrap_servers(MY_BOOTSTRAP_SERVERS) \\ + ... .set_record_serializer(record_serializer) \\ + ... .build() + + One can also configure different :class:`DeliveryGuarantee` by using + :meth:`set_delivery_guarantee` but keep in mind when using + :attr:`DeliveryGuarantee.EXACTLY_ONCE`, one must set the transactional id prefix + :meth:`set_transactional_id_prefix`. + + .. versionadded:: 1.16.0 + """ + + def __init__(self): + jvm = get_gateway().jvm + self._j_builder = jvm.org.apache.flink.connector.kafka.sink.KafkaSink.builder() + self._preprocessing = None + + def build(self) -> 'KafkaSink': + """ + Constructs the :class:`KafkaSink` with the configured properties. + """ + return KafkaSink(self._j_builder.build(), self._preprocessing) + + def set_bootstrap_servers(self, bootstrap_servers: str) -> 'KafkaSinkBuilder': + """ + Sets the Kafka bootstrap servers. + + :param bootstrap_servers: A comma separated list of valid URIs to reach the Kafka broker. + """ + self._j_builder.setBootstrapServers(bootstrap_servers) + return self + + def set_delivery_guarantee(self, delivery_guarantee: DeliveryGuarantee) -> 'KafkaSinkBuilder': + """ + Sets the wanted :class:`DeliveryGuarantee`. The default delivery guarantee is + :attr:`DeliveryGuarantee.NONE`. + + :param delivery_guarantee: The wanted :class:`DeliveryGuarantee`. + """ + self._j_builder.setDeliveryGuarantee(delivery_guarantee._to_j_delivery_guarantee()) + return self + + def set_transactional_id_prefix(self, transactional_id_prefix: str) -> 'KafkaSinkBuilder': + """ + Sets the prefix for all created transactionalIds if :attr:`DeliveryGuarantee.EXACTLY_ONCE` + is configured. + + It is mandatory to always set this value with :attr:`DeliveryGuarantee.EXACTLY_ONCE` to + prevent corrupted transactions if multiple jobs using the KafkaSink run against the same + Kafka Cluster. The default prefix is ``"kafka-sink"``. + + The size of the prefix is capped by MAXIMUM_PREFIX_BYTES (6400) formatted with UTF-8. + + It is important to keep the prefix stable across application restarts. If the prefix changes + it might happen that lingering transactions are not correctly aborted and newly written + messages are not immediately consumable until transactions timeout. + + :param transactional_id_prefix: The transactional id prefix. + """ + self._j_builder.setTransactionalIdPrefix(transactional_id_prefix) + return self + + def set_record_serializer(self, record_serializer: 'KafkaRecordSerializationSchema') \ + -> 'KafkaSinkBuilder': + """ + Sets the :class:`KafkaRecordSerializationSchema` that transforms incoming records to kafka + producer records. + + :param record_serializer: The :class:`KafkaRecordSerializationSchema`. + """ + # NOTE: If topic selector is a generated first-column selector, do extra preprocessing + j_topic_selector = get_field_value(record_serializer._j_serialization_schema, + 'topicSelector') + if ( + j_topic_selector.getClass().getCanonicalName() == + 'org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchemaBuilder.' + 'CachingTopicSelector' + ) and ( + get_field_value(j_topic_selector, 'topicSelector').getClass().getCanonicalName() + is not None and + (get_field_value(j_topic_selector, 'topicSelector').getClass().getCanonicalName() + .startswith('com.sun.proxy') or + get_field_value(j_topic_selector, 'topicSelector').getClass().getCanonicalName() + .startswith('jdk.proxy')) + ): + record_serializer._wrap_serialization_schema() + self._preprocessing = record_serializer._build_preprocessing() + + self._j_builder.setRecordSerializer(record_serializer._j_serialization_schema) + return self + + def set_property(self, key: str, value: str) -> 'KafkaSinkBuilder': + """ + Sets kafka producer config. + + :param key: Kafka producer config key. + :param value: Kafka producer config value. + """ + self._j_builder.setProperty(key, value) + return self + + +class KafkaRecordSerializationSchema(SerializationSchema): + """ + A serialization schema which defines how to convert the stream record to kafka producer record. + + .. versionadded:: 1.16.0 + """ + + def __init__(self, j_serialization_schema, + topic_selector: Optional['KafkaTopicSelector'] = None): + super().__init__(j_serialization_schema) + self._topic_selector = topic_selector + + @staticmethod + def builder() -> 'KafkaRecordSerializationSchemaBuilder': + """ + Creates a default schema builder to provide common building blocks i.e. key serialization, + value serialization, topic selection. + """ + return KafkaRecordSerializationSchemaBuilder() + + def _wrap_serialization_schema(self): + jvm = get_gateway().jvm + + def _wrap_schema(field_name): + j_schema_field = get_field(self._j_serialization_schema.getClass(), field_name) + if j_schema_field.get(self._j_serialization_schema) is not None: + j_schema_field.set( + self._j_serialization_schema, + jvm.org.apache.flink.python.util.PythonConnectorUtils + .SecondColumnSerializationSchema( + j_schema_field.get(self._j_serialization_schema) + ) + ) + + _wrap_schema('keySerializationSchema') + _wrap_schema('valueSerializationSchema') + + def _build_preprocessing(self) -> StreamTransformer: + class SelectTopicTransformer(StreamTransformer): + + def __init__(self, topic_selector: KafkaTopicSelector): + self._topic_selector = topic_selector + + def apply(self, ds): + output_type = Types.ROW([Types.STRING(), ds.get_type()]) + return ds.map(lambda v: Row(self._topic_selector.apply(v), v), + output_type=output_type) + + return SelectTopicTransformer(self._topic_selector) + + +class KafkaRecordSerializationSchemaBuilder(object): + """ + Builder to construct :class:`KafkaRecordSerializationSchema`. + + Example: + :: + + >>> KafkaRecordSerializationSchema.builder() \\ + ... .set_topic('topic') \\ + ... .set_key_serialization_schema(SimpleStringSchema()) \\ + ... .set_value_serialization_schema(SimpleStringSchema()) \\ + ... .build() + + And the sink topic can be calculated dynamically from each record: + :: + + >>> KafkaRecordSerializationSchema.builder() \\ + ... .set_topic_selector(lambda row: 'topic-' + row['category']) \\ + ... .set_value_serialization_schema( + ... JsonRowSerializationSchema.builder().with_type_info(ROW_TYPE).build()) \\ + ... .build() + + It is necessary to configure exactly one serialization method for the value and a topic. + + .. versionadded:: 1.16.0 + """ + + def __init__(self): + jvm = get_gateway().jvm + self._j_builder = jvm.org.apache.flink.connector.kafka.sink \ + .KafkaRecordSerializationSchemaBuilder() + self._fixed_topic = True # type: bool + self._topic_selector = None # type: Optional[KafkaTopicSelector] + self._key_serialization_schema = None # type: Optional[SerializationSchema] + self._value_serialization_schema = None # type: Optional[SerializationSchema] + + def build(self) -> 'KafkaRecordSerializationSchema': + """ + Constructs the :class:`KafkaRecordSerializationSchemaBuilder` with the configured + properties. + """ + if self._fixed_topic: + return KafkaRecordSerializationSchema(self._j_builder.build()) + else: + return KafkaRecordSerializationSchema(self._j_builder.build(), self._topic_selector) + + def set_topic(self, topic: str) -> 'KafkaRecordSerializationSchemaBuilder': + """ + Sets a fixed topic which used as destination for all records. + + :param topic: The fixed topic. + """ + self._j_builder.setTopic(topic) + self._fixed_topic = True + return self + + def set_topic_selector(self, topic_selector: Union[Callable[[Any], str], 'KafkaTopicSelector'])\ + -> 'KafkaRecordSerializationSchemaBuilder': + """ + Sets a topic selector which computes the target topic for every incoming record. + + :param topic_selector: A :class:`KafkaTopicSelector` implementation or a function that + consumes each incoming record and return the topic string. + """ + if not isinstance(topic_selector, KafkaTopicSelector) and not callable(topic_selector): + raise TypeError('topic_selector must be KafkaTopicSelector or a callable') + if not isinstance(topic_selector, KafkaTopicSelector): + class TopicSelectorFunctionAdapter(KafkaTopicSelector): + + def __init__(self, f: Callable[[Any], str]): + self._f = f + + def apply(self, data) -> str: + return self._f(data) + + topic_selector = TopicSelectorFunctionAdapter(topic_selector) + + jvm = get_gateway().jvm + self._j_builder.setTopicSelector( + jvm.org.apache.flink.python.util.PythonConnectorUtils.createFirstColumnTopicSelector( + get_java_class(jvm.org.apache.flink.connector.kafka.sink.TopicSelector) + ) + ) + self._fixed_topic = False + self._topic_selector = topic_selector + return self + + def set_key_serialization_schema(self, key_serialization_schema: SerializationSchema) \ + -> 'KafkaRecordSerializationSchemaBuilder': + """ + Sets a :class:`SerializationSchema` which is used to serialize the incoming element to the + key of the producer record. The key serialization is optional, if not set, the key of the + producer record will be null. + + :param key_serialization_schema: The :class:`SerializationSchema` to serialize each incoming + record as the key of producer record. + """ + self._key_serialization_schema = key_serialization_schema + self._j_builder.setKeySerializationSchema(key_serialization_schema._j_serialization_schema) + return self + + def set_value_serialization_schema(self, value_serialization_schema: SerializationSchema) \ + -> 'KafkaRecordSerializationSchemaBuilder': + """ + Sets a :class:`SerializationSchema` which is used to serialize the incoming element to the + value of the producer record. The value serialization is required. + + :param value_serialization_schema: The :class:`SerializationSchema` to serialize each data + record as the key of producer record. + """ + self._value_serialization_schema = value_serialization_schema + self._j_builder.setValueSerializationSchema( + value_serialization_schema._j_serialization_schema) + return self + + +class KafkaTopicSelector(ABC): + """ + Select topic for an incoming record + + .. versionadded:: 1.16.0 + """ + + @abstractmethod + def apply(self, data) -> str: + pass diff --git a/flink-python/pyflink/datastream/connectors/tests/test_kafka.py b/flink-python/pyflink/datastream/connectors/tests/test_kafka.py new file mode 100644 index 000000000..dea06b3e0 --- /dev/null +++ b/flink-python/pyflink/datastream/connectors/tests/test_kafka.py @@ -0,0 +1,669 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import json +from typing import Dict + +import pyflink.datastream.data_stream as data_stream +from pyflink.common import typeinfo + +from pyflink.common.configuration import Configuration +from pyflink.common.serialization import SimpleStringSchema, DeserializationSchema +from pyflink.common.typeinfo import Types +from pyflink.common.types import Row +from pyflink.common.watermark_strategy import WatermarkStrategy +from pyflink.datastream.connectors.base import DeliveryGuarantee +from pyflink.datastream.connectors.kafka import KafkaSource, KafkaTopicPartition, \ + KafkaOffsetsInitializer, KafkaOffsetResetStrategy, KafkaRecordSerializationSchema, KafkaSink, \ + FlinkKafkaProducer, FlinkKafkaConsumer +from pyflink.datastream.formats.avro import AvroRowDeserializationSchema, AvroRowSerializationSchema +from pyflink.datastream.formats.csv import CsvRowDeserializationSchema, CsvRowSerializationSchema +from pyflink.datastream.formats.json import JsonRowDeserializationSchema, JsonRowSerializationSchema +from pyflink.java_gateway import get_gateway +from pyflink.testing.test_case_utils import ( + PyFlinkStreamingTestCase, + PyFlinkTestCase, + invoke_java_object_method, + to_java_data_structure, +) +from pyflink.util.java_utils import to_jarray, is_instance_of, get_field_value + + +class KafkaSourceTests(PyFlinkStreamingTestCase): + + def test_legacy_kafka_connector(self): + source_topic = 'test_source_topic' + sink_topic = 'test_sink_topic' + props = {'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group'} + type_info = Types.ROW([Types.INT(), Types.STRING()]) + + # Test for kafka consumer + deserialization_schema = JsonRowDeserializationSchema.builder() \ + .type_info(type_info=type_info).build() + + flink_kafka_consumer = FlinkKafkaConsumer(source_topic, deserialization_schema, props) + flink_kafka_consumer.set_start_from_earliest() + flink_kafka_consumer.set_commit_offsets_on_checkpoints(True) + + j_properties = get_field_value(flink_kafka_consumer.get_java_function(), 'properties') + self.assertEqual('localhost:9092', j_properties.getProperty('bootstrap.servers')) + self.assertEqual('test_group', j_properties.getProperty('group.id')) + self.assertTrue(get_field_value(flink_kafka_consumer.get_java_function(), + 'enableCommitOnCheckpoints')) + j_start_up_mode = get_field_value(flink_kafka_consumer.get_java_function(), 'startupMode') + + j_deserializer = get_field_value(flink_kafka_consumer.get_java_function(), 'deserializer') + j_deserialize_type_info = invoke_java_object_method(j_deserializer, "getProducedType") + deserialize_type_info = typeinfo._from_java_type(j_deserialize_type_info) + self.assertTrue(deserialize_type_info == type_info) + self.assertTrue(j_start_up_mode.equals(get_gateway().jvm + .org.apache.flink.streaming.connectors + .kafka.config.StartupMode.EARLIEST)) + j_topic_desc = get_field_value(flink_kafka_consumer.get_java_function(), + 'topicsDescriptor') + j_topics = invoke_java_object_method(j_topic_desc, 'getFixedTopics') + self.assertEqual(['test_source_topic'], list(j_topics)) + + # Test for kafka producer + serialization_schema = JsonRowSerializationSchema.builder().with_type_info(type_info) \ + .build() + flink_kafka_producer = FlinkKafkaProducer(sink_topic, serialization_schema, props) + flink_kafka_producer.set_write_timestamp_to_kafka(False) + + j_producer_config = get_field_value(flink_kafka_producer.get_java_function(), + 'producerConfig') + self.assertEqual('localhost:9092', j_producer_config.getProperty('bootstrap.servers')) + self.assertEqual('test_group', j_producer_config.getProperty('group.id')) + self.assertFalse(get_field_value(flink_kafka_producer.get_java_function(), + 'writeTimestampToKafka')) + + def test_compiling(self): + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .build() + + ds = self.env.from_source(source=source, + watermark_strategy=WatermarkStrategy.for_monotonous_timestamps(), + source_name='kafka source') + ds.print() + plan = json.loads(self.env.get_execution_plan()) + self.assertEqual('Source: kafka source', plan['nodes'][0]['type']) + + def test_set_properties(self): + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_group_id('test_group_id') \ + .set_client_id_prefix('test_client_id_prefix') \ + .set_property('test_property', 'test_value') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .build() + conf = self._get_kafka_source_configuration(source) + self.assertEqual(conf.get_string('bootstrap.servers', ''), 'localhost:9092') + self.assertEqual(conf.get_string('group.id', ''), 'test_group_id') + self.assertEqual(conf.get_string('client.id.prefix', ''), 'test_client_id_prefix') + self.assertEqual(conf.get_string('test_property', ''), 'test_value') + + def test_set_topics(self): + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic1', 'test_topic2') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .build() + kafka_subscriber = get_field_value(source.get_java_function(), 'subscriber') + self.assertEqual( + kafka_subscriber.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.subscriber.TopicListSubscriber' + ) + topics = get_field_value(kafka_subscriber, 'topics') + self.assertTrue(is_instance_of(topics, get_gateway().jvm.java.util.List)) + self.assertEqual(topics.size(), 2) + self.assertEqual(topics[0], 'test_topic1') + self.assertEqual(topics[1], 'test_topic2') + + def test_set_topic_pattern(self): + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topic_pattern('test_topic*') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .build() + kafka_subscriber = get_field_value(source.get_java_function(), 'subscriber') + self.assertEqual( + kafka_subscriber.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.subscriber.TopicPatternSubscriber' + ) + topic_pattern = get_field_value(kafka_subscriber, 'topicPattern') + self.assertTrue(is_instance_of(topic_pattern, get_gateway().jvm.java.util.regex.Pattern)) + self.assertEqual(topic_pattern.toString(), 'test_topic*') + + def test_set_partitions(self): + topic_partition_1 = KafkaTopicPartition('test_topic', 1) + topic_partition_2 = KafkaTopicPartition('test_topic', 2) + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_partitions({topic_partition_1, topic_partition_2}) \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .build() + kafka_subscriber = get_field_value(source.get_java_function(), 'subscriber') + self.assertEqual( + kafka_subscriber.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.subscriber.PartitionSetSubscriber' + ) + partitions = get_field_value(kafka_subscriber, 'subscribedPartitions') + self.assertTrue(is_instance_of(partitions, get_gateway().jvm.java.util.Set)) + self.assertTrue(topic_partition_1._to_j_topic_partition() in partitions) + self.assertTrue(topic_partition_2._to_j_topic_partition() in partitions) + + def test_set_starting_offsets(self): + def _build_source(initializer: KafkaOffsetsInitializer): + return KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .set_group_id('test_group') \ + .set_starting_offsets(initializer) \ + .build() + + self._check_latest_offsets_initializer( + _build_source(KafkaOffsetsInitializer.latest())) + self._check_reader_handled_offsets_initializer( + _build_source(KafkaOffsetsInitializer.earliest()), -2, + KafkaOffsetResetStrategy.EARLIEST + ) + self._check_reader_handled_offsets_initializer( + _build_source(KafkaOffsetsInitializer.committed_offsets()), -3, + KafkaOffsetResetStrategy.NONE + ) + self._check_reader_handled_offsets_initializer( + _build_source(KafkaOffsetsInitializer.committed_offsets( + KafkaOffsetResetStrategy.LATEST + )), -3, KafkaOffsetResetStrategy.LATEST + ) + self._check_timestamp_offsets_initializer( + _build_source(KafkaOffsetsInitializer.timestamp(100)), 100 + ) + specified_offsets = { + KafkaTopicPartition('test_topic1', 1): 1000, + KafkaTopicPartition('test_topic2', 2): 2000 + } + self._check_specified_offsets_initializer( + _build_source(KafkaOffsetsInitializer.offsets(specified_offsets)), specified_offsets, + KafkaOffsetResetStrategy.EARLIEST + ) + self._check_specified_offsets_initializer( + _build_source(KafkaOffsetsInitializer.offsets( + specified_offsets, + KafkaOffsetResetStrategy.LATEST + )), + specified_offsets, + KafkaOffsetResetStrategy.LATEST + ) + + def test_bounded(self): + def _build_source(initializer: KafkaOffsetsInitializer): + return KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .set_group_id('test_group') \ + .set_bounded(initializer) \ + .build() + + def _check_bounded(source: KafkaSource): + self.assertEqual( + get_field_value(source.get_java_function(), 'boundedness').toString(), 'BOUNDED' + ) + + self._test_set_bounded_or_unbounded(_build_source, _check_bounded) + + def test_unbounded(self): + def _build_source(initializer: KafkaOffsetsInitializer): + return KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .set_group_id('test_group') \ + .set_unbounded(initializer) \ + .build() + + def _check_bounded(source: KafkaSource): + self.assertEqual( + get_field_value(source.get_java_function(), 'boundedness').toString(), + 'CONTINUOUS_UNBOUNDED' + ) + + self._test_set_bounded_or_unbounded(_build_source, _check_bounded) + + def _test_set_bounded_or_unbounded(self, _build_source, _check_boundedness): + source = _build_source(KafkaOffsetsInitializer.latest()) + _check_boundedness(source) + self._check_latest_offsets_initializer(source, False) + source = _build_source(KafkaOffsetsInitializer.earliest()) + _check_boundedness(source) + self._check_reader_handled_offsets_initializer( + source, -2, KafkaOffsetResetStrategy.EARLIEST, False + ) + source = _build_source(KafkaOffsetsInitializer.committed_offsets()) + _check_boundedness(source) + self._check_reader_handled_offsets_initializer( + source, -3, KafkaOffsetResetStrategy.NONE, False + ) + source = _build_source(KafkaOffsetsInitializer.committed_offsets( + KafkaOffsetResetStrategy.LATEST + )) + _check_boundedness(source) + self._check_reader_handled_offsets_initializer( + source, -3, KafkaOffsetResetStrategy.LATEST, False + ) + source = _build_source(KafkaOffsetsInitializer.timestamp(100)) + _check_boundedness(source) + self._check_timestamp_offsets_initializer(source, 100, False) + specified_offsets = { + KafkaTopicPartition('test_topic1', 1): 1000, + KafkaTopicPartition('test_topic2', 2): 2000 + } + source = _build_source(KafkaOffsetsInitializer.offsets(specified_offsets)) + _check_boundedness(source) + self._check_specified_offsets_initializer( + source, specified_offsets, KafkaOffsetResetStrategy.EARLIEST, False + ) + source = _build_source(KafkaOffsetsInitializer.offsets( + specified_offsets, + KafkaOffsetResetStrategy.LATEST) + ) + _check_boundedness(source) + self._check_specified_offsets_initializer( + source, specified_offsets, KafkaOffsetResetStrategy.LATEST, False + ) + + def test_set_value_only_deserializer(self): + def _check(schema: DeserializationSchema, class_name: str): + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(schema) \ + .build() + deserialization_schema_wrapper = get_field_value(source.get_java_function(), + 'deserializationSchema') + self.assertEqual( + deserialization_schema_wrapper.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.reader.deserializer' + '.KafkaValueOnlyDeserializationSchemaWrapper' + ) + deserialization_schema = get_field_value(deserialization_schema_wrapper, + 'deserializationSchema') + self.assertEqual(deserialization_schema.getClass().getCanonicalName(), + class_name) + + _check(SimpleStringSchema(), 'org.apache.flink.api.common.serialization.SimpleStringSchema') + _check( + JsonRowDeserializationSchema.builder().type_info(Types.ROW([Types.STRING()])).build(), + 'org.apache.flink.formats.json.JsonRowDeserializationSchema' + ) + _check( + CsvRowDeserializationSchema.Builder(Types.ROW([Types.STRING()])).build(), + 'org.apache.flink.formats.csv.CsvRowDeserializationSchema' + ) + avro_schema_string = """ + { + "type": "record", + "name": "test_record", + "fields": [] + } + """ + _check( + AvroRowDeserializationSchema(avro_schema_string=avro_schema_string), + 'org.apache.flink.formats.avro.AvroRowDeserializationSchema' + ) + + def _check_reader_handled_offsets_initializer(self, + source: KafkaSource, + offset: int, + reset_strategy: KafkaOffsetResetStrategy, + is_start: bool = True): + if is_start: + field_name = 'startingOffsetsInitializer' + else: + field_name = 'stoppingOffsetsInitializer' + offsets_initializer = get_field_value(source.get_java_function(), field_name) + self.assertEqual( + offsets_initializer.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.initializer' + '.ReaderHandledOffsetsInitializer' + ) + + starting_offset = get_field_value(offsets_initializer, 'startingOffset') + self.assertEqual(starting_offset, offset) + + offset_reset_strategy = get_field_value(offsets_initializer, 'offsetResetStrategy') + self.assertTrue( + offset_reset_strategy.equals(reset_strategy._to_j_offset_reset_strategy()) + ) + + def _check_latest_offsets_initializer(self, + source: KafkaSource, + is_start: bool = True): + if is_start: + field_name = 'startingOffsetsInitializer' + else: + field_name = 'stoppingOffsetsInitializer' + offsets_initializer = get_field_value(source.get_java_function(), field_name) + self.assertEqual( + offsets_initializer.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.initializer' + '.LatestOffsetsInitializer' + ) + + def _check_timestamp_offsets_initializer(self, + source: KafkaSource, + timestamp: int, + is_start: bool = True): + if is_start: + field_name = 'startingOffsetsInitializer' + else: + field_name = 'stoppingOffsetsInitializer' + offsets_initializer = get_field_value(source.get_java_function(), field_name) + self.assertEqual( + offsets_initializer.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.initializer' + '.TimestampOffsetsInitializer' + ) + + starting_timestamp = get_field_value(offsets_initializer, 'startingTimestamp') + self.assertEqual(starting_timestamp, timestamp) + + def _check_specified_offsets_initializer(self, + source: KafkaSource, + offsets: Dict[KafkaTopicPartition, int], + reset_strategy: KafkaOffsetResetStrategy, + is_start: bool = True): + if is_start: + field_name = 'startingOffsetsInitializer' + else: + field_name = 'stoppingOffsetsInitializer' + offsets_initializer = get_field_value(source.get_java_function(), field_name) + self.assertEqual( + offsets_initializer.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.initializer' + '.SpecifiedOffsetsInitializer' + ) + + initial_offsets = get_field_value(offsets_initializer, 'initialOffsets') + self.assertTrue(is_instance_of(initial_offsets, get_gateway().jvm.java.util.Map)) + self.assertEqual(initial_offsets.size(), len(offsets)) + for j_topic_partition in initial_offsets: + topic_partition = KafkaTopicPartition(j_topic_partition.topic(), + j_topic_partition.partition()) + self.assertIsNotNone(offsets.get(topic_partition)) + self.assertEqual(initial_offsets[j_topic_partition], offsets[topic_partition]) + + offset_reset_strategy = get_field_value(offsets_initializer, 'offsetResetStrategy') + self.assertTrue( + offset_reset_strategy.equals(reset_strategy._to_j_offset_reset_strategy()) + ) + + @staticmethod + def _get_kafka_source_configuration(source: KafkaSource): + jvm = get_gateway().jvm + j_source = source.get_java_function() + j_to_configuration = j_source.getClass().getDeclaredMethod( + 'getConfiguration', to_jarray(jvm.java.lang.Class, []) + ) + j_to_configuration.setAccessible(True) + j_configuration = j_to_configuration.invoke(j_source, to_jarray(jvm.java.lang.Object, [])) + return Configuration(j_configuration=j_configuration) + + +class KafkaSinkTests(PyFlinkStreamingTestCase): + + def test_compile(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + + ds = self.env.from_collection([], type_info=Types.STRING()) + ds.sink_to(sink) + + plan = json.loads(self.env.get_execution_plan()) + self.assertEqual(plan['nodes'][1]['type'], 'Sink: Writer') + self.assertEqual(plan['nodes'][2]['type'], 'Sink: Committer') + + def test_set_bootstrap_severs(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092,localhost:9093') \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + config = get_field_value(sink.get_java_function(), 'kafkaProducerConfig') + self.assertEqual(config.get('bootstrap.servers'), 'localhost:9092,localhost:9093') + + def test_set_delivery_guarantee(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + guarantee = get_field_value(sink.get_java_function(), 'deliveryGuarantee') + self.assertEqual(guarantee.toString(), 'none') + + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_delivery_guarantee(DeliveryGuarantee.AT_LEAST_ONCE) \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + guarantee = get_field_value(sink.get_java_function(), 'deliveryGuarantee') + self.assertEqual(guarantee.toString(), 'at-least-once') + + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_delivery_guarantee(DeliveryGuarantee.EXACTLY_ONCE) \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + guarantee = get_field_value(sink.get_java_function(), 'deliveryGuarantee') + self.assertEqual(guarantee.toString(), 'exactly-once') + + def test_set_transactional_id_prefix(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_transactional_id_prefix('test-prefix') \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + prefix = get_field_value(sink.get_java_function(), 'transactionalIdPrefix') + self.assertEqual(prefix, 'test-prefix') + + def test_set_property(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_record_serializer(self._build_serialization_schema()) \ + .set_property('test-key', 'test-value') \ + .build() + config = get_field_value(sink.get_java_function(), 'kafkaProducerConfig') + self.assertEqual(config.get('test-key'), 'test-value') + + def test_set_record_serializer(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + serializer = get_field_value(sink.get_java_function(), 'recordSerializer') + self.assertEqual(serializer.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.sink.' + 'KafkaRecordSerializationSchemaBuilder.' + 'KafkaRecordSerializationSchemaWrapper') + topic_selector = get_field_value(serializer, 'topicSelector') + self.assertEqual(topic_selector.apply(None), 'test-topic') + value_serializer = get_field_value(serializer, 'valueSerializationSchema') + self.assertEqual(value_serializer.getClass().getCanonicalName(), + 'org.apache.flink.api.common.serialization.SimpleStringSchema') + + @staticmethod + def _build_serialization_schema() -> KafkaRecordSerializationSchema: + return KafkaRecordSerializationSchema.builder() \ + .set_topic('test-topic') \ + .set_value_serialization_schema(SimpleStringSchema()) \ + .build() + + +class KafkaRecordSerializationSchemaTests(PyFlinkTestCase): + + def test_set_topic(self): + input_type = Types.ROW([Types.STRING()]) + + serialization_schema = KafkaRecordSerializationSchema.builder() \ + .set_topic('test-topic') \ + .set_value_serialization_schema( + JsonRowSerializationSchema.builder().with_type_info(input_type).build()) \ + .build() + jvm = get_gateway().jvm + serialization_schema._j_serialization_schema.open( + jvm.org.apache.flink.connector.testutils.formats.DummyInitializationContext(), + jvm.org.apache.flink.connector.kafka.sink.DefaultKafkaSinkContext( + 0, 1, jvm.java.util.Properties())) + + j_record = serialization_schema._j_serialization_schema.serialize( + to_java_data_structure(Row('test')), None, None + ) + self.assertEqual(j_record.topic(), 'test-topic') + self.assertIsNone(j_record.key()) + self.assertEqual(j_record.value(), b'{"f0":"test"}') + + def test_set_topic_selector(self): + def _select(data): + data = data[0] + if data == 'a': + return 'topic-a' + elif data == 'b': + return 'topic-b' + else: + return 'topic-dead-letter' + + def _check_record(data, topic, serialized_data): + input_type = Types.ROW([Types.STRING()]) + + serialization_schema = KafkaRecordSerializationSchema.builder() \ + .set_topic_selector(_select) \ + .set_value_serialization_schema( + JsonRowSerializationSchema.builder().with_type_info(input_type).build()) \ + .build() + jvm = get_gateway().jvm + serialization_schema._j_serialization_schema.open( + jvm.org.apache.flink.connector.testutils.formats.DummyInitializationContext(), + jvm.org.apache.flink.connector.kafka.sink.DefaultKafkaSinkContext( + 0, 1, jvm.java.util.Properties())) + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_record_serializer(serialization_schema) \ + .build() + + ds = MockDataStream(Types.ROW([Types.STRING()])) + ds.sink_to(sink) + row = Row(data) + topic_row = ds.feed(row) # type: Row + j_record = serialization_schema._j_serialization_schema.serialize( + to_java_data_structure(topic_row), None, None + ) + self.assertEqual(j_record.topic(), topic) + self.assertIsNone(j_record.key()) + self.assertEqual(j_record.value(), serialized_data) + + _check_record('a', 'topic-a', b'{"f0":"a"}') + _check_record('b', 'topic-b', b'{"f0":"b"}') + _check_record('c', 'topic-dead-letter', b'{"f0":"c"}') + _check_record('d', 'topic-dead-letter', b'{"f0":"d"}') + + def test_set_key_serialization_schema(self): + def _check_key_serialization_schema(key_serialization_schema, expected_class): + serialization_schema = KafkaRecordSerializationSchema.builder() \ + .set_topic('test-topic') \ + .set_key_serialization_schema(key_serialization_schema) \ + .set_value_serialization_schema(SimpleStringSchema()) \ + .build() + schema_field = get_field_value(serialization_schema._j_serialization_schema, + 'keySerializationSchema') + self.assertIsNotNone(schema_field) + self.assertEqual(schema_field.getClass().getCanonicalName(), expected_class) + + self._check_serialization_schema_implementations(_check_key_serialization_schema) + + def test_set_value_serialization_schema(self): + def _check_value_serialization_schema(value_serialization_schema, expected_class): + serialization_schema = KafkaRecordSerializationSchema.builder() \ + .set_topic('test-topic') \ + .set_value_serialization_schema(value_serialization_schema) \ + .build() + schema_field = get_field_value(serialization_schema._j_serialization_schema, + 'valueSerializationSchema') + self.assertIsNotNone(schema_field) + self.assertEqual(schema_field.getClass().getCanonicalName(), expected_class) + + self._check_serialization_schema_implementations(_check_value_serialization_schema) + + @staticmethod + def _check_serialization_schema_implementations(check_function): + input_type = Types.ROW([Types.STRING()]) + + check_function( + JsonRowSerializationSchema.builder().with_type_info(input_type).build(), + 'org.apache.flink.formats.json.JsonRowSerializationSchema' + ) + check_function( + CsvRowSerializationSchema.Builder(input_type).build(), + 'org.apache.flink.formats.csv.CsvRowSerializationSchema' + ) + avro_schema_string = """ + { + "type": "record", + "name": "test_record", + "fields": [] + } + """ + check_function( + AvroRowSerializationSchema(avro_schema_string=avro_schema_string), + 'org.apache.flink.formats.avro.AvroRowSerializationSchema' + ) + check_function( + SimpleStringSchema(), + 'org.apache.flink.api.common.serialization.SimpleStringSchema' + ) + + +class MockDataStream(data_stream.DataStream): + + def __init__(self, original_type=None): + super().__init__(None) + self._operators = [] + self._type = original_type + + def feed(self, data): + for op in self._operators: + data = op(data) + return data + + def get_type(self): + return self._type + + def map(self, f, output_type=None): + self._operators.append(f) + self._type = output_type + + def sink_to(self, sink): + ds = self + from pyflink.datastream.connectors.base import SupportsPreprocessing + if isinstance(sink, SupportsPreprocessing) and sink.get_transformer() is not None: + ds = sink.get_transformer().apply(self) + return ds diff --git a/flink-python/pyflink/pyflink_gateway_server.py b/flink-python/pyflink/pyflink_gateway_server.py new file mode 100644 index 000000000..1cf25a54f --- /dev/null +++ b/flink-python/pyflink/pyflink_gateway_server.py @@ -0,0 +1,288 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +# This is a copy of the pyflink_gateway_server.py file from the Flink. +# The original file which is accessible here: +# https://github.com/apache/flink/blob/master/flink-python/pyflink/pyflink_gateway_server.py +# Additional change is the handling of the FLINK_TEST_LIB_DIR environmental variable. +# It could be used to add extra testing jars for the gateway classpath. +# The plan is to remove this once Pyflink 1.19 is released + +import argparse +import getpass +import glob +import os +import platform +import signal +import socket +import sys +from collections import namedtuple +from string import Template +from subprocess import Popen, PIPE + +from pyflink.find_flink_home import _find_flink_home, _find_flink_source_root + +KEY_ENV_LOG_DIR = "env.log.dir" +KEY_ENV_YARN_CONF_DIR = "env.yarn.conf.dir" +KEY_ENV_HADOOP_CONF_DIR = "env.hadoop.conf.dir" +KEY_ENV_HBASE_CONF_DIR = "env.hbase.conf.dir" +KEY_ENV_JAVA_HOME = "env.java.home" +KEY_ENV_JAVA_OPTS = "env.java.opts.all" +KEY_ENV_JAVA_OPTS_DEPRECATED = "env.java.opts" + + +def on_windows(): + return platform.system() == "Windows" + + +def read_from_config(key, default_value, flink_conf_file): + value = default_value + # get the realpath of tainted path value to avoid CWE22 problem that constructs a path or URI + # using the tainted value and might allow an attacker to access, modify, or test the existence + # of critical or sensitive files. + with open(os.path.realpath(flink_conf_file), "r") as f: + while True: + line = f.readline() + if not line: + break + if line.startswith("#") or len(line.strip()) == 0: + continue + k, v = line.split(":", 1) + if k.strip() == key: + value = v.strip() + return value + + +def find_java_executable(): + java_executable = "java.exe" if on_windows() else "java" + flink_home = _find_flink_home() + flink_conf_file = os.path.join(flink_home, "conf", "flink-conf.yaml") + java_home = read_from_config(KEY_ENV_JAVA_HOME, None, flink_conf_file) + + if java_home is None and "JAVA_HOME" in os.environ: + java_home = os.environ["JAVA_HOME"] + + if java_home is not None: + java_executable = os.path.join(java_home, "bin", java_executable) + + return java_executable + + +def prepare_environment_variables(env): + flink_home = _find_flink_home() + # get the realpath of tainted path value to avoid CWE22 problem that constructs a path or URI + # using the tainted value and might allow an attacker to access, modify, or test the existence + # of critical or sensitive files. + real_flink_home = os.path.realpath(flink_home) + + if 'FLINK_CONF_DIR' in env: + flink_conf_directory = os.path.realpath(env['FLINK_CONF_DIR']) + else: + flink_conf_directory = os.path.join(real_flink_home, "conf") + env['FLINK_CONF_DIR'] = flink_conf_directory + + if 'FLINK_LIB_DIR' in env: + flink_lib_directory = os.path.realpath(env['FLINK_LIB_DIR']) + else: + flink_lib_directory = os.path.join(real_flink_home, "lib") + env['FLINK_LIB_DIR'] = flink_lib_directory + + if 'FLINK_OPT_DIR' in env: + flink_opt_directory = os.path.realpath(env['FLINK_OPT_DIR']) + else: + flink_opt_directory = os.path.join(real_flink_home, "opt") + env['FLINK_OPT_DIR'] = flink_opt_directory + + if 'FLINK_PLUGINS_DIR' in env: + flink_plugins_directory = os.path.realpath(env['FLINK_PLUGINS_DIR']) + else: + flink_plugins_directory = os.path.join(real_flink_home, "plugins") + env['FLINK_PLUGINS_DIR'] = flink_plugins_directory + + env["FLINK_BIN_DIR"] = os.path.join(real_flink_home, "bin") + + +def construct_log_settings(env): + templates = [ + "-Dlog.file=${flink_log_dir}/flink-${flink_ident_string}-python-${hostname}.log", + "-Dlog4j.configuration=${log4j_properties}", + "-Dlog4j.configurationFile=${log4j_properties}", + "-Dlogback.configurationFile=${logback_xml}" + ] + + flink_home = os.path.realpath(_find_flink_home()) + flink_conf_dir = env['FLINK_CONF_DIR'] + flink_conf_file = os.path.join(env['FLINK_CONF_DIR'], "flink-conf.yaml") + + if "FLINK_LOG_DIR" in env: + flink_log_dir = env["FLINK_LOG_DIR"] + else: + flink_log_dir = read_from_config( + KEY_ENV_LOG_DIR, os.path.join(flink_home, "log"), flink_conf_file) + + if "LOG4J_PROPERTIES" in env: + log4j_properties = env["LOG4J_PROPERTIES"] + else: + log4j_properties = "%s/log4j-cli.properties" % flink_conf_dir + + if "LOGBACK_XML" in env: + logback_xml = env["LOGBACK_XML"] + else: + logback_xml = "%s/logback.xml" % flink_conf_dir + + if "FLINK_IDENT_STRING" in env: + flink_ident_string = env["FLINK_IDENT_STRING"] + else: + flink_ident_string = getpass.getuser() + + hostname = socket.gethostname() + log_settings = [] + for template in templates: + log_settings.append(Template(template).substitute( + log4j_properties=log4j_properties, + logback_xml=logback_xml, + flink_log_dir=flink_log_dir, + flink_ident_string=flink_ident_string, + hostname=hostname)) + return log_settings + + +def get_jvm_opts(env): + flink_conf_file = os.path.join(env['FLINK_CONF_DIR'], "flink-conf.yaml") + jvm_opts = env.get( + 'FLINK_ENV_JAVA_OPTS', + read_from_config( + KEY_ENV_JAVA_OPTS, + read_from_config(KEY_ENV_JAVA_OPTS_DEPRECATED, "", flink_conf_file), + flink_conf_file)) + + # Remove leading and ending double quotes (if present) of value + jvm_opts = jvm_opts.strip("\"") + return jvm_opts.split(" ") + + +def construct_flink_classpath(env): + flink_home = _find_flink_home() + flink_lib_directory = env['FLINK_LIB_DIR'] + flink_opt_directory = env['FLINK_OPT_DIR'] + + if on_windows(): + # The command length is limited on Windows. To avoid the problem we should shorten the + # command length as much as possible. + lib_jars = os.path.join(flink_lib_directory, "*") + else: + lib_jars = os.pathsep.join(glob.glob(os.path.join(flink_lib_directory, "*.jar"))) + + flink_python_jars = glob.glob(os.path.join(flink_opt_directory, "flink-python*.jar")) + if len(flink_python_jars) < 1: + print("The flink-python jar is not found in the opt folder of the FLINK_HOME: %s" % + flink_home) + return lib_jars + flink_python_jar = flink_python_jars[0] + + return os.pathsep.join([lib_jars, flink_python_jar]) + + +def construct_hadoop_classpath(env): + flink_conf_file = os.path.join(env['FLINK_CONF_DIR'], "flink-conf.yaml") + + hadoop_conf_dir = "" + if 'HADOOP_CONF_DIR' not in env and 'HADOOP_CLASSPATH' not in env: + if os.path.isdir("/etc/hadoop/conf"): + print("Setting HADOOP_CONF_DIR=/etc/hadoop/conf because no HADOOP_CONF_DIR or" + "HADOOP_CLASSPATH was set.") + hadoop_conf_dir = "/etc/hadoop/conf" + + hbase_conf_dir = "" + if 'HBASE_CONF_DIR' not in env: + if os.path.isdir("/etc/hbase/conf"): + print("Setting HBASE_CONF_DIR=/etc/hbase/conf because no HBASE_CONF_DIR was set.") + hbase_conf_dir = "/etc/hbase/conf" + + return os.pathsep.join( + [env.get("HADOOP_CLASSPATH", ""), + env.get("YARN_CONF_DIR", + read_from_config(KEY_ENV_YARN_CONF_DIR, "", flink_conf_file)), + env.get("HADOOP_CONF_DIR", + read_from_config(KEY_ENV_HADOOP_CONF_DIR, hadoop_conf_dir, flink_conf_file)), + env.get("HBASE_CONF_DIR", + read_from_config(KEY_ENV_HBASE_CONF_DIR, hbase_conf_dir, flink_conf_file))]) + + +def construct_test_classpath(env): + test_jar_patterns = [ + "flink-python/target/test-dependencies/*", + "flink-python/target/artifacts/testDataStream.jar", + "flink-python/target/flink-python*-tests.jar", + ] + test_jars = [] + + # Connector tests need to add specific jars to the gateway classpath + if 'FLINK_TEST_LIBS' in env: + test_jars += glob.glob(env['FLINK_TEST_LIBS']) + else: + flink_source_root = _find_flink_source_root() + for pattern in test_jar_patterns: + pattern = pattern.replace("/", os.path.sep) + test_jars += glob.glob(os.path.join(flink_source_root, pattern)) + return os.path.pathsep.join(test_jars) + + +def construct_program_args(args): + parser = argparse.ArgumentParser() + parser.add_argument("-c", "--class", required=True) + parser.add_argument("cluster_type", choices=["local", "remote", "yarn"]) + parse_result, other_args = parser.parse_known_args(args) + main_class = getattr(parse_result, "class") + cluster_type = parse_result.cluster_type + return namedtuple( + "ProgramArgs", ["main_class", "cluster_type", "other_args"])( + main_class, cluster_type, other_args) + + +def launch_gateway_server_process(env, args): + prepare_environment_variables(env) + program_args = construct_program_args(args) + if program_args.cluster_type == "local": + java_executable = find_java_executable() + log_settings = construct_log_settings(env) + jvm_args = env.get('JVM_ARGS', '') + jvm_opts = get_jvm_opts(env) + classpath = os.pathsep.join( + [construct_flink_classpath(env), construct_hadoop_classpath(env)]) + if "FLINK_TESTING" in env: + classpath = os.pathsep.join([classpath, construct_test_classpath(env)]) + command = [java_executable, jvm_args, "-XX:+IgnoreUnrecognizedVMOptions", + "--add-opens=jdk.proxy2/jdk.proxy2=ALL-UNNAMED"] \ + + jvm_opts + log_settings \ + + ["-cp", classpath, program_args.main_class] + program_args.other_args + else: + command = [os.path.join(env["FLINK_BIN_DIR"], "flink"), "run"] + program_args.other_args \ + + ["-c", program_args.main_class] + preexec_fn = None + if not on_windows(): + def preexec_func(): + # ignore ctrl-c / SIGINT + signal.signal(signal.SIGINT, signal.SIG_IGN) + preexec_fn = preexec_func + return Popen(list(filter(lambda c: len(c) != 0, command)), + stdin=PIPE, stderr=PIPE, preexec_fn=preexec_fn, env=env) + + +if __name__ == "__main__": + launch_gateway_server_process(os.environ, sys.argv[1:]) diff --git a/flink-python/setup.py b/flink-python/setup.py new file mode 100644 index 000000000..8e788d4e5 --- /dev/null +++ b/flink-python/setup.py @@ -0,0 +1,158 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +from __future__ import print_function + +import glob +import io +import os +import sys + +from setuptools import setup +from shutil import copy, rmtree +from xml.etree import ElementTree as ET + +PACKAGE_NAME = 'apache-flink-connector-kafka' +# Source files, directories +CURRENT_DIR = os.path.abspath(os.path.dirname(__file__)) +POM_FILE = os.path.join(CURRENT_DIR, '../pom.xml') +README_FILE = os.path.join(CURRENT_DIR, 'README.txt') + +# Generated files and directories +VERSION_FILE = os.path.join(CURRENT_DIR, 'pyflink/datastream/connectors/kafka_connector_version.py') +LIB_PATH = os.path.join(CURRENT_DIR, 'pyflink/lib') +DEPENDENCY_FILE = os.path.join(CURRENT_DIR, 'dev/dev-requirements.txt') + + +# Removes a file or directory if exists. +def remove_if_exists(file_path): + if os.path.exists(file_path): + if os.path.isfile(file_path): + os.remove(file_path) + if os.path.isdir(file_path): + rmtree(file_path) + + +# Reads the content of the README.txt file. +def readme_content(): + with io.open(README_FILE, 'r', encoding='utf-8') as f: + return f.read() + + +# Reads the parameters used by the setup command. +# The source is the kafka_connector_version.py and the README.txt. +def setup_parameters(): + try: + exec(open(VERSION_FILE).read()) + return locals()['__connector_version__'], locals()['__flink_dependency__'], readme_content() + except IOError: + print("Failed to load PyFlink version file for packaging. " + + "'%s' not found!" % VERSION_FILE, + file=sys.stderr) + sys.exit(-1) + + +# Reads and parses the flink-connector-kafka main pom.xml. +# Based on the version data in the pom.xml prepares the pyflink dir: +# - Generates kafka_connector_version.py +# - Generates dev-requirements.txt +# - Copies the flink-sql-connector-kafka*.jar to the pyflink/lib dir +def prepare_pyflink_dir(): + # source files + pom_root = ET.parse(POM_FILE).getroot() + flink_version = pom_root.findall( + "./{http://maven.apache.org/POM/4.0.0}properties/" + + "{http://maven.apache.org/POM/4.0.0}flink.version" + )[0].text + connector_version = pom_root.findall( + "./{http://maven.apache.org/POM/4.0.0}version")[0].text.replace("-SNAPSHOT", ".dev0") + + flink_dependency = "apache-flink>=" + flink_version + + os.makedirs(LIB_PATH) + connector_jar = \ + glob.glob(CURRENT_DIR + '/target/test-dependencies/flink-sql-connector-kafka*.jar')[0] + copy(connector_jar, LIB_PATH) + + with io.open(VERSION_FILE, 'w', encoding='utf-8') as f: + f.write('# Generated file, do not edit\n') + f.write('__connector_version__ = "' + connector_version + '"\n') + f.write('__flink_dependency__ = "' + flink_dependency + '"\n') + + with io.open(DEPENDENCY_FILE, 'w', encoding='utf-8') as f: + f.write('# Generated file, do not edit\n') + f.write(flink_dependency + '\n') + + +# Main +print("Python version used to package: " + sys.version) + +# Python version check +if sys.version_info < (3, 7): + print("Python versions prior to 3.7 are not supported for PyFlink.", + file=sys.stderr) + sys.exit(-1) + +# Checks the running environment: +# - In the connector source root directory - package preparation +# - Otherwise - package deployment +in_flink_source = os.path.isfile("../flink-connector-kafka/src/main/" + + "java/org/apache/flink/connector/kafka/source/KafkaSource.java") + +# Cleans up the generated files and directories and regenerate them. +if in_flink_source: + remove_if_exists(VERSION_FILE) + remove_if_exists(DEPENDENCY_FILE) + remove_if_exists(LIB_PATH) + prepare_pyflink_dir() + print("\nPreparing Flink Kafka connector package") + +# Reads the current setup data from the kafka_connector_version.py file and the README.txt +(connector_version, flink_dependency, long_description) = setup_parameters() + +print("\nConnector version: " + connector_version) +print("Flink dependency: " + flink_dependency + "\n") + +if in_flink_source: + # Removes temporary directory used by the setup tool + remove_if_exists(PACKAGE_NAME.replace('-', '_') + '.egg-info') + +# Runs the python setup +setup( + name=PACKAGE_NAME, + version=connector_version, + include_package_data=True, + url='https://flink.apache.org', + license='https://www.apache.org/licenses/LICENSE-2.0', + author='Apache Software Foundation', + author_email='dev@flink.apache.org', + python_requires='>=3.8', + install_requires=[flink_dependency], + description='Apache Flink Python Kafka Connector API', + long_description=long_description, + long_description_content_type='text/plain', + zip_safe=False, + classifiers=[ + 'Development Status :: 5 - Production/Stable', + 'License :: OSI Approved :: Apache Software License', + 'Programming Language :: Python :: 3.7', + 'Programming Language :: Python :: 3.8', + 'Programming Language :: Python :: 3.9', + 'Programming Language :: Python :: 3.10'] +) + +print("\nFlink Kafka connector package is ready\n") diff --git a/flink-python/tox.ini b/flink-python/tox.ini new file mode 100644 index 000000000..c21c00f7f --- /dev/null +++ b/flink-python/tox.ini @@ -0,0 +1,51 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +[tox] +# tox (https://tox.readthedocs.io/) is a tool for running tests +# in multiple virtualenvs. This configuration file will run the +# test suite on all supported python versions. +# new environments will be excluded by default unless explicitly added to envlist. +envlist = {py38, py39, py310}-cython + +[testenv] +whitelist_externals = /bin/bash +deps = apache-flink +passenv = * +commands = + python --version + pip install pytest + bash ./dev/integration_test.sh +# Replace the default installation command with a custom retry installation script, because on high-speed +# networks, downloading a package may raise a ConnectionResetError: [Errno 104] Peer reset connection. +install_command = {toxinidir}/dev/install_command.sh {opts} {packages} + +[flake8] +# We follow PEP 8 (https://www.python.org/dev/peps/pep-0008/) with one exception: lines can be +# up to 100 characters in length, not 79. +ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 +max-line-length=100 +exclude=.tox/*,dev/*,lib/*,target/*,build/*,dist/* + +[mypy] +files=pyflink/datastream/connectors/*.py +ignore_missing_imports = True +strict_optional=False + +[mypy-pyflink.fn_execution.*] +ignore_errors = True diff --git a/pom.xml b/pom.xml index ff8efac36..d4cd25524 100644 --- a/pom.xml +++ b/pom.xml @@ -46,6 +46,7 @@ under the License. flink-connector-kafka flink-sql-connector-kafka flink-connector-kafka-e2e-tests + flink-python From 825052f55754e401176083c121ffaf38362b7a26 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Mon, 11 Dec 2023 23:08:44 +0100 Subject: [PATCH 247/322] [FLINK-33361][connectors/kafka] Add Java 17 compatibility to Flink Kafka connector --- .github/workflows/push_pr.yml | 6 +++++- flink-connector-kafka/pom.xml | 8 ++++++++ pom.xml | 7 +++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 8f53a5bde..099a181bf 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -26,10 +26,14 @@ jobs: strategy: matrix: flink: [ 1.17.1, 1.18.0 ] + jdk: [ 8, 11, 17 ] + exclude: + - jdk: 17 + flink: 1.17.1 uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} - + jdk_version: ${{ matrix.jdk }} python_test: strategy: matrix: diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 7c549fdac..470f5da2c 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -34,6 +34,14 @@ under the License. jar + + --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.locks=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED + + diff --git a/pom.xml b/pom.xml index d4cd25524..b70e98c20 100644 --- a/pom.xml +++ b/pom.xml @@ -78,6 +78,13 @@ under the License. 2.17.1 flink-connector-kafka-parent + + + + -XX:+UseG1GC -Xms256m -XX:+IgnoreUnrecognizedVMOptions ${flink.connector.module.config} From eaeb7817788a2da6fed3d9433850e10499e91852 Mon Sep 17 00:00:00 2001 From: mas-chen Date: Tue, 8 Aug 2023 18:01:40 -0700 Subject: [PATCH 248/322] [FLINK-32416] initial implementation of DynamicKafkaSource with bounded/unbounded support and unit/integration tests add dynamic kafka source docs --- .../docs/connectors/table/dynamic-kafka.md | 141 +++ .../connectors/datastream/dynamic-kafka.md | 141 +++ .../pom.xml | 4 +- .../984f05c0-ec82-405e-9bcc-d202dbe7202e | 357 +++++++ flink-connector-kafka/pom.xml | 5 + .../dynamic/metadata/ClusterMetadata.java | 92 ++ .../metadata/KafkaMetadataService.java | 53 + .../kafka/dynamic/metadata/KafkaStream.java | 94 ++ .../SingleClusterTopicMetadataService.java | 118 +++ .../dynamic/source/DynamicKafkaSource.java | 222 ++++ .../source/DynamicKafkaSourceBuilder.java | 328 ++++++ .../source/DynamicKafkaSourceOptions.java | 60 ++ .../source/GetMetadataUpdateEvent.java | 26 + .../dynamic/source/MetadataUpdateEvent.java | 77 ++ .../DynamicKafkaSourceEnumState.java | 58 ++ ...DynamicKafkaSourceEnumStateSerializer.java | 187 ++++ .../DynamicKafkaSourceEnumerator.java | 539 ++++++++++ .../StoppableKafkaEnumContextProxy.java | 297 ++++++ .../subscriber/KafkaStreamSetSubscriber.java | 42 + .../subscriber/KafkaStreamSubscriber.java | 42 + .../subscriber/StreamPatternSubscriber.java | 53 + .../metrics/KafkaClusterMetricGroup.java | 142 +++ .../KafkaClusterMetricGroupManager.java | 76 ++ .../reader/DynamicKafkaSourceReader.java | 549 ++++++++++ .../KafkaPartitionSplitReaderWrapper.java | 98 ++ .../source/split/DynamicKafkaSourceSplit.java | 85 ++ .../DynamicKafkaSourceSplitSerializer.java | 76 ++ .../kafka/source/KafkaPropertiesUtil.java | 67 ++ .../enumerator/KafkaSourceEnumState.java | 2 +- .../source/DynamicKafkaSourceITTest.java | 694 +++++++++++++ ...micKafkaSourceEnumStateSerializerTest.java | 118 +++ .../DynamicKafkaSourceEnumeratorTest.java | 964 ++++++++++++++++++ .../StoppableKafkaEnumContextProxyTest.java | 210 ++++ ...SingleClusterTopicMetadataServiceTest.java | 117 +++ .../metrics/KafkaClusterMetricGroupTest.java | 95 ++ .../reader/DynamicKafkaSourceReaderTest.java | 347 +++++++ ...DynamicKafkaSourceSplitSerializerTest.java | 47 + .../DynamicKafkaSourceExternalContext.java | 263 +++++ ...amicKafkaSourceExternalContextFactory.java | 63 ++ .../testutils/MockKafkaMetadataService.java | 93 ++ .../kafka/testutils/TwoKafkaContainers.java | 62 ++ .../testutils/YamlFileMetadataService.java | 361 +++++++ .../YamlFileMetadataServiceTest.java | 79 ++ .../kafka/DynamicKafkaSourceTestHelper.java | 229 +++++ .../connectors/kafka/KafkaTestBase.java | 89 +- .../kafka/KafkaTestEnvironment.java | 2 + .../kafka/KafkaTestEnvironmentImpl.java | 7 +- .../src/test/resources/log4j2-test.properties | 3 + .../src/test/resources/stream-metadata.yaml | 19 + pom.xml | 7 + tools/maven/checkstyle.xml | 4 - 51 files changed, 7891 insertions(+), 13 deletions(-) create mode 100644 docs/content.zh/docs/connectors/table/dynamic-kafka.md create mode 100644 docs/content/docs/connectors/datastream/dynamic-kafka.md create mode 100644 flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java create mode 100644 flink-connector-kafka/src/test/resources/stream-metadata.yaml diff --git a/docs/content.zh/docs/connectors/table/dynamic-kafka.md b/docs/content.zh/docs/connectors/table/dynamic-kafka.md new file mode 100644 index 000000000..f00a2f226 --- /dev/null +++ b/docs/content.zh/docs/connectors/table/dynamic-kafka.md @@ -0,0 +1,141 @@ +--- +title: Kafka +weight: 3 +type: docs +aliases: + - /zh/dev/connectors/dynamic-kafka.html +--- + + +# Dynamic Kafka Source _`Experimental`_ + +Flink provides an [Apache Kafka](https://kafka.apache.org) connector for reading data from and +writing data to Kafka topics from one or more Kafka clusters. This connector achieves this in a dynamic +fashion, without requiring a job restart, using a Kafka metadata service to facilitate changes in +topics and/or clusters. This is especially useful in transparent Kafka cluster addition/removal without +Flink job restart, transparent Kafka topic addition/removal without Flink job restart, and direct integration +with Hybrid Source. + +## Dependency + +For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). + +{{< connector_artifact flink-connector-kafka 3.1.0 >}} + +Flink's streaming connectors are not part of the binary distribution. +See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). + +## Dynamic Kafka Source +{{< hint info >}} +This part describes the Dynamic Kafka Source based on the new +[data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. +{{< /hint >}} + +## Usage + +Dynamic Kafka Source provides a builder class to initialize the DynamicKafkaSource. The code snippet +below shows how to build a DynamicKafkaSource to consume messages from the earliest offset of the +stream "input-stream" and deserialize only the value of the +ConsumerRecord as a string, using "MyKafkaMetadataService" to resolve the cluster(s) and topic(s) +corresponding to "input-stream". + +{{< tabs "KafkaSource" >}} +{{< tab "Java" >}} +```java + +DynamicKafkaSource source = DynamicKafkaSource.builder() + .setKafkaMetadataService(new MyKafkaMetadataService()) + .setStreamIds(Collections.singleton("input-stream")) + .setStartingOffsets(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) + .setDeserializer(new SimpleStringSchema()) + .setProperties(properties) + .build(); + +env.fromSource(source, WatermarkStrategy.noWatermarks(), "Kafka Source"); +``` +{{< /tab >}} +{{< /tabs >}} + +### Kafka Metadata Service + +An interface is provided to resolve the logical Kafka stream(s) into the corresponding physical +topic(s) and cluster(s). Typically, these implementations are based on services that align well +with internal Kafka infrastructure--if that is not available, an in-memory implementation +would also work. An example of in-memory implementation can be found in our tests. + +This source achieves its dynamic characteristic by periodically polling this Kafka metadata service +for any changes to the Kafka stream(s) and reconciling the reader tasks to subscribe to the new +Kafka metadata returned by the service. For example, in the case of a Kafka migration, the source would +swap from one cluster to the new cluster when the service makes that change in the Kafka stream metadata. + +### Additional Details + +For additional details on deserialization, event time and watermarks, idleness, consumer offset +committing, security, and more, you can refer to the Kafka Source documentation. This is possible because the +Dynamic Kafka Source leverages components of the Kafka Source, and the implementation will be +discussed in the next section. + +### Behind the Scene +{{< hint info >}} +If you are interested in how Kafka source works under the design of new data source API, you may +want to read this part as a reference. For details about the new data source API, +[documentation of data source]({{< ref "docs/dev/datastream/sources.md" >}}) and +FLIP-27 +provide more descriptive discussions. +{{< /hint >}} + + +Under the abstraction of the new data source API, Dynamic Kafka Source consists of the following components: +#### Source Split +A source split in Dynamic Kafka Source represents a partition of a Kafka topic, with cluster information. It +consists of: +* A Kafka cluster id that can be resolved by the Kafka metadata service. +* A Kafka Source Split (TopicPartition, starting offset, stopping offset). + +You can check the class `DynamicKafkaSourceSplit` for more details. + +#### Split Enumerator + +This enumerator is responsible for discovering and assigning splits from 1+ cluster. At startup, the +enumerator will discover metadata belonging to the Kafka stream ids. Using the metadata, it can +initialize KafkaSourceEnumerators to handle the functions of assigning splits to the readers. In addition, +source events will be sent to the source reader to reconcile the metadata. This enumerator has the ability to poll the +KafkaMetadataService, periodically for stream discovery. In addition, restarting enumerators when metadata changes involve +clearing outdated metrics since clusters may be removed and so should their metrics. + +#### Source Reader + +This reader is responsible for reading from 1+ clusters and using the KafkaSourceReader to fetch +records from topics and clusters based on the metadata. When new metadata is discovered by the enumerator, +the reader will reconcile metadata changes to possibly restart the KafkaSourceReader to read from the new +set of topics and clusters. + +#### Kafka Metadata Service + +This interface represents the source of truth for the current metadata for the configured Kafka stream ids. +Metadata that is removed in between polls is considered non-active (e.g. removing a cluster from the +return value, means that a cluster is non-active and should not be read from). The cluster metadata +contains an immutable Kafka cluster id, the set of topics, and properties needed to connect to the +Kafka cluster. + +#### FLIP 246 + +To understand more behind the scenes, please read [FLIP-246](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320) +for more details and discussion. diff --git a/docs/content/docs/connectors/datastream/dynamic-kafka.md b/docs/content/docs/connectors/datastream/dynamic-kafka.md new file mode 100644 index 000000000..903a26d6b --- /dev/null +++ b/docs/content/docs/connectors/datastream/dynamic-kafka.md @@ -0,0 +1,141 @@ +--- +title: Kafka +weight: 3 +type: docs +aliases: + - /dev/connectors/dynamic-kafka.html +--- + + +# Dynamic Kafka Source _`Experimental`_ + +Flink provides an [Apache Kafka](https://kafka.apache.org) connector for reading data from and +writing data to Kafka topics from one or more Kafka clusters. This connector achieves this in a dynamic +fashion, without requiring a job restart, using a Kafka metadata service to facilitate changes in +topics and/or clusters. This is especially useful in transparent Kafka cluster addition/removal without +Flink job restart, transparent Kafka topic addition/removal without Flink job restart, and direct integration +with Hybrid Source. + +## Dependency + +For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). + +{{< connector_artifact flink-connector-kafka 3.1.0 >}} + +Flink's streaming connectors are not part of the binary distribution. +See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). + +## Dynamic Kafka Source +{{< hint info >}} +This part describes the Dynamic Kafka Source based on the new +[data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. +{{< /hint >}} + +## Usage + +Dynamic Kafka Source provides a builder class to initialize the DynamicKafkaSource. The code snippet +below shows how to build a DynamicKafkaSource to consume messages from the earliest offset of the +stream "input-stream" and deserialize only the value of the +ConsumerRecord as a string, using "MyKafkaMetadataService" to resolve the cluster(s) and topic(s) +corresponding to "input-stream". + +{{< tabs "KafkaSource" >}} +{{< tab "Java" >}} +```java + +DynamicKafkaSource source = DynamicKafkaSource.builder() + .setKafkaMetadataService(new MyKafkaMetadataService()) + .setStreamIds(Collections.singleton("input-stream")) + .setStartingOffsets(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) + .setDeserializer(new SimpleStringSchema()) + .setProperties(properties) + .build(); + +env.fromSource(source, WatermarkStrategy.noWatermarks(), "Kafka Source"); +``` +{{< /tab >}} +{{< /tabs >}} + +### Kafka Metadata Service + +An interface is provided to resolve the logical Kafka stream(s) into the corresponding physical +topic(s) and cluster(s). Typically, these implementations are based on services that align well +with internal Kafka infrastructure--if that is not available, an in-memory implementation +would also work. An example of in-memory implementation can be found in our tests. + +This source achieves its dynamic characteristic by periodically polling this Kafka metadata service +for any changes to the Kafka stream(s) and reconciling the reader tasks to subscribe to the new +Kafka metadata returned by the service. For example, in the case of a Kafka migration, the source would +swap from one cluster to the new cluster when the service makes that change in the Kafka stream metadata. + +### Additional Details + +For additional details on deserialization, event time and watermarks, idleness, consumer offset +committing, security, and more, you can refer to the Kafka Source documentation. This is possible because the +Dynamic Kafka Source leverages components of the Kafka Source, and the implementation will be +discussed in the next section. + +### Behind the Scene +{{< hint info >}} +If you are interested in how Kafka source works under the design of new data source API, you may +want to read this part as a reference. For details about the new data source API, +[documentation of data source]({{< ref "docs/dev/datastream/sources.md" >}}) and +FLIP-27 +provide more descriptive discussions. +{{< /hint >}} + + +Under the abstraction of the new data source API, Dynamic Kafka Source consists of the following components: +#### Source Split +A source split in Dynamic Kafka Source represents a partition of a Kafka topic, with cluster information. It +consists of: +* A Kafka cluster id that can be resolved by the Kafka metadata service. +* A Kafka Source Split (TopicPartition, starting offset, stopping offset). + +You can check the class `DynamicKafkaSourceSplit` for more details. + +#### Split Enumerator + +This enumerator is responsible for discovering and assigning splits from 1+ cluster. At startup, the +enumerator will discover metadata belonging to the Kafka stream ids. Using the metadata, it can +initialize KafkaSourceEnumerators to handle the functions of assigning splits to the readers. In addition, +source events will be sent to the source reader to reconcile the metadata. This enumerator has the ability to poll the +KafkaMetadataService, periodically for stream discovery. In addition, restarting enumerators when metadata changes involve +clearing outdated metrics since clusters may be removed and so should their metrics. + +#### Source Reader + +This reader is responsible for reading from 1+ clusters and using the KafkaSourceReader to fetch +records from topics and clusters based on the metadata. When new metadata is discovered by the enumerator, +the reader will reconcile metadata changes to possibly restart the KafkaSourceReader to read from the new +set of topics and clusters. + +#### Kafka Metadata Service + +This interface represents the source of truth for the current metadata for the configured Kafka stream ids. +Metadata that is removed in between polls is considered non-active (e.g. removing a cluster from the +return value, means that a cluster is non-active and should not be read from). The cluster metadata +contains an immutable Kafka cluster id, the set of topics, and properties needed to connect to the +Kafka cluster. + +#### FLIP 246 + +To understand more behind the scenes, please read [FLIP-246](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320) +for more details and discussion. diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index 94227a4b4..cc0aa1bd9 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -171,7 +171,7 @@ under the License. com.google.guava guava - 32.1.2-jre + ${guava.version} @@ -250,7 +250,7 @@ under the License. com.google.guava guava - 32.1.2-jre + ${guava.version} guava.jar jar ${project.build.directory}/dependencies diff --git a/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e b/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e new file mode 100644 index 000000000..28b74b8c8 --- /dev/null +++ b/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e @@ -0,0 +1,357 @@ +Class extends class in (YamlFileMetadataService.java:0) +Class is annotated with in (KafkaSourceEnumerator.java:0) +Class is annotated with in (KafkaSourceEnumerator.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class extends class in (StreamKafkaShuffleSink.java:0) +Constructor (java.util.function.Function, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.kafka.sink.HeaderProvider)> calls method in (KafkaRecordSerializationSchemaBuilder.java:308) +Constructor (java.util.function.Function, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.kafka.sink.HeaderProvider)> calls method in (KafkaRecordSerializationSchemaBuilder.java:309) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:51) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:53) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:54) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:134) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:135) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:136) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:137) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:138) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:173) +Constructor (java.lang.String)> calls method in (KafkaWriterState.java:28) +Constructor (java.lang.Object, java.util.function.Consumer)> calls method in (Recyclable.java:31) +Constructor (java.lang.Object, java.util.function.Consumer)> calls method in (Recyclable.java:32) +Constructor (int, int, java.util.function.Function, java.util.function.Consumer)> calls method in (TransactionAborter.java:60) +Constructor (java.lang.Class)> calls constructor ()> in (YamlFileMetadataService.java:270) +Constructor (org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber, org.apache.flink.connector.kafka.source.enumerator.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, java.util.Properties, org.apache.flink.api.connector.source.Boundedness, org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState, org.apache.flink.connector.kafka.dynamic.source.enumerator.StoppableKafkaEnumContextProxy$StoppableKafkaEnumContextProxyFactory)> is annotated with in (DynamicKafkaSourceEnumerator.java:0) +Constructor (java.lang.String, org.apache.flink.connector.kafka.source.enumerator.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext)> calls constructor (java.lang.String)> in (StoppableKafkaEnumContextProxy.java:90) +Constructor (org.apache.flink.api.connector.source.SourceReaderContext, org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema, java.util.Properties)> calls constructor (int)> in (DynamicKafkaSourceReader.java:111) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, long, boolean)> calls method in (FlinkKafkaConsumerBase.java:253) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, long, boolean)> calls method in (FlinkKafkaConsumerBase.java:251) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1733) +Constructor (java.util.Set)> calls method in (FlinkKafkaProducer.java:1599) +Constructor (java.util.Set)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (java.lang.String, long, short, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1879) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1630) +Constructor (java.lang.String, org.apache.flink.streaming.util.serialization.KeyedSerializationSchema, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner)> calls constructor ()> in (FlinkKafkaProducerBase.java:120) +Constructor (java.lang.String, org.apache.flink.streaming.util.serialization.KeyedSerializationSchema, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner)> calls method in (FlinkKafkaProducerBase.java:144) +Constructor (java.lang.String, org.apache.flink.streaming.util.serialization.KeyedSerializationSchema, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner)> calls method in (FlinkKafkaProducerBase.java:146) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> calls method in (AbstractFetcher.java:584) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> calls method in (AbstractFetcher.java:586) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls constructor (org.apache.flink.api.common.eventtime.WatermarkOutput)> in (AbstractFetcher.java:154) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls method in (AbstractFetcher.java:152) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls method in (AbstractFetcher.java:156) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls method in (AbstractFetcher.java:159) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> has generic parameter type >> with type argument depending on in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor, int, int)> calls method in (AbstractPartitionDiscoverer.java:81) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaConsumerThread, org.apache.flink.streaming.connectors.kafka.internals.KafkaCommitCallback)> calls method in (KafkaConsumerThread.java:539) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:136) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:137) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:138) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:139) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:140) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:142) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean)> has generic parameter type >> with type argument depending on in (KafkaFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (KafkaFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (KafkaFetcher.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor, int, int, java.util.Properties)> calls method in (KafkaPartitionDiscoverer.java:50) +Constructor (org.apache.flink.api.common.typeutils.TypeSerializer)> is annotated with in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean, org.apache.flink.api.common.typeutils.TypeSerializer, int)> has generic parameter type >> with type argument depending on in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean, org.apache.flink.api.common.typeutils.TypeSerializer, int)> has parameter of type in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean, org.apache.flink.api.common.typeutils.TypeSerializer, int)> has parameter of type in (KafkaShuffleFetcher.java:0) +Constructor (java.util.List, java.util.regex.Pattern)> calls method in (KafkaTopicsDescriptor.java:45) +Constructor (java.util.List, java.util.regex.Pattern)> calls method in (KafkaTopicsDescriptor.java:51) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:56) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:57) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:58) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:59) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:61) +Constructor (java.lang.String, org.apache.flink.api.common.serialization.TypeInformationSerializationSchema, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties)> calls method in (FlinkKafkaShuffleConsumer.java:56) +Constructor (java.lang.String, org.apache.flink.api.common.serialization.TypeInformationSerializationSchema, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties)> calls method in (FlinkKafkaShuffleConsumer.java:59) +Constructor (java.lang.String, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties, org.apache.flink.api.java.functions.KeySelector, org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$Semantic, int)> calls method in (FlinkKafkaShuffleProducer.java:71) +Constructor (java.lang.String, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties, org.apache.flink.api.java.functions.KeySelector, org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$Semantic, int)> calls method in (FlinkKafkaShuffleProducer.java:74) +Constructor (org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffleProducer)> calls constructor (org.apache.flink.streaming.api.functions.sink.SinkFunction)> in (StreamKafkaShuffleSink.java:35) +Constructor (int, org.apache.flink.api.common.serialization.DeserializationSchema, [I, org.apache.flink.api.common.serialization.DeserializationSchema, [I, boolean, [Lorg.apache.flink.streaming.connectors.kafka.table.DynamicKafkaDeserializationSchema$MetadataConverter;, org.apache.flink.api.common.typeinfo.TypeInformation, boolean)> calls method in (DynamicKafkaDeserializationSchema.java:70) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:59) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:63) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:66) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:175) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:156) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:158) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:161) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:162) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:163) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:169) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:170) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:173) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:210) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:194) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:198) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:201) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:203) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:216) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:218) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:220) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:224) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:226) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:70) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:71) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:72) +Field has type in (YamlFileMetadataService.java:0) +Field has generic type > with type argument depending on in (KafkaClusterMetricGroupManager.java:0) +Field has type in (DynamicKafkaSourceReader.java:0) +Field has type in (FlinkKafkaConsumerBase.java:0) +Field has type in (FlinkKafkaProducerBase.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (KafkaShuffleFetcher.java:0) +Field has type in (FlinkKafkaShuffleProducer.java:0) +Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Method calls method in (FlinkKafkaInternalProducer.java:100) +Method calls method in (FlinkKafkaInternalProducer.java:109) +Method calls method in (FlinkKafkaInternalProducer.java:296) +Method calls method in (FlinkKafkaInternalProducer.java:297) +Method calls method in (FlinkKafkaInternalProducer.java:174) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:254) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:255) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:269) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:265) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:204) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:140) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:98) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:110) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:109) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:125) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:123) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:190) +Method calls method in (KafkaSerializerWrapper.java:71) +Method calls method in (KafkaSerializerWrapper.java:88) +Method is annotated with in (KafkaSink.java:0) +Method calls method in (KafkaSinkBuilder.java:194) +Method calls method in (KafkaSinkBuilder.java:202) +Method calls method in (KafkaSinkBuilder.java:198) +Method calls method in (KafkaSinkBuilder.java:111) +Method calls method in (KafkaSinkBuilder.java:97) +Method calls method in (KafkaSinkBuilder.java:123) +Method calls method in (KafkaSinkBuilder.java:133) +Method calls method in (KafkaSinkBuilder.java:152) +Method calls method in (KafkaSinkBuilder.java:151) +Method calls method in (KafkaSinkBuilder.java:175) +Method calls method in (KafkaSinkBuilder.java:176) +Method calls method in (KafkaWriter.java:244) +Method calls method in (KafkaWriter.java:245) +Method calls method in (KafkaWriter.java:246) +Method is annotated with in (KafkaWriter.java:0) +Method is annotated with in (KafkaWriter.java:0) +Method calls method in (KafkaWriter.java:311) +Method calls method in (Recyclable.java:36) +Method is annotated with in (DynamicKafkaSource.java:0) +Method calls method in (DynamicKafkaSourceBuilder.java:290) +Method calls method in (DynamicKafkaSourceBuilder.java:292) +Method calls method in (DynamicKafkaSourceBuilder.java:294) +Method calls method in (DynamicKafkaSourceBuilder.java:298) +Method calls method in (DynamicKafkaSourceBuilder.java:99) +Method calls method in (DynamicKafkaSourceBuilder.java:72) +Method calls method in (DynamicKafkaSourceBuilder.java:85) +Method calls method in (DynamicKafkaSourceOptions.java:69) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method calls method in (KafkaSourceBuilder.java:514) +Method calls method in (KafkaSourceBuilder.java:519) +Method calls method in (KafkaSourceBuilder.java:523) +Method calls method in (KafkaSourceBuilder.java:525) +Method calls method in (KafkaSourceBuilder.java:200) +Method calls method in (YamlFileMetadataService.java:279) +Method calls method in (YamlFileMetadataService.java:277) +Method checks instanceof in (YamlFileMetadataService.java:276) +Method has parameter of type in (YamlFileMetadataService.java:0) +Method calls method in (YamlFileMetadataService.java:283) +Method calls method in (YamlFileMetadataService.java:283) +Method has parameter of type in (YamlFileMetadataService.java:0) +Method calls constructor (java.lang.Class)> in (YamlFileMetadataService.java:260) +Method calls constructor (org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.constructor.BaseConstructor, org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.representer.Representer)> in (YamlFileMetadataService.java:263) +Method calls constructor ()> in (YamlFileMetadataService.java:258) +Method calls method in (YamlFileMetadataService.java:259) +Method calls method in (YamlFileMetadataService.java:261) +Method calls method in (YamlFileMetadataService.java:262) +Method gets field in (YamlFileMetadataService.java:262) +Method gets field in (YamlFileMetadataService.java:259) +Method has return type in (YamlFileMetadataService.java:0) +Method calls method in (YamlFileMetadataService.java:220) +Method is annotated with in (YamlFileMetadataService.java:0) +Method calls method in (YamlFileMetadataService.java:165) +Method calls method in (DynamicKafkaSourceEnumStateSerializer.java:140) +Method calls method in (DynamicKafkaSourceEnumerator.java:506) +Method is annotated with in (KafkaSourceEnumStateSerializer.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method calls method in (StoppableKafkaEnumContextProxy.java:225) +Method calls method in (StoppableKafkaEnumContextProxy.java:247) +Method calls method in (ReaderHandledOffsetsInitializer.java:75) +Method calls method in (SpecifiedOffsetsInitializer.java:105) +Method calls method in (KafkaClusterMetricGroupManager.java:73) +Method calls method in (KafkaClusterMetricGroupManager.java:62) +Method checks instanceof in (KafkaClusterMetricGroupManager.java:42) +Method calls constructor (int)> in (DynamicKafkaSourceReader.java:474) +Method calls method in (DynamicKafkaSourceReader.java:473) +Method calls method in (DynamicKafkaSourceReader.java:488) +Method calls method in (DynamicKafkaSourceReader.java:417) +Method has return type in (DynamicKafkaSourceReader.java:0) +Method is annotated with in (DynamicKafkaSourceReader.java:0) +Method calls method in (DynamicKafkaSourceReader.java:226) +Method is annotated with in (DynamicKafkaSourceReader.java:0) +Method calls method in (DynamicKafkaSourceReader.java:382) +Method calls method in (DynamicKafkaSourceReader.java:380) +Method calls method in (DynamicKafkaSourceReader.java:499) +Method calls method in (KafkaPartitionSplitReader.java:540) +Method calls method in (KafkaPartitionSplitReader.java:359) +Method is annotated with in (KafkaPartitionSplitReader.java:0) +Method is annotated with in (KafkaPartitionSplitReader.java:0) +Method is annotated with in (KafkaSourceReader.java:0) +Method is annotated with in (KafkaSourceReader.java:0) +Method calls method in (KafkaValueOnlyDeserializerWrapper.java:65) +Method calls constructor (java.lang.Object)> in (FlinkKafkaConsumerBase.java:306) +Method calls method in (FlinkKafkaConsumerBase.java:304) +Method calls method in (FlinkKafkaConsumerBase.java:301) +Method calls constructor (org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks)> in (FlinkKafkaConsumerBase.java:398) +Method calls method in (FlinkKafkaConsumerBase.java:397) +Method calls method in (FlinkKafkaConsumerBase.java:390) +Method has generic parameter type > with type argument depending on in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method calls constructor (org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks)> in (FlinkKafkaConsumerBase.java:352) +Method calls method in (FlinkKafkaConsumerBase.java:351) +Method calls method in (FlinkKafkaConsumerBase.java:344) +Method has generic parameter type > with type argument depending on in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method calls method in (FlinkKafkaConsumerBase.java:967) +Method has generic parameter type >> with type argument depending on in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method calls constructor (java.lang.Class, [Lorg.apache.flink.api.common.typeutils.TypeSerializer;)> in (FlinkKafkaConsumerBase.java:1225) +Method calls constructor (java.lang.Class, org.apache.flink.api.common.ExecutionConfig)> in (FlinkKafkaConsumerBase.java:1217) +Method gets field in (FlinkKafkaConsumerBase.java:1217) +Method has return type in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method calls method in (FlinkKafkaConsumerBase.java:752) +Method calls method in (FlinkKafkaConsumerBase.java:566) +Method calls method in (FlinkKafkaConsumerBase.java:535) +Method calls method in (FlinkKafkaConsumerBase.java:472) +Method calls method in (FlinkKafkaConsumerBase.java:477) +Method is annotated with in (FlinkKafkaProducerBase.java:0) +Method calls method in (FlinkKafkaProducerBase.java:404) +Method is annotated with in (FlinkKafkaProducerBase.java:0) +Method calls method in (FlinkKafkaProducerBase.java:223) +Method calls method in (FlinkKafkaProducerBase.java:260) +Method calls method in (AbstractFetcher.java:604) +Method calls method in (AbstractFetcher.java:608) +Method calls method in (AbstractFetcher.java:608) +Method calls method in (AbstractFetcher.java:593) +Method calls method in (AbstractFetcher.java:593) +Method has generic parameter type >> with type argument depending on in (AbstractFetcher.java:0) +Method has parameter of type in (AbstractFetcher.java:0) +Method calls method in (AbstractFetcher.java:430) +Method calls method in (AbstractFetcher.java:428) +Method calls method in (AbstractFetcher.java:426) +Method calls method in (AbstractFetcher.java:419) +Method has generic parameter type >> with type argument depending on in (AbstractFetcher.java:0) +Method has parameter of type in (AbstractFetcher.java:0) +Method is annotated with in (FlinkKafkaInternalProducer.java:0) +Method calls method in (FlinkKafkaInternalProducer.java:203) +Method calls method in (Handover.java:82) +Method calls method in (Handover.java:109) +Method calls method in (Handover.java:149) +Method is annotated with in (KafkaConsumerThread.java:0) +Method is annotated with in (KafkaConsumerThread.java:0) +Method calls method in (KafkaFetcher.java:223) +Method calls method in (KafkaFetcher.java:150) +Method calls constructor ([B)> in (KafkaShuffleFetcher.java:240) +Method calls method in (KafkaShuffleFetcher.java:244) +Method calls method in (KafkaShuffleFetcher.java:245) +Method calls method in (KafkaShuffleFetcher.java:254) +Method calls method in (KafkaShuffleFetcher.java:251) +Method calls method in (KafkaShuffleFetcher.java:255) +Method calls method in (KafkaShuffleFetcher.java:238) +Method gets field in (KafkaShuffleFetcher.java:244) +Method gets field in (KafkaShuffleFetcher.java:245) +Method gets field in (KafkaShuffleFetcher.java:253) +Method gets field in (KafkaShuffleFetcher.java:250) +Method gets field in (KafkaShuffleFetcher.java:254) +Method is annotated with in (KafkaShuffleFetcher.java:0) +Method calls method in (KafkaShuffleFetcher.java:279) +Method calls method in (FlinkFixedPartitioner.java:67) +Method calls method in (FlinkFixedPartitioner.java:69) +Method calls method in (FlinkFixedPartitioner.java:77) +Method calls constructor (org.apache.flink.api.dag.Transformation, java.lang.String, org.apache.flink.streaming.api.operators.StreamSink, int, boolean)> in (FlinkKafkaShuffle.java:380) +Method calls method in (FlinkKafkaShuffle.java:371) +Method calls method in (FlinkKafkaShuffle.java:383) +Method calls constructor ([I, org.apache.flink.api.common.typeinfo.TypeInformation)> in (FlinkKafkaShuffle.java:394) +Method calls method in (FlinkKafkaShuffle.java:392) +Method calls method in (FlinkKafkaShuffle.java:396) +Method calls method in (FlinkKafkaShuffle.java:151) +Method calls method in (FlinkKafkaShuffle.java:352) +Method calls method in (FlinkKafkaShuffle.java:340) +Method calls method in (FlinkKafkaShuffle.java:344) +Method calls method in (FlinkKafkaShuffle.java:256) +Method calls method in (FlinkKafkaShuffle.java:260) +Method calls method in (FlinkKafkaShuffleConsumer.java:81) +Method calls method in (FlinkKafkaShuffleConsumer.java:87) +Method calls method in (FlinkKafkaShuffleConsumer.java:80) +Method calls method in (FlinkKafkaShuffleConsumer.java:83) +Method calls method in (FlinkKafkaShuffleConsumer.java:82) +Method has generic parameter type >> with type argument depending on in (FlinkKafkaShuffleConsumer.java:0) +Method has parameter of type in (FlinkKafkaShuffleConsumer.java:0) +Method has parameter of type in (FlinkKafkaShuffleConsumer.java:0) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:186) +Method calls method in (FlinkKafkaShuffleProducer.java:205) +Method calls method in (FlinkKafkaShuffleProducer.java:204) +Method calls method in (FlinkKafkaShuffleProducer.java:190) +Method calls method in (FlinkKafkaShuffleProducer.java:193) +Method calls method in (FlinkKafkaShuffleProducer.java:195) +Method calls method in (FlinkKafkaShuffleProducer.java:196) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:212) +Method calls method in (FlinkKafkaShuffleProducer.java:225) +Method calls method in (FlinkKafkaShuffleProducer.java:224) +Method calls method in (FlinkKafkaShuffleProducer.java:216) +Method calls method in (FlinkKafkaShuffleProducer.java:217) +Method calls method in (FlinkKafkaShuffleProducer.java:218) +Method calls method in (FlinkKafkaShuffleProducer.java:219) +Method calls method in (FlinkKafkaShuffleProducer.java:160) +Method calls method in (FlinkKafkaShuffleProducer.java:99) +Method calls method in (StreamKafkaShuffleSink.java:40) +Method has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Method calls method in (KafkaConnectorOptionsUtil.java:543) +Method calls method in (KafkaConnectorOptionsUtil.java:518) +Method calls method in (KafkaConnectorOptionsUtil.java:587) +Method calls method in (KafkaConnectorOptionsUtil.java:585) +Method calls method in (KafkaConnectorOptionsUtil.java:498) +Method calls method in (KafkaDynamicSink.java:386) +Method has return type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (KafkaDynamicSink.java:0) +Method calls method in (KafkaDynamicSource.java:566) +Method calls method in (ReducingUpsertWriter.java:177) diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 470f5da2c..40d6a9f3b 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -81,6 +81,11 @@ under the License. ${kafka.version} + + com.google.guava + guava + + diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java new file mode 100644 index 000000000..1ff18930e --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; + +import com.google.common.base.MoreObjects; + +import java.io.Serializable; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; + +/** + * {@link ClusterMetadata} provides readers information about a cluster on what topics to read and + * how to connect to a cluster. + */ +@Experimental +public class ClusterMetadata implements Serializable { + private final Set topics; + private final Properties properties; + + /** + * Constructs the {@link ClusterMetadata} with the required properties. + * + * @param topics the topics belonging to a cluster. + * @param properties the properties to access a cluster. + */ + public ClusterMetadata(Set topics, Properties properties) { + this.topics = topics; + this.properties = properties; + } + + /** + * Get the topics. + * + * @return the topics. + */ + public Set getTopics() { + return topics; + } + + /** + * Get the properties. + * + * @return the properties. + */ + public Properties getProperties() { + return properties; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("topics", topics) + .add("properties", properties) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ClusterMetadata that = (ClusterMetadata) o; + return Objects.equals(topics, that.topics) && Objects.equals(properties, that.properties); + } + + @Override + public int hashCode() { + return Objects.hash(topics, properties); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java new file mode 100644 index 000000000..179277dad --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Map; +import java.util.Set; + +/** Metadata service that returns Kafka details. */ +@Experimental +public interface KafkaMetadataService extends AutoCloseable, Serializable { + /** + * Get current metadata for all streams. + * + * @return set of all streams + */ + Set getAllStreams(); + + /** + * Get current metadata for queried streams. + * + * @param streamIds stream full names + * @return map of stream name to metadata + */ + Map describeStreams(Collection streamIds); + + /** + * Check if the cluster is active. + * + * @param kafkaClusterId Kafka cluster id + * @return boolean whether the cluster is active + */ + boolean isClusterActive(String kafkaClusterId); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java new file mode 100644 index 000000000..bea9872c7 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; + +import com.google.common.base.MoreObjects; + +import java.io.Serializable; +import java.util.Map; +import java.util.Objects; + +/** + * Kafka stream represents multiple topics over multiple Kafka clusters and this class encapsulates + * all the necessary information to initiate Kafka consumers to read a stream. + */ +@Experimental +public class KafkaStream implements Serializable { + private final String streamId; + private final Map clusterMetadataMap; + + /** + * Construct a {@link KafkaStream} by passing Kafka information in order to connect to the + * stream. + * + * @param streamId the stream id. + * @param clusterMetadataMap the map of clusters to {@link ClusterMetadata} to connect to the + * stream. + */ + public KafkaStream(String streamId, Map clusterMetadataMap) { + this.streamId = streamId; + this.clusterMetadataMap = clusterMetadataMap; + } + + /** + * Get the stream id. + * + * @return the stream id. + */ + public String getStreamId() { + return streamId; + } + + /** + * Get the metadata to connect to the various cluster(s). + * + * @return the cluster metadata map. + */ + public Map getClusterMetadataMap() { + return clusterMetadataMap; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("streamId", streamId) + .add("clusterMetadataMap", clusterMetadataMap) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KafkaStream that = (KafkaStream) o; + return Objects.equals(streamId, that.streamId) + && Objects.equals(clusterMetadataMap, that.clusterMetadataMap); + } + + @Override + public int hashCode() { + return Objects.hash(streamId, clusterMetadataMap); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java new file mode 100644 index 000000000..6cef3ab31 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +/** + * A {@link KafkaMetadataService} that delegates metadata fetching to a single {@link AdminClient}, + * which is scoped to a single cluster. The stream ids are equivalent to topics. + */ +@Experimental +public class SingleClusterTopicMetadataService implements KafkaMetadataService { + + private final String kafkaClusterId; + private final Properties properties; + private transient AdminClient adminClient; + + /** + * Create a {@link SingleClusterTopicMetadataService}. + * + * @param kafkaClusterId the id of the Kafka cluster. + * @param properties the properties of the Kafka cluster. + */ + public SingleClusterTopicMetadataService(String kafkaClusterId, Properties properties) { + this.kafkaClusterId = kafkaClusterId; + this.properties = properties; + } + + /** {@inheritDoc} */ + @Override + public Set getAllStreams() { + try { + return getAdminClient().listTopics().names().get().stream() + .map(this::createKafkaStream) + .collect(Collectors.toSet()); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Fetching all streams failed", e); + } + } + + /** {@inheritDoc} */ + @Override + public Map describeStreams(Collection streamIds) { + try { + return getAdminClient().describeTopics(new ArrayList<>(streamIds)).all().get().keySet() + .stream() + .collect(Collectors.toMap(topic -> topic, this::createKafkaStream)); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Fetching all streams failed", e); + } + } + + private KafkaStream createKafkaStream(String topic) { + ClusterMetadata clusterMetadata = + new ClusterMetadata(Collections.singleton(topic), properties); + + return new KafkaStream(topic, Collections.singletonMap(kafkaClusterId, clusterMetadata)); + } + + private AdminClient getAdminClient() { + if (adminClient == null) { + Properties adminClientProps = new Properties(); + KafkaPropertiesUtil.copyProperties(properties, adminClientProps); + String clientIdPrefix = + adminClientProps.getProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key()); + adminClientProps.setProperty( + CommonClientConfigs.CLIENT_ID_CONFIG, + clientIdPrefix + "-single-cluster-topic-metadata-service"); + adminClient = AdminClient.create(adminClientProps); + } + + return adminClient; + } + + /** {@inheritDoc} */ + @Override + public boolean isClusterActive(String kafkaClusterId) { + return this.kafkaClusterId.equals(kafkaClusterId); + } + + /** {@inheritDoc} */ + @Override + public void close() { + if (adminClient != null) { + adminClient.close(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java new file mode 100644 index 000000000..9a93d7b88 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumStateSerializer; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumerator; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.reader.DynamicKafkaSourceReader; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplitSerializer; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.util.Properties; + +/** + * Factory class for the DynamicKafkaSource components. FLIP-246: DynamicKafkaSource + * + *

This source's key difference from {@link KafkaSource} is that it enables users to read + * dynamically, which does not require job restart, from streams (topics that belong to one or more + * clusters). If using {@link KafkaSource}, users need to restart the job by deleting the job and + * reconfiguring the topics and clusters. + * + *

This example shows how to configure a {@link DynamicKafkaSource} that emits Integer records: + * + *

{@code
+ * DynamicKafkaSource dynamicKafkaSource =
+ *                     DynamicKafkaSource.builder()
+ *                             .setStreamIds(Collections.singleton("MY_STREAM_ID"))
+ *                             // custom metadata service that resolves `MY_STREAM_ID` to the associated clusters and topics
+ *                             .setKafkaMetadataService(kafkaMetadataService)
+ *                             .setDeserializer(
+ *                                     KafkaRecordDeserializationSchema.valueOnly(
+ *                                             IntegerDeserializer.class))
+ *                             .setStartingOffsets(OffsetsInitializer.earliest())
+ *                             // common properties for all Kafka clusters
+ *                             .setProperties(properties)
+ *                             .build();
+ * }
+ * + *

See more configuration options in {@link DynamicKafkaSourceBuilder} and {@link + * DynamicKafkaSourceOptions}. + * + * @param Record type + */ +@Experimental +public class DynamicKafkaSource + implements Source, + ResultTypeQueryable { + + private final KafkaStreamSubscriber kafkaStreamSubscriber; + private final KafkaMetadataService kafkaMetadataService; + private final KafkaRecordDeserializationSchema deserializationSchema; + private final OffsetsInitializer startingOffsetsInitializer; + private final OffsetsInitializer stoppingOffsetsInitializer; + private final Properties properties; + private final Boundedness boundedness; + + DynamicKafkaSource( + KafkaStreamSubscriber kafkaStreamSubscriber, + KafkaMetadataService kafkaMetadataService, + KafkaRecordDeserializationSchema deserializationSchema, + OffsetsInitializer startingOffsetsInitializer, + OffsetsInitializer stoppingOffsetsInitializer, + Properties properties, + Boundedness boundedness) { + this.kafkaStreamSubscriber = kafkaStreamSubscriber; + this.deserializationSchema = deserializationSchema; + this.properties = properties; + this.kafkaMetadataService = kafkaMetadataService; + this.startingOffsetsInitializer = startingOffsetsInitializer; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + this.boundedness = boundedness; + } + + /** + * Get a builder for this source. + * + * @return a {@link DynamicKafkaSourceBuilder}. + */ + public static DynamicKafkaSourceBuilder builder() { + return new DynamicKafkaSourceBuilder<>(); + } + + /** + * Get the {@link Boundedness}. + * + * @return the {@link Boundedness}. + */ + @Override + public Boundedness getBoundedness() { + return boundedness; + } + + /** + * Create the {@link DynamicKafkaSourceReader}. + * + * @param readerContext The {@link SourceReaderContext context} for the source reader. + * @return the {@link DynamicKafkaSourceReader}. + */ + @Internal + @Override + public SourceReader createReader( + SourceReaderContext readerContext) { + return new DynamicKafkaSourceReader<>(readerContext, deserializationSchema, properties); + } + + /** + * Create the {@link DynamicKafkaSourceEnumerator}. + * + * @param enumContext The {@link SplitEnumeratorContext context} for the split enumerator. + * @return the {@link DynamicKafkaSourceEnumerator}. + */ + @Internal + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new DynamicKafkaSourceEnumerator( + kafkaStreamSubscriber, + kafkaMetadataService, + enumContext, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + properties, + boundedness, + new DynamicKafkaSourceEnumState()); + } + + /** + * Restore the {@link DynamicKafkaSourceEnumerator}. + * + * @param enumContext The {@link SplitEnumeratorContext context} for the restored split + * enumerator. + * @param checkpoint The checkpoint to restore the SplitEnumerator from. + * @return the {@link DynamicKafkaSourceEnumerator}. + */ + @Internal + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, + DynamicKafkaSourceEnumState checkpoint) { + return new DynamicKafkaSourceEnumerator( + kafkaStreamSubscriber, + kafkaMetadataService, + enumContext, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + properties, + boundedness, + checkpoint); + } + + /** + * Get the {@link DynamicKafkaSourceSplitSerializer}. + * + * @return the {@link DynamicKafkaSourceSplitSerializer}. + */ + @Internal + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new DynamicKafkaSourceSplitSerializer(); + } + + /** + * Get the {@link DynamicKafkaSourceEnumStateSerializer}. + * + * @return the {@link DynamicKafkaSourceEnumStateSerializer}. + */ + @Internal + @Override + public SimpleVersionedSerializer + getEnumeratorCheckpointSerializer() { + return new DynamicKafkaSourceEnumStateSerializer(); + } + + /** + * Get the {@link TypeInformation} of the source. + * + * @return the {@link TypeInformation}. + */ + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } + + @VisibleForTesting + public KafkaStreamSubscriber getKafkaStreamSubscriber() { + return kafkaStreamSubscriber; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java new file mode 100644 index 000000000..eab37c4ee --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSetSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.StreamPatternSubscriber; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; +import java.util.Set; +import java.util.regex.Pattern; + +/** A builder class to make it easier for users to construct a {@link DynamicKafkaSource}. */ +@Experimental +public class DynamicKafkaSourceBuilder { + private static final Logger logger = LoggerFactory.getLogger(DynamicKafkaSourceBuilder.class); + private KafkaStreamSubscriber kafkaStreamSubscriber; + private KafkaMetadataService kafkaMetadataService; + private KafkaRecordDeserializationSchema deserializationSchema; + private OffsetsInitializer startingOffsetsInitializer; + private OffsetsInitializer stoppingOffsetsInitializer; + private Boundedness boundedness; + private final Properties props; + + DynamicKafkaSourceBuilder() { + this.kafkaStreamSubscriber = null; + this.kafkaMetadataService = null; + this.deserializationSchema = null; + this.startingOffsetsInitializer = OffsetsInitializer.earliest(); + this.stoppingOffsetsInitializer = new NoStoppingOffsetsInitializer(); + this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; + this.props = new Properties(); + } + + /** + * Set the stream ids belonging to the {@link KafkaMetadataService}. + * + * @param streamIds the stream ids. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setStreamIds(Set streamIds) { + Preconditions.checkNotNull(streamIds); + ensureSubscriberIsNull("streamIds"); + this.kafkaStreamSubscriber = new KafkaStreamSetSubscriber(streamIds); + return this; + } + + /** + * Set the stream pattern to determine stream ids belonging to the {@link KafkaMetadataService}. + * + * @param streamPattern the stream pattern. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setStreamPattern(Pattern streamPattern) { + Preconditions.checkNotNull(streamPattern); + ensureSubscriberIsNull("stream pattern"); + this.kafkaStreamSubscriber = new StreamPatternSubscriber(streamPattern); + return this; + } + + /** + * Set a custom Kafka stream subscriber. + * + * @param kafkaStreamSubscriber the {@link KafkaStreamSubscriber}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setKafkaStreamSubscriber( + KafkaStreamSubscriber kafkaStreamSubscriber) { + Preconditions.checkNotNull(kafkaStreamSubscriber); + ensureSubscriberIsNull("custom"); + this.kafkaStreamSubscriber = kafkaStreamSubscriber; + return this; + } + + /** + * Set the source in bounded mode and specify what offsets to end at. This is used for all + * clusters. + * + * @param stoppingOffsetsInitializer the {@link OffsetsInitializer}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setBounded(OffsetsInitializer stoppingOffsetsInitializer) { + this.boundedness = Boundedness.BOUNDED; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + return this; + } + + /** + * Set the {@link KafkaMetadataService}. + * + * @param kafkaMetadataService the {@link KafkaMetadataService}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setKafkaMetadataService( + KafkaMetadataService kafkaMetadataService) { + this.kafkaMetadataService = kafkaMetadataService; + return this; + } + + /** + * Set the {@link KafkaRecordDeserializationSchema}. + * + * @param recordDeserializer the {@link KafkaRecordDeserializationSchema}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setDeserializer( + KafkaRecordDeserializationSchema recordDeserializer) { + this.deserializationSchema = recordDeserializer; + return this; + } + + /** + * Set the starting offsets of the stream. This will be applied to all clusters. + * + * @param startingOffsetsInitializer the {@link OffsetsInitializer}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setStartingOffsets( + OffsetsInitializer startingOffsetsInitializer) { + this.startingOffsetsInitializer = startingOffsetsInitializer; + return this; + } + + /** + * Set the properties of the consumers. This will be applied to all clusters and properties like + * {@link CommonClientConfigs#BOOTSTRAP_SERVERS_CONFIG} may be overriden by the {@link + * KafkaMetadataService}. + * + * @param properties the properties. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setProperties(Properties properties) { + this.props.putAll(properties); + return this; + } + + /** + * Set a property for the consumers. This will be applied to all clusters and properties like + * {@link CommonClientConfigs#BOOTSTRAP_SERVERS_CONFIG} may be overriden by the {@link + * KafkaMetadataService}. + * + * @param key the property key. + * @param value the properties value. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setProperty(String key, String value) { + this.props.setProperty(key, value); + return this; + } + + /** + * Set the property for {@link CommonClientConfigs#GROUP_ID_CONFIG}. This will be applied to all + * clusters. + * + * @param groupId the group id. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setGroupId(String groupId) { + return setProperty(CommonClientConfigs.GROUP_ID_CONFIG, groupId); + } + + /** + * Set the client id prefix. This applies {@link KafkaSourceOptions#CLIENT_ID_PREFIX} to all + * clusters. + * + * @param prefix the client id prefix. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setClientIdPrefix(String prefix) { + return setProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key(), prefix); + } + + /** + * Construct the source with the configuration that was set. + * + * @return the {@link DynamicKafkaSource}. + */ + public DynamicKafkaSource build() { + logger.info("Building the DynamicKafkaSource"); + sanityCheck(); + setRequiredConsumerProperties(); + return new DynamicKafkaSource<>( + kafkaStreamSubscriber, + kafkaMetadataService, + deserializationSchema, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + props, + boundedness); + } + + // Below are utility methods, code and structure are mostly copied over from KafkaSourceBuilder + + private void setRequiredConsumerProperties() { + maybeOverride( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName(), + true); + maybeOverride( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName(), + true); + if (!props.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + logger.warn( + "Offset commit on checkpoint is disabled because {} is not specified", + ConsumerConfig.GROUP_ID_CONFIG); + maybeOverride(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key(), "false", false); + } + maybeOverride(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false", false); + maybeOverride( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + startingOffsetsInitializer.getAutoOffsetResetStrategy().name().toLowerCase(), + true); + + // If the source is bounded, do not run periodic partition discovery. + maybeOverride( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), + "-1", + boundedness == Boundedness.BOUNDED); + + // If the source is bounded, do not run periodic metadata discovery + maybeOverride( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), + "-1", + boundedness == Boundedness.BOUNDED); + maybeOverride( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "0", + boundedness == Boundedness.BOUNDED); + + // If the client id prefix is not set, reuse the consumer group id as the client id prefix, + // or generate a random string if consumer group id is not specified. + maybeOverride( + KafkaSourceOptions.CLIENT_ID_PREFIX.key(), + props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) + ? props.getProperty(ConsumerConfig.GROUP_ID_CONFIG) + : "DynamicKafkaSource-" + RandomStringUtils.randomAlphabetic(8), + false); + } + + private boolean maybeOverride(String key, String value, boolean override) { + boolean overridden = false; + String userValue = props.getProperty(key); + if (userValue != null) { + if (override) { + logger.warn( + String.format( + "Property %s is provided but will be overridden from %s to %s", + key, userValue, value)); + props.setProperty(key, value); + overridden = true; + } + } else { + props.setProperty(key, value); + } + return overridden; + } + + private void sanityCheck() { + Preconditions.checkNotNull( + kafkaStreamSubscriber, "Kafka stream subscriber is required but not provided"); + Preconditions.checkNotNull( + kafkaMetadataService, "Kafka Metadata Service is required but not provided"); + Preconditions.checkNotNull( + deserializationSchema, "Deserialization schema is required but not provided."); + + // Check consumer group ID + Preconditions.checkState( + props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) || !offsetCommitEnabledManually(), + String.format( + "Property %s is required when offset commit is enabled", + ConsumerConfig.GROUP_ID_CONFIG)); + } + + private boolean offsetCommitEnabledManually() { + boolean autoCommit = + props.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) + && Boolean.parseBoolean( + props.getProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); + boolean commitOnCheckpoint = + props.containsKey(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key()) + && Boolean.parseBoolean( + props.getProperty( + KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key())); + return autoCommit || commitOnCheckpoint; + } + + private void ensureSubscriberIsNull(String attemptingSubscribeMode) { + if (kafkaStreamSubscriber != null) { + throw new IllegalStateException( + String.format( + "Cannot use %s for consumption because a %s is already set for consumption.", + attemptingSubscribeMode, + kafkaStreamSubscriber.getClass().getSimpleName())); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java new file mode 100644 index 000000000..074798ce8 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import java.util.Properties; +import java.util.function.Function; + +/** + * The connector options for {@link DynamicKafkaSource} that can be passed through the source + * properties e.g. {@link DynamicKafkaSourceBuilder#setProperties(Properties)}. + */ +@Internal +public class DynamicKafkaSourceOptions { + + private DynamicKafkaSourceOptions() {} + + public static final ConfigOption STREAM_METADATA_DISCOVERY_INTERVAL_MS = + ConfigOptions.key("stream-metadata-discovery-interval-ms") + .longType() + .defaultValue(-1L) + .withDescription( + "The interval in milliseconds for the sink to discover " + + "the changes in stream metadata. A non-positive value disables the stream metadata discovery."); + + public static final ConfigOption STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD = + ConfigOptions.key("stream-metadata-discovery-failure-threshold") + .intType() + .defaultValue(1) + .withDescription( + "The number of consecutive failures before letting the exception from Kafka metadata service discovery " + + "trigger jobmanager failure and global failover. The default is one to at least catch startup " + + "failures. This is only implemented for the source"); + + @Internal + public static T getOption( + Properties props, ConfigOption configOption, Function parser) { + String value = props.getProperty(configOption.key()); + return (T) (value == null ? configOption.defaultValue() : parser.apply(value)); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java new file mode 100644 index 000000000..4d7ff7b2d --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceEvent; + +/** Event to signal to enumerator that a reader needs to know the current metadata. */ +@Internal +public class GetMetadataUpdateEvent implements SourceEvent {} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java new file mode 100644 index 000000000..09b32d031 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.reader.DynamicKafkaSourceReader; + +import com.google.common.base.MoreObjects; + +import java.util.Objects; +import java.util.Set; + +/** + * Signals {@link DynamicKafkaSourceReader} to stop their underlying readers. The restart process is + * as follows: + * + *

1. Detect metadata change in enumerator 2. Stop sub enumerators and don't snapshot state 3. + * Send this event to all readers 4. Stop sub readers and snapshot state (offsets) 5. Start new sub + * enumerators with clean state and do total split reassignment to readers 6. Readers obtain splits, + * starting sub readers dynamically, and do reconciliation of starting offsets with the cached + * offsets + * + *

We don't snapshot enumerator state because we want to reassign previously assigned splits. + * After restart, readers need to reinitialize the sub readers by using the received splits. + */ +@Internal +public class MetadataUpdateEvent implements SourceEvent { + private final Set kafkaStreams; + + public MetadataUpdateEvent(Set kafkaStreams) { + this.kafkaStreams = kafkaStreams; + } + + public Set getKafkaStreams() { + return kafkaStreams; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("kafkaStreams", kafkaStreams).toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MetadataUpdateEvent that = (MetadataUpdateEvent) o; + return Objects.equals(kafkaStreams, that.kafkaStreams); + } + + @Override + public int hashCode() { + return Objects.hash(kafkaStreams); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java new file mode 100644 index 000000000..0dd0a7744 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * The enumerator state keeps track of the state of the sub enumerators assigned splits and + * metadata. + */ +@Internal +public class DynamicKafkaSourceEnumState { + private final Set kafkaStreams; + private final Map clusterEnumeratorStates; + + public DynamicKafkaSourceEnumState() { + this.kafkaStreams = new HashSet<>(); + this.clusterEnumeratorStates = new HashMap<>(); + } + + public DynamicKafkaSourceEnumState( + Set kafkaStreams, + Map clusterEnumeratorStates) { + this.kafkaStreams = kafkaStreams; + this.clusterEnumeratorStates = clusterEnumeratorStates; + } + + public Set getKafkaStreams() { + return kafkaStreams; + } + + public Map getClusterEnumeratorStates() { + return clusterEnumeratorStates; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java new file mode 100644 index 000000000..b34e536c4 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumStateSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.Preconditions; + +import org.apache.kafka.clients.CommonClientConfigs; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +/** (De)serializer for {@link DynamicKafkaSourceEnumState}. */ +@Internal +public class DynamicKafkaSourceEnumStateSerializer + implements SimpleVersionedSerializer { + + private static final int VERSION_1 = 1; + + private final KafkaSourceEnumStateSerializer kafkaSourceEnumStateSerializer; + + public DynamicKafkaSourceEnumStateSerializer() { + this.kafkaSourceEnumStateSerializer = new KafkaSourceEnumStateSerializer(); + } + + @Override + public int getVersion() { + return VERSION_1; + } + + @Override + public byte[] serialize(DynamicKafkaSourceEnumState state) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + + Set kafkaStreams = state.getKafkaStreams(); + serialize(kafkaStreams, out); + + Map clusterEnumeratorStates = + state.getClusterEnumeratorStates(); + out.writeInt(kafkaSourceEnumStateSerializer.getVersion()); + + // write sub enumerator states + out.writeInt(clusterEnumeratorStates.size()); + for (Map.Entry clusterEnumeratorState : + clusterEnumeratorStates.entrySet()) { + String kafkaClusterId = clusterEnumeratorState.getKey(); + out.writeUTF(kafkaClusterId); + byte[] bytes = + kafkaSourceEnumStateSerializer.serialize(clusterEnumeratorState.getValue()); + // we need to know the exact size of the byte array since + // KafkaSourceEnumStateSerializer + // will throw exception if there are leftover unread bytes in deserialization. + out.writeInt(bytes.length); + out.write(bytes); + } + + return baos.toByteArray(); + } + } + + @Override + public DynamicKafkaSourceEnumState deserialize(int version, byte[] serialized) + throws IOException { + if (version == VERSION_1) { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + Set kafkaStreams = deserialize(in); + + Map clusterEnumeratorStates = new HashMap<>(); + int kafkaSourceEnumStateSerializerVersion = in.readInt(); + + int clusterEnumeratorStateMapSize = in.readInt(); + for (int i = 0; i < clusterEnumeratorStateMapSize; i++) { + String kafkaClusterId = in.readUTF(); + int byteArraySize = in.readInt(); + KafkaSourceEnumState kafkaSourceEnumState = + kafkaSourceEnumStateSerializer.deserialize( + kafkaSourceEnumStateSerializerVersion, + readNBytes(in, byteArraySize)); + clusterEnumeratorStates.put(kafkaClusterId, kafkaSourceEnumState); + } + + return new DynamicKafkaSourceEnumState(kafkaStreams, clusterEnumeratorStates); + } + } + + throw new IOException( + String.format( + "The bytes are serialized with version %d, " + + "while this deserializer only supports version up to %d", + version, getVersion())); + } + + private void serialize(Set kafkaStreams, DataOutputStream out) throws IOException { + out.writeInt(kafkaStreams.size()); + for (KafkaStream kafkaStream : kafkaStreams) { + out.writeUTF(kafkaStream.getStreamId()); + Map clusterMetadataMap = kafkaStream.getClusterMetadataMap(); + out.writeInt(clusterMetadataMap.size()); + for (Map.Entry entry : clusterMetadataMap.entrySet()) { + String kafkaClusterId = entry.getKey(); + ClusterMetadata clusterMetadata = entry.getValue(); + out.writeUTF(kafkaClusterId); + out.writeInt(clusterMetadata.getTopics().size()); + for (String topic : clusterMetadata.getTopics()) { + out.writeUTF(topic); + } + + // only write bootstrap server for now, can extend later to serialize the complete + // properties + out.writeUTF( + Preconditions.checkNotNull( + clusterMetadata + .getProperties() + .getProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, + "Bootstrap servers must be specified in properties"))); + } + } + } + + private Set deserialize(DataInputStream in) throws IOException { + + Set kafkaStreams = new HashSet<>(); + int numStreams = in.readInt(); + for (int i = 0; i < numStreams; i++) { + String streamId = in.readUTF(); + Map clusterMetadataMap = new HashMap<>(); + int clusterMetadataMapSize = in.readInt(); + for (int j = 0; j < clusterMetadataMapSize; j++) { + String kafkaClusterId = in.readUTF(); + int topicsSize = in.readInt(); + Set topics = new HashSet<>(); + for (int k = 0; k < topicsSize; k++) { + topics.add(in.readUTF()); + } + + String bootstrapServers = in.readUTF(); + Properties properties = new Properties(); + properties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + + clusterMetadataMap.put(kafkaClusterId, new ClusterMetadata(topics, properties)); + } + + kafkaStreams.add(new KafkaStream(streamId, clusterMetadataMap)); + } + + return kafkaStreams; + } + + private static byte[] readNBytes(DataInputStream in, int size) throws IOException { + byte[] bytes = new byte[size]; + in.readFully(bytes); + return bytes; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java new file mode 100644 index 000000000..cce8ab28b --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java @@ -0,0 +1,539 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSourceOptions; +import org.apache.flink.connector.kafka.dynamic.source.GetMetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.util.Preconditions; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.MapDifference; +import com.google.common.collect.Maps; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * This enumerator manages multiple {@link KafkaSourceEnumerator}'s, which does not have any + * synchronization since it assumes single threaded execution. + */ +@Internal +public class DynamicKafkaSourceEnumerator + implements SplitEnumerator { + private static final Logger logger = + LoggerFactory.getLogger(DynamicKafkaSourceEnumerator.class); + + // Each cluster will have its own sub enumerator + private final Map> + clusterEnumeratorMap; + + // The mapping that the split enumerator context needs to be able to forward certain requests. + private final Map clusterEnumContextMap; + private final KafkaStreamSubscriber kafkaStreamSubscriber; + private final SplitEnumeratorContext enumContext; + private final KafkaMetadataService kafkaMetadataService; + private final Properties properties; + private final OffsetsInitializer startingOffsetsInitializer; + private final OffsetsInitializer stoppingOffsetInitializer; + private final Boundedness boundedness; + private final StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory + stoppableKafkaEnumContextProxyFactory; + + // options + private final long kafkaMetadataServiceDiscoveryIntervalMs; + private final int kafkaMetadataServiceDiscoveryFailureThreshold; + + // state + private int kafkaMetadataServiceDiscoveryFailureCount; + private Map> latestClusterTopicsMap; + private Set latestKafkaStreams; + + public DynamicKafkaSourceEnumerator( + KafkaStreamSubscriber kafkaStreamSubscriber, + KafkaMetadataService kafkaMetadataService, + SplitEnumeratorContext enumContext, + OffsetsInitializer startingOffsetsInitializer, + OffsetsInitializer stoppingOffsetInitializer, + Properties properties, + Boundedness boundedness, + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState) { + this( + kafkaStreamSubscriber, + kafkaMetadataService, + enumContext, + startingOffsetsInitializer, + stoppingOffsetInitializer, + properties, + boundedness, + dynamicKafkaSourceEnumState, + StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory + .getDefaultFactory()); + } + + @VisibleForTesting + DynamicKafkaSourceEnumerator( + KafkaStreamSubscriber kafkaStreamSubscriber, + KafkaMetadataService kafkaMetadataService, + SplitEnumeratorContext enumContext, + OffsetsInitializer startingOffsetsInitializer, + OffsetsInitializer stoppingOffsetInitializer, + Properties properties, + Boundedness boundedness, + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState, + StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory + stoppableKafkaEnumContextProxyFactory) { + this.kafkaStreamSubscriber = kafkaStreamSubscriber; + this.boundedness = boundedness; + + this.startingOffsetsInitializer = startingOffsetsInitializer; + this.stoppingOffsetInitializer = stoppingOffsetInitializer; + this.properties = properties; + this.enumContext = enumContext; + + // options + this.kafkaMetadataServiceDiscoveryIntervalMs = + DynamicKafkaSourceOptions.getOption( + properties, + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS, + Long::parseLong); + this.kafkaMetadataServiceDiscoveryFailureThreshold = + DynamicKafkaSourceOptions.getOption( + properties, + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD, + Integer::parseInt); + this.kafkaMetadataServiceDiscoveryFailureCount = 0; + + this.kafkaMetadataService = kafkaMetadataService; + this.stoppableKafkaEnumContextProxyFactory = stoppableKafkaEnumContextProxyFactory; + + // handle checkpoint state and rebuild contexts + this.clusterEnumeratorMap = new HashMap<>(); + this.clusterEnumContextMap = new HashMap<>(); + this.latestKafkaStreams = dynamicKafkaSourceEnumState.getKafkaStreams(); + + Map clusterProperties = new HashMap<>(); + for (KafkaStream kafkaStream : latestKafkaStreams) { + for (Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + clusterProperties.put(entry.getKey(), entry.getValue().getProperties()); + } + } + + this.latestClusterTopicsMap = new HashMap<>(); + for (Entry clusterEnumState : + dynamicKafkaSourceEnumState.getClusterEnumeratorStates().entrySet()) { + this.latestClusterTopicsMap.put( + clusterEnumState.getKey(), + clusterEnumState.getValue().assignedPartitions().stream() + .map(TopicPartition::topic) + .collect(Collectors.toSet())); + + createEnumeratorWithAssignedTopicPartitions( + clusterEnumState.getKey(), + this.latestClusterTopicsMap.get(clusterEnumState.getKey()), + clusterEnumState.getValue(), + clusterProperties.get(clusterEnumState.getKey())); + } + } + + /** + * Discover Kafka clusters and initialize sub enumerators. Bypass kafka metadata service + * discovery if there exists prior state. Exceptions with initializing Kafka source are treated + * the same as Kafka state and metadata inconsistency. + */ + @Override + public void start() { + // if there is checkpoint state, start all enumerators first. + if (!clusterEnumeratorMap.isEmpty()) { + startAllEnumerators(); + } + + if (kafkaMetadataServiceDiscoveryIntervalMs <= 0) { + enumContext.callAsync( + () -> kafkaStreamSubscriber.getSubscribedStreams(kafkaMetadataService), + this::onHandleSubscribedStreamsFetch); + } else { + enumContext.callAsync( + () -> kafkaStreamSubscriber.getSubscribedStreams(kafkaMetadataService), + this::onHandleSubscribedStreamsFetch, + 0, + kafkaMetadataServiceDiscoveryIntervalMs); + } + } + + private void handleNoMoreSplits() { + if (Boundedness.BOUNDED.equals(boundedness)) { + enumContext.runInCoordinatorThread( + () -> { + boolean allEnumeratorsHaveSignalledNoMoreSplits = true; + for (StoppableKafkaEnumContextProxy context : + clusterEnumContextMap.values()) { + allEnumeratorsHaveSignalledNoMoreSplits = + allEnumeratorsHaveSignalledNoMoreSplits + && context.isNoMoreSplits(); + } + + if (allEnumeratorsHaveSignalledNoMoreSplits) { + logger.info( + "Signal no more splits to all readers: {}", + enumContext.registeredReaders().keySet()); + enumContext + .registeredReaders() + .keySet() + .forEach(enumContext::signalNoMoreSplits); + } + }); + } + } + + // --------------- private methods for metadata discovery --------------- + + private void onHandleSubscribedStreamsFetch(Set fetchedKafkaStreams, Throwable t) { + Set handledFetchKafkaStreams = + handleFetchSubscribedStreamsError(fetchedKafkaStreams, t); + + Map> newClustersTopicsMap = new HashMap<>(); + Map clusterProperties = new HashMap<>(); + for (KafkaStream kafkaStream : handledFetchKafkaStreams) { + for (Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + String kafkaClusterId = entry.getKey(); + ClusterMetadata clusterMetadata = entry.getValue(); + + newClustersTopicsMap + .computeIfAbsent(kafkaClusterId, (unused) -> new HashSet<>()) + .addAll(clusterMetadata.getTopics()); + clusterProperties.put(kafkaClusterId, clusterMetadata.getProperties()); + } + } + + // don't do anything if no change + if (latestClusterTopicsMap.equals(newClustersTopicsMap)) { + return; + } + + if (logger.isInfoEnabled()) { + MapDifference> metadataDifference = + Maps.difference(latestClusterTopicsMap, newClustersTopicsMap); + logger.info( + "Common cluster topics after metadata refresh: {}", + metadataDifference.entriesInCommon()); + logger.info( + "Removed cluster topics after metadata refresh: {}", + metadataDifference.entriesOnlyOnLeft()); + logger.info( + "Additional cluster topics after metadata refresh: {}", + metadataDifference.entriesOnlyOnRight()); + } + + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState; + try { + dynamicKafkaSourceEnumState = snapshotState(-1); + } catch (Exception e) { + throw new RuntimeException("unable to snapshot state in metadata change", e); + } + + logger.info("Closing enumerators due to metadata change"); + + closeAllEnumeratorsAndContexts(); + latestClusterTopicsMap = newClustersTopicsMap; + latestKafkaStreams = handledFetchKafkaStreams; + sendMetadataUpdateEventToAvailableReaders(); + + // create enumerators + for (Entry> activeClusterTopics : latestClusterTopicsMap.entrySet()) { + final Set activeTopicPartitions = new HashSet<>(); + + if (dynamicKafkaSourceEnumState + .getClusterEnumeratorStates() + .get(activeClusterTopics.getKey()) + != null) { + Set oldTopicPartitions = + dynamicKafkaSourceEnumState + .getClusterEnumeratorStates() + .get(activeClusterTopics.getKey()) + .assignedPartitions(); + // filter out removed topics + for (TopicPartition oldTopicPartition : oldTopicPartitions) { + if (activeClusterTopics.getValue().contains(oldTopicPartition.topic())) { + activeTopicPartitions.add(oldTopicPartition); + } + } + } + + // restarts enumerator from state using only the active topic partitions, to avoid + // sending duplicate splits from enumerator + createEnumeratorWithAssignedTopicPartitions( + activeClusterTopics.getKey(), + activeClusterTopics.getValue(), + dynamicKafkaSourceEnumState + .getClusterEnumeratorStates() + .getOrDefault( + activeClusterTopics.getKey(), + new KafkaSourceEnumState( + Collections.emptySet(), Collections.emptySet(), false)), + clusterProperties.get(activeClusterTopics.getKey())); + } + + startAllEnumerators(); + } + + private Set handleFetchSubscribedStreamsError( + Set fetchedKafkaStreams, @Nullable Throwable t) { + if (t != null) { + if (!latestKafkaStreams.isEmpty() + && ++kafkaMetadataServiceDiscoveryFailureCount + <= kafkaMetadataServiceDiscoveryFailureThreshold) { + logger.warn("Swallowing metadata service error", t); + // reuse state + return latestKafkaStreams; + } else { + throw new RuntimeException( + "Fetching subscribed Kafka streams failed and no metadata to fallback", t); + } + } else { + // reset count in absence of failure + kafkaMetadataServiceDiscoveryFailureCount = 0; + return fetchedKafkaStreams; + } + } + + /** NOTE: Must run on coordinator thread. */ + private void sendMetadataUpdateEventToAvailableReaders() { + for (int readerId : enumContext.registeredReaders().keySet()) { + MetadataUpdateEvent metadataUpdateEvent = new MetadataUpdateEvent(latestKafkaStreams); + logger.info("sending metadata update to reader {}: {}", readerId, metadataUpdateEvent); + enumContext.sendEventToSourceReader(readerId, metadataUpdateEvent); + } + } + + /** + * Initialize KafkaEnumerators, maybe with the topic partitions that are already assigned to by + * readers, to avoid duplicate re-assignment of splits. This is especially important in the + * restart mechanism when duplicate split assignment can cause undesired starting offsets (e.g. + * not assigning to the offsets prior to reader restart). Split offset resolution is mostly + * managed by the readers. + * + *

NOTE: Must run on coordinator thread + */ + private KafkaSourceEnumerator createEnumeratorWithAssignedTopicPartitions( + String kafkaClusterId, + Set topics, + KafkaSourceEnumState kafkaSourceEnumState, + Properties fetchedProperties) { + StoppableKafkaEnumContextProxy context = + stoppableKafkaEnumContextProxyFactory.create( + enumContext, kafkaClusterId, kafkaMetadataService); + + Properties consumerProps = new Properties(); + KafkaPropertiesUtil.copyProperties(fetchedProperties, consumerProps); + KafkaPropertiesUtil.copyProperties(properties, consumerProps); + KafkaPropertiesUtil.setClientIdPrefix(consumerProps, kafkaClusterId); + + KafkaSourceEnumerator enumerator = + new KafkaSourceEnumerator( + KafkaSubscriber.getTopicListSubscriber(new ArrayList<>(topics)), + startingOffsetsInitializer, + stoppingOffsetInitializer, + consumerProps, + context, + boundedness, + kafkaSourceEnumState); + + clusterEnumContextMap.put(kafkaClusterId, context); + clusterEnumeratorMap.put(kafkaClusterId, enumerator); + + return enumerator; + } + + private void startAllEnumerators() { + for (String kafkaClusterId : latestClusterTopicsMap.keySet()) { + try { + // starts enumerators and handles split discovery and assignment + clusterEnumeratorMap.get(kafkaClusterId).start(); + } catch (KafkaException e) { + if (kafkaMetadataService.isClusterActive(kafkaClusterId)) { + throw new RuntimeException( + String.format("Failed to create enumerator for %s", kafkaClusterId), e); + } else { + logger.info( + "Found inactive cluster {} while initializing, removing enumerator", + kafkaClusterId, + e); + try { + clusterEnumContextMap.remove(kafkaClusterId).close(); + clusterEnumeratorMap.remove(kafkaClusterId).close(); + } catch (Exception ex) { + // closing enumerator throws an exception, let error propagate and restart + // the job + throw new RuntimeException( + "Failed to close enum context for " + kafkaClusterId, ex); + } + } + } + } + } + + private void closeAllEnumeratorsAndContexts() { + clusterEnumeratorMap.forEach( + (cluster, subEnumerator) -> { + try { + clusterEnumContextMap.get(cluster).close(); + subEnumerator.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + clusterEnumContextMap.clear(); + clusterEnumeratorMap.clear(); + } + + /** + * Multi cluster Kafka source readers will not request splits. Splits will be pushed to them, + * similarly for the sub enumerators. + */ + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + throw new UnsupportedOperationException("Kafka enumerators only assign splits to readers."); + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + logger.debug("Adding splits back for {}", subtaskId); + // separate splits by cluster + ArrayListMultimap kafkaPartitionSplits = + ArrayListMultimap.create(); + for (DynamicKafkaSourceSplit split : splits) { + kafkaPartitionSplits.put(split.getKafkaClusterId(), split.getKafkaPartitionSplit()); + } + + // add splits back and assign pending splits for all enumerators + for (String kafkaClusterId : kafkaPartitionSplits.keySet()) { + if (clusterEnumeratorMap.containsKey(kafkaClusterId)) { + clusterEnumeratorMap + .get(kafkaClusterId) + .addSplitsBack(kafkaPartitionSplits.get(kafkaClusterId), subtaskId); + } else { + logger.warn( + "Split refers to inactive cluster {} with current clusters being {}", + kafkaClusterId, + clusterEnumeratorMap.keySet()); + } + } + + handleNoMoreSplits(); + } + + /** NOTE: this happens at startup and failover. */ + @Override + public void addReader(int subtaskId) { + logger.debug("Adding reader {}", subtaskId); + // assign pending splits from the sub enumerator + clusterEnumeratorMap.forEach( + (cluster, subEnumerator) -> subEnumerator.addReader(subtaskId)); + handleNoMoreSplits(); + } + + /** + * Besides for checkpointing, this method is used in the restart sequence to retain the relevant + * assigned splits so that there is no reader duplicate split assignment. See {@link + * #createEnumeratorWithAssignedTopicPartitions(String, Set, KafkaSourceEnumState, Properties)}} + */ + @Override + public DynamicKafkaSourceEnumState snapshotState(long checkpointId) throws Exception { + Map subEnumeratorStateByCluster = new HashMap<>(); + + // populate map for all assigned splits + for (Entry> + clusterEnumerator : clusterEnumeratorMap.entrySet()) { + subEnumeratorStateByCluster.put( + clusterEnumerator.getKey(), + clusterEnumerator.getValue().snapshotState(checkpointId)); + } + + return new DynamicKafkaSourceEnumState(latestKafkaStreams, subEnumeratorStateByCluster); + } + + @Override + public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { + Preconditions.checkArgument( + sourceEvent instanceof GetMetadataUpdateEvent, + "Received invalid source event: " + sourceEvent); + + if (enumContext.registeredReaders().containsKey(subtaskId)) { + MetadataUpdateEvent metadataUpdateEvent = new MetadataUpdateEvent(latestKafkaStreams); + logger.info("sending metadata update to reader {}: {}", subtaskId, metadataUpdateEvent); + enumContext.sendEventToSourceReader(subtaskId, metadataUpdateEvent); + } else { + logger.warn("Got get metadata update but subtask was unavailable"); + } + } + + @Override + public void close() throws IOException { + try { + // close contexts first since they may have running tasks + for (StoppableKafkaEnumContextProxy subEnumContext : clusterEnumContextMap.values()) { + subEnumContext.close(); + } + + for (Entry> + clusterEnumerator : clusterEnumeratorMap.entrySet()) { + clusterEnumerator.getValue().close(); + } + + kafkaMetadataService.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java new file mode 100644 index 000000000..5506c4446 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java @@ -0,0 +1,297 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; + +import org.apache.kafka.common.KafkaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; + +/** + * A proxy enumerator context that supports life cycle management of underlying threads related to a + * sub {@link org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator}. This is + * motivated by the need to cancel the periodic partition discovery in scheduled tasks when sub + * Kafka Enumerators are restarted. The worker thread pool in {@link + * org.apache.flink.runtime.source.coordinator.SourceCoordinatorContext} should not contain tasks of + * inactive KafkaSourceEnumerators, after source restart. + * + *

Due to the inability to cancel scheduled tasks from {@link + * org.apache.flink.runtime.source.coordinator.SourceCoordinatorContext}, this enumerator context + * will safely catch exceptions during enumerator restart and use a closeable proxy scheduler to + * invoke tasks on the coordinator main thread to maintain the single threaded property. + */ +@Internal +public class StoppableKafkaEnumContextProxy + implements SplitEnumeratorContext, AutoCloseable { + private static final Logger logger = + LoggerFactory.getLogger(StoppableKafkaEnumContextProxy.class); + + private final String kafkaClusterId; + private final KafkaMetadataService kafkaMetadataService; + private final SplitEnumeratorContext enumContext; + private final ScheduledExecutorService subEnumeratorWorker; + private boolean noMoreSplits = false; + private volatile boolean isClosing; + + /** + * Constructor for the enumerator context. + * + * @param kafkaClusterId The Kafka cluster id in order to maintain the mapping to the sub + * KafkaSourceEnumerator + * @param kafkaMetadataService the Kafka metadata service to facilitate error handling + * @param enumContext the underlying enumerator context + */ + public StoppableKafkaEnumContextProxy( + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService, + SplitEnumeratorContext enumContext) { + this.kafkaClusterId = kafkaClusterId; + this.kafkaMetadataService = kafkaMetadataService; + this.enumContext = enumContext; + this.subEnumeratorWorker = + Executors.newScheduledThreadPool( + 1, new ExecutorThreadFactory(kafkaClusterId + "-enum-worker")); + this.isClosing = false; + } + + @Override + public SplitEnumeratorMetricGroup metricGroup() { + return enumContext.metricGroup(); + } + + @Override + public void sendEventToSourceReader(int subtaskId, SourceEvent event) { + enumContext.sendEventToSourceReader(subtaskId, event); + } + + @Override + public int currentParallelism() { + return enumContext.currentParallelism(); + } + + @Override + public Map registeredReaders() { + return enumContext.registeredReaders(); + } + + /** Wrap splits with cluster metadata. */ + @Override + public void assignSplits(SplitsAssignment newSplitAssignments) { + if (logger.isInfoEnabled()) { + logger.info( + "Assigning {} splits for cluster {}: {}", + newSplitAssignments.assignment().values().stream() + .mapToLong(Collection::size) + .sum(), + kafkaClusterId, + newSplitAssignments); + } + + Map> readerToSplitsMap = new HashMap<>(); + newSplitAssignments + .assignment() + .forEach( + (subtask, splits) -> + readerToSplitsMap.put( + subtask, + splits.stream() + .map( + split -> + new DynamicKafkaSourceSplit( + kafkaClusterId, split)) + .collect(Collectors.toList()))); + + if (!readerToSplitsMap.isEmpty()) { + enumContext.assignSplits(new SplitsAssignment<>(readerToSplitsMap)); + } + } + + @Override + public void signalNoMoreSplits(int subtask) { + // there are no more splits for this cluster + noMoreSplits = true; + } + + /** Execute the one time callables in the coordinator. */ + @Override + public void callAsync(Callable callable, BiConsumer handler) { + enumContext.callAsync( + wrapCallAsyncCallable(callable), wrapCallAsyncCallableHandler(handler)); + } + + /** + * Schedule task via internal thread pool to proxy task so that the task handler callback can + * execute in the single threaded source coordinator thread pool to avoid synchronization needs. + * + *

Having the scheduled task in the internal thread pool also allows us to cancel the task + * when the context needs to close due to dynamic enumerator restart. + * + *

In the case of KafkaEnumerator partition discovery, the callback modifies KafkaEnumerator + * object state. + */ + @Override + public void callAsync( + Callable callable, + BiConsumer handler, + long initialDelay, + long period) { + subEnumeratorWorker.scheduleAtFixedRate( + () -> callAsync(callable, handler), initialDelay, period, TimeUnit.MILLISECONDS); + } + + @Override + public void runInCoordinatorThread(Runnable runnable) { + enumContext.runInCoordinatorThread(runnable); + } + + public boolean isNoMoreSplits() { + return noMoreSplits; + } + + /** + * Note that we can't close the source coordinator here, because these contexts can be closed + * during metadata change when the coordinator still needs to continue to run. We can only close + * the coordinator context in Flink job shutdown, which Flink will do for us. That's why there + * is the complexity of the internal thread pools in this class. + * + *

TODO: Attach Flink JIRA ticket -- discuss with upstream how to cancel scheduled tasks + * belonging to enumerator. + */ + @Override + public void close() throws Exception { + logger.info("Closing enum context for {}", kafkaClusterId); + if (subEnumeratorWorker != null) { + // KafkaSubscriber worker thread will fail if admin client is closed in the middle. + // Swallow the error and set the context to closed state. + isClosing = true; + subEnumeratorWorker.shutdown(); + subEnumeratorWorker.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS); + } + } + + /** + * Wraps callable in call async executed in worker thread pool with exception propagation to + * optimize on doing IO in non-coordinator thread. + */ + protected Callable wrapCallAsyncCallable(Callable callable) { + return () -> { + try { + return callable.call(); + } catch (Exception e) { + if (isClosing) { + throw new HandledFlinkKafkaException(e, kafkaClusterId); + } + + Optional throwable = + ExceptionUtils.findThrowable(e, KafkaException.class); + // check if Kafka related and if Kafka cluster is inactive + if (throwable.isPresent() + && !kafkaMetadataService.isClusterActive(kafkaClusterId)) { + throw new HandledFlinkKafkaException(throwable.get(), kafkaClusterId); + } + + throw e; + } + }; + } + + /** + * Handle exception that is propagated by a callable, executed on coordinator thread. Depending + * on condition(s) the exception may be swallowed or forwarded. This is the Kafka topic + * partition discovery callable handler. + */ + protected BiConsumer wrapCallAsyncCallableHandler( + BiConsumer mainHandler) { + return (result, t) -> { + // check if exception is handled + Optional throwable = + ExceptionUtils.findThrowable(t, HandledFlinkKafkaException.class); + if (throwable.isPresent()) { + logger.warn("Swallowed handled exception for {}.", kafkaClusterId, throwable.get()); + return; + } + + // let the main handler deal with the potential exception + mainHandler.accept(result, t); + }; + } + + /** + * General exception to signal to internal exception handling mechanisms that a benign error + * occurred. + */ + @Internal + public static class HandledFlinkKafkaException extends RuntimeException { + private static final String ERROR_MESSAGE = "An error occurred with %s"; + + private final String kafkaClusterId; + + public HandledFlinkKafkaException(Throwable cause, String kafkaClusterId) { + super(cause); + this.kafkaClusterId = kafkaClusterId; + } + + public String getMessage() { + return String.format(ERROR_MESSAGE, kafkaClusterId); + } + } + + /** + * This factory exposes a way to override the {@link StoppableKafkaEnumContextProxy} used in the + * enumerator. This pluggable factory is extended in unit tests to facilitate invoking the + * periodic discovery loops on demand. + */ + @Internal + public interface StoppableKafkaEnumContextProxyFactory { + + StoppableKafkaEnumContextProxy create( + SplitEnumeratorContext enumContext, + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService); + + static StoppableKafkaEnumContextProxyFactory getDefaultFactory() { + return (enumContext, kafkaClusterId, kafkaMetadataService) -> + new StoppableKafkaEnumContextProxy( + kafkaClusterId, kafkaMetadataService, enumContext); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java new file mode 100644 index 000000000..01dabf59c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import java.util.HashSet; +import java.util.Set; + +/** Subscribe to streams based on the set of ids. */ +@Internal +public class KafkaStreamSetSubscriber implements KafkaStreamSubscriber { + + private final Set streamIds; + + public KafkaStreamSetSubscriber(Set streamIds) { + this.streamIds = streamIds; + } + + @Override + public Set getSubscribedStreams(KafkaMetadataService kafkaMetadataService) { + return new HashSet<>(kafkaMetadataService.describeStreams(streamIds).values()); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java new file mode 100644 index 000000000..bbd4e7f6c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import java.io.Serializable; +import java.util.Set; + +/** + * The subscriber interacts with {@link KafkaMetadataService} to find which Kafka streams the source + * will subscribe to. + */ +@Experimental +public interface KafkaStreamSubscriber extends Serializable { + + /** + * Get the subscribed {@link KafkaStream}s. + * + * @param kafkaMetadataService the {@link KafkaMetadataService}. + * @return the subscribed {@link KafkaStream}s. + */ + Set getSubscribedStreams(KafkaMetadataService kafkaMetadataService); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java new file mode 100644 index 000000000..d95f7e0f0 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import com.google.common.collect.ImmutableSet; + +import java.util.Set; +import java.util.regex.Pattern; + +/** To subscribe to streams based on a pattern. */ +@Internal +public class StreamPatternSubscriber implements KafkaStreamSubscriber { + + private final Pattern streamPattern; + + public StreamPatternSubscriber(Pattern streamPattern) { + this.streamPattern = streamPattern; + } + + @Override + public Set getSubscribedStreams(KafkaMetadataService kafkaMetadataService) { + Set allStreams = kafkaMetadataService.getAllStreams(); + ImmutableSet.Builder builder = ImmutableSet.builder(); + for (KafkaStream kafkaStream : allStreams) { + String streamId = kafkaStream.getStreamId(); + if (streamPattern.matcher(streamId).find()) { + builder.add(kafkaStream); + } + } + + return builder.build(); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java new file mode 100644 index 000000000..451a05b3a --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.metrics; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SourceReaderMetricGroup; +import org.apache.flink.runtime.metrics.MetricNames; + +import java.util.Map; + +/** + * A custom proxy metric group in order to group {@link + * org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics} by Kafka cluster. + * + *

Reports pending records per cluster under DynamicKafkaSource metric group, motivated by + * standardized connector metrics: + * https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics. + */ +@Internal +public class KafkaClusterMetricGroup implements SourceReaderMetricGroup { + + public static final String DYNAMIC_KAFKA_SOURCE_METRIC_GROUP = "DynamicKafkaSource"; + public static final String KAFKA_CLUSTER_GROUP_NAME = "kafkaCluster"; + + private final MetricGroup internalClusterSpecificMetricGroup; + private final OperatorIOMetricGroup delegateIOMetricGroup; + + public KafkaClusterMetricGroup( + MetricGroup dynamicKafkaSourceMetricGroup, + SourceReaderMetricGroup delegateSourceReaderMetricGroup, + String kafkaClusterId) { + this.internalClusterSpecificMetricGroup = + dynamicKafkaSourceMetricGroup.addGroup(KAFKA_CLUSTER_GROUP_NAME, kafkaClusterId); + this.delegateIOMetricGroup = delegateSourceReaderMetricGroup.getIOMetricGroup(); + } + + MetricGroup getInternalClusterSpecificMetricGroup() { + return internalClusterSpecificMetricGroup; + } + + @Override + public Counter getNumRecordsInErrorsCounter() { + throw new UnsupportedOperationException( + "This is not invoked/supported by KafkaSourceReader as of Flink 1.14."); + } + + @Override + public void setPendingBytesGauge(Gauge gauge) { + throw new UnsupportedOperationException( + "This is not invoked/supported by KafkaSourceReader as of Flink 1.14."); + } + + @Override + public void setPendingRecordsGauge(Gauge pendingRecordsGauge) { + gauge(MetricNames.PENDING_RECORDS, pendingRecordsGauge); + } + + @Override + public OperatorIOMetricGroup getIOMetricGroup() { + return delegateIOMetricGroup; + } + + // --------------- MetricGroup methods --------------- + + @Override + public Counter counter(String name) { + return internalClusterSpecificMetricGroup.counter(name); + } + + @Override + public C counter(String name, C counter) { + return internalClusterSpecificMetricGroup.counter(name, counter); + } + + @Override + public > G gauge(String name, G gauge) { + return internalClusterSpecificMetricGroup.gauge(name, gauge); + } + + @Override + public H histogram(String name, H histogram) { + return internalClusterSpecificMetricGroup.histogram(name, histogram); + } + + @Override + public M meter(String name, M meter) { + return internalClusterSpecificMetricGroup.meter(name, meter); + } + + @Override + public MetricGroup addGroup(String name) { + return internalClusterSpecificMetricGroup.addGroup(name); + } + + @Override + public MetricGroup addGroup(String key, String value) { + return internalClusterSpecificMetricGroup.addGroup(key, value); + } + + @Override + public String[] getScopeComponents() { + return internalClusterSpecificMetricGroup.getScopeComponents(); + } + + @Override + public Map getAllVariables() { + return internalClusterSpecificMetricGroup.getAllVariables(); + } + + @Override + public String getMetricIdentifier(String metricName) { + return internalClusterSpecificMetricGroup.getMetricIdentifier(metricName); + } + + @Override + public String getMetricIdentifier(String metricName, CharacterFilter filter) { + return internalClusterSpecificMetricGroup.getMetricIdentifier(metricName, filter); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java new file mode 100644 index 000000000..0659f528e --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.metrics; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +/** Manages metric groups for each cluster. */ +@Internal +public class KafkaClusterMetricGroupManager implements AutoCloseable { + private static final Logger logger = + LoggerFactory.getLogger(KafkaClusterMetricGroupManager.class); + private final Map metricGroups; + + public KafkaClusterMetricGroupManager() { + this.metricGroups = new HashMap<>(); + } + + public void register(String kafkaClusterId, KafkaClusterMetricGroup kafkaClusterMetricGroup) { + if (kafkaClusterMetricGroup.getInternalClusterSpecificMetricGroup() + instanceof AbstractMetricGroup) { + metricGroups.put( + kafkaClusterId, + (AbstractMetricGroup) + kafkaClusterMetricGroup.getInternalClusterSpecificMetricGroup()); + } else { + logger.warn( + "MetricGroup {} is an instance of {}, which is not supported. Please use an implementation of AbstractMetricGroup.", + kafkaClusterMetricGroup.getInternalClusterSpecificMetricGroup(), + kafkaClusterMetricGroup + .getInternalClusterSpecificMetricGroup() + .getClass() + .getSimpleName()); + } + } + + public void close(String kafkaClusterId) { + AbstractMetricGroup metricGroup = metricGroups.remove(kafkaClusterId); + if (metricGroup != null) { + metricGroup.close(); + } else { + logger.warn( + "Tried to close metric group for {} but it is not registered for lifecycle management", + kafkaClusterId); + } + } + + @Override + public void close() throws Exception { + for (AbstractMetricGroup metricGroup : metricGroups.values()) { + metricGroup.close(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java new file mode 100644 index 000000000..f7193418d --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java @@ -0,0 +1,549 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.GetMetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup; +import org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroupManager; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; +import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics; +import org.apache.flink.connector.kafka.source.reader.KafkaRecordEmitter; +import org.apache.flink.connector.kafka.source.reader.KafkaSourceReader; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.source.reader.fetcher.KafkaSourceFetcherManager; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.runtime.io.MultipleFuturesAvailabilityHelper; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.UserCodeClassLoader; + +import com.google.common.collect.ArrayListMultimap; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Properties; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +/** + * Manages state about underlying {@link KafkaSourceReader} to collect records and commit offsets + * from multiple Kafka clusters. This reader also handles changes to Kafka topology by reacting to + * restart sequence initiated by the enumerator and suspending inconsistent sub readers. + * + *

First, in the restart sequence, we will receive the {@link MetadataUpdateEvent} from the + * enumerator, stop all KafkaSourceReaders, and retain the relevant splits. Second, enumerator will + * send all new splits that readers should work on (old splits will not be sent again). + */ +@Internal +public class DynamicKafkaSourceReader implements SourceReader { + private static final Logger logger = LoggerFactory.getLogger(DynamicKafkaSourceReader.class); + private final KafkaRecordDeserializationSchema deserializationSchema; + private final Properties properties; + private final MetricGroup dynamicKafkaSourceMetricGroup; + private final Gauge kafkaClusterCount; + private final SourceReaderContext readerContext; + private final KafkaClusterMetricGroupManager kafkaClusterMetricGroupManager; + + // needs have a strict ordering for readers to guarantee availability future consistency + private final NavigableMap> clusterReaderMap; + private final Map clustersProperties; + private final List pendingSplits; + + private MultipleFuturesAvailabilityHelper availabilityHelper; + private boolean isActivelyConsumingSplits; + private boolean isNoMoreSplits; + private AtomicBoolean restartingReaders; + + public DynamicKafkaSourceReader( + SourceReaderContext readerContext, + KafkaRecordDeserializationSchema deserializationSchema, + Properties properties) { + this.readerContext = readerContext; + this.clusterReaderMap = new TreeMap<>(); + this.deserializationSchema = deserializationSchema; + this.properties = properties; + this.kafkaClusterCount = clusterReaderMap::size; + this.dynamicKafkaSourceMetricGroup = + readerContext + .metricGroup() + .addGroup(KafkaClusterMetricGroup.DYNAMIC_KAFKA_SOURCE_METRIC_GROUP); + this.kafkaClusterMetricGroupManager = new KafkaClusterMetricGroupManager(); + this.pendingSplits = new ArrayList<>(); + this.availabilityHelper = new MultipleFuturesAvailabilityHelper(0); + this.isNoMoreSplits = false; + this.isActivelyConsumingSplits = false; + this.restartingReaders = new AtomicBoolean(); + this.clustersProperties = new HashMap<>(); + } + + /** + * This is invoked first only in reader startup without state. In stateful startup, splits are + * added before this method is invoked. + */ + @Override + public void start() { + logger.trace("Starting reader for subtask index={}", readerContext.getIndexOfSubtask()); + // metrics cannot be registered in the enumerator + readerContext.metricGroup().gauge("kafkaClusterCount", kafkaClusterCount); + readerContext.sendSourceEventToCoordinator(new GetMetadataUpdateEvent()); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + // do not return end of input if no more splits has not yet been signaled + if (!isNoMoreSplits && clusterReaderMap.isEmpty()) { + return logAndReturnInputStatus(InputStatus.NOTHING_AVAILABLE); + } + + if (restartingReaders.get()) { + logger.info("Poll next invoked while restarting readers"); + return logAndReturnInputStatus(InputStatus.NOTHING_AVAILABLE); + } + + boolean isMoreAvailable = false; + boolean isNothingAvailable = false; + + for (Map.Entry> clusterReader : clusterReaderMap.entrySet()) { + InputStatus inputStatus = clusterReader.getValue().pollNext(readerOutput); + switch (inputStatus) { + case MORE_AVAILABLE: + isMoreAvailable = true; + break; + case NOTHING_AVAILABLE: + isNothingAvailable = true; + break; + } + } + + return logAndReturnInputStatus(consolidateInputStatus(isMoreAvailable, isNothingAvailable)); + } + + private InputStatus consolidateInputStatus( + boolean atLeastOneMoreAvailable, boolean atLeastOneNothingAvailable) { + final InputStatus inputStatus; + if (atLeastOneMoreAvailable) { + inputStatus = InputStatus.MORE_AVAILABLE; + } else if (atLeastOneNothingAvailable) { + inputStatus = InputStatus.NOTHING_AVAILABLE; + } else { + inputStatus = InputStatus.END_OF_INPUT; + } + return inputStatus; + } + + // we also need to filter splits at startup in case checkpoint is not consistent bwtn enumerator + // and reader + @Override + public void addSplits(List splits) { + logger.info("Adding splits to reader {}: {}", readerContext.getIndexOfSubtask(), splits); + // at startup, don't add splits until we get confirmation from enumerator of the current + // metadata + if (!isActivelyConsumingSplits) { + pendingSplits.addAll(splits); + return; + } + + ArrayListMultimap clusterSplitsMap = + ArrayListMultimap.create(); + for (DynamicKafkaSourceSplit split : splits) { + clusterSplitsMap.put(split.getKafkaClusterId(), split); + } + + Set kafkaClusterIds = clusterSplitsMap.keySet(); + + boolean newCluster = false; + for (String kafkaClusterId : kafkaClusterIds) { + // if a reader corresponding to the split doesn't exist, create it + // it is possible that the splits come before the source event + if (!clusterReaderMap.containsKey(kafkaClusterId)) { + try { + KafkaSourceReader kafkaSourceReader = createReader(kafkaClusterId); + clusterReaderMap.put(kafkaClusterId, kafkaSourceReader); + kafkaSourceReader.start(); + newCluster = true; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + // add splits + KafkaSourceReader reader = clusterReaderMap.get(kafkaClusterId); + reader.addSplits(clusterSplitsMap.get(kafkaClusterId)); + } + + // reset the availability future to also depend on the new sub readers + if (newCluster) { + completeAndResetAvailabilityHelper(); + } + } + + /** + * Duplicate source events are handled with idempotency. No metadata change means we simply skip + * the restart logic. + */ + @Override + public void handleSourceEvents(SourceEvent sourceEvent) { + Preconditions.checkArgument( + sourceEvent instanceof MetadataUpdateEvent, + "Received invalid source event: " + sourceEvent); + + logger.info( + "Received source event {}: subtask={}", + sourceEvent, + readerContext.getIndexOfSubtask()); + Set newKafkaStreams = ((MetadataUpdateEvent) sourceEvent).getKafkaStreams(); + Map> newClustersAndTopics = new HashMap<>(); + Map newClustersProperties = new HashMap<>(); + for (KafkaStream kafkaStream : newKafkaStreams) { + for (Map.Entry clusterMetadataMapEntry : + kafkaStream.getClusterMetadataMap().entrySet()) { + newClustersAndTopics + .computeIfAbsent( + clusterMetadataMapEntry.getKey(), (unused) -> new HashSet<>()) + .addAll(clusterMetadataMapEntry.getValue().getTopics()); + + newClustersProperties.put( + clusterMetadataMapEntry.getKey(), + clusterMetadataMapEntry.getValue().getProperties()); + } + } + + // filter current splits with the metadata update + List currentSplitState = snapshotStateFromAllReaders(-1); + logger.info( + "Snapshotting split state for reader {}: {}", + readerContext.getIndexOfSubtask(), + currentSplitState); + Map> currentMetadataFromState = new HashMap<>(); + Map> filteredNewClusterSplitStateMap = new HashMap<>(); + + // the data structures above + for (DynamicKafkaSourceSplit split : currentSplitState) { + currentMetadataFromState + .computeIfAbsent(split.getKafkaClusterId(), (ignore) -> new HashSet<>()) + .add(split.getKafkaPartitionSplit().getTopic()); + // check if cluster topic exists in the metadata update + if (newClustersAndTopics.containsKey(split.getKafkaClusterId()) + && newClustersAndTopics + .get(split.getKafkaClusterId()) + .contains(split.getKafkaPartitionSplit().getTopic())) { + filteredNewClusterSplitStateMap + .computeIfAbsent(split.getKafkaClusterId(), (ignore) -> new ArrayList<>()) + .add(split); + } else { + logger.info("Skipping outdated split due to metadata changes: {}", split); + } + } + + // only restart if there was metadata change to handle duplicate MetadataUpdateEvent from + // enumerator. We can possibly only restart the readers whose metadata has changed but that + // comes at the cost of complexity and it is an optimization for a corner case. We can + // revisit if necessary. + if (!newClustersAndTopics.equals(currentMetadataFromState)) { + restartingReaders.set(true); + closeAllReadersAndClearState(); + + clustersProperties.putAll(newClustersProperties); + for (String kafkaClusterId : newClustersAndTopics.keySet()) { + try { + // restart kafka source readers with the relevant state + KafkaSourceReader kafkaSourceReader = createReader(kafkaClusterId); + clusterReaderMap.put(kafkaClusterId, kafkaSourceReader); + if (filteredNewClusterSplitStateMap.containsKey(kafkaClusterId)) { + kafkaSourceReader.addSplits( + filteredNewClusterSplitStateMap.get(kafkaClusterId)); + } + kafkaSourceReader.start(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + // reset the availability future to also depend on the new sub readers + completeAndResetAvailabilityHelper(); + } else { + // update properties even on no metadata change + clustersProperties.clear(); + clustersProperties.putAll(newClustersProperties); + } + + // finally mark the reader as active, if not already and add pending splits + if (!isActivelyConsumingSplits) { + isActivelyConsumingSplits = true; + } + + if (!pendingSplits.isEmpty()) { + List validPendingSplits = + pendingSplits.stream() + // Pending splits is used to cache splits at startup, before metadata + // update event arrives. Splits in state could be old and it's possible + // to not have another metadata update event, so need to filter the + // splits at this point. + .filter( + pendingSplit -> { + boolean splitValid = + isSplitForActiveClusters( + pendingSplit, newClustersAndTopics); + if (!splitValid) { + logger.info( + "Removing invalid split for reader: {}", + pendingSplit); + } + return splitValid; + }) + .collect(Collectors.toList()); + + addSplits(validPendingSplits); + pendingSplits.clear(); + if (isNoMoreSplits) { + notifyNoMoreSplits(); + } + } + } + + private static boolean isSplitForActiveClusters( + DynamicKafkaSourceSplit split, Map> metadata) { + return metadata.containsKey(split.getKafkaClusterId()) + && metadata.get(split.getKafkaClusterId()) + .contains(split.getKafkaPartitionSplit().getTopic()); + } + + @Override + public List snapshotState(long checkpointId) { + List splits = snapshotStateFromAllReaders(checkpointId); + + // pending splits should be typically empty, since we do not add splits to pending splits if + // reader has started + splits.addAll(pendingSplits); + return splits; + } + + private List snapshotStateFromAllReaders(long checkpointId) { + List splits = new ArrayList<>(); + for (Map.Entry> clusterReader : clusterReaderMap.entrySet()) { + clusterReader + .getValue() + .snapshotState(checkpointId) + .forEach( + kafkaPartitionSplit -> + splits.add( + new DynamicKafkaSourceSplit( + clusterReader.getKey(), kafkaPartitionSplit))); + } + + return splits; + } + + @Override + public CompletableFuture isAvailable() { + availabilityHelper.resetToUnAvailable(); + syncAvailabilityHelperWithReaders(); + return (CompletableFuture) availabilityHelper.getAvailableFuture(); + } + + @Override + public void notifyNoMoreSplits() { + logger.info("notify no more splits for reader {}", readerContext.getIndexOfSubtask()); + if (pendingSplits.isEmpty()) { + clusterReaderMap.values().forEach(KafkaSourceReader::notifyNoMoreSplits); + } + + isNoMoreSplits = true; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + logger.debug("Notify checkpoint complete for {}", clusterReaderMap.keySet()); + for (KafkaSourceReader subReader : clusterReaderMap.values()) { + subReader.notifyCheckpointComplete(checkpointId); + } + } + + @Override + public void close() throws Exception { + for (KafkaSourceReader subReader : clusterReaderMap.values()) { + subReader.close(); + } + kafkaClusterMetricGroupManager.close(); + } + + private KafkaSourceReader createReader(String kafkaClusterId) throws Exception { + FutureCompletingBlockingQueue>> + elementsQueue = new FutureCompletingBlockingQueue<>(); + Properties readerSpecificProperties = new Properties(); + KafkaPropertiesUtil.copyProperties(properties, readerSpecificProperties); + KafkaPropertiesUtil.copyProperties( + Preconditions.checkNotNull( + clustersProperties.get(kafkaClusterId), + "Properties for cluster %s is not found. Current Kafka cluster ids: %s", + kafkaClusterId, + clustersProperties.keySet()), + readerSpecificProperties); + KafkaPropertiesUtil.setClientIdPrefix(readerSpecificProperties, kafkaClusterId); + + // layer a kafka cluster group to distinguish metrics by cluster + KafkaClusterMetricGroup kafkaClusterMetricGroup = + new KafkaClusterMetricGroup( + dynamicKafkaSourceMetricGroup, readerContext.metricGroup(), kafkaClusterId); + kafkaClusterMetricGroupManager.register(kafkaClusterId, kafkaClusterMetricGroup); + KafkaSourceReaderMetrics kafkaSourceReaderMetrics = + new KafkaSourceReaderMetrics(kafkaClusterMetricGroup); + + deserializationSchema.open( + new DeserializationSchema.InitializationContext() { + @Override + public MetricGroup getMetricGroup() { + // adding kafkaClusterMetricGroup instead of the sourceReaderMetricGroup + // since there could be metric collision, so `kafkaCluster` group is + // necessary to + // distinguish between instances of this metric + return kafkaClusterMetricGroup.addGroup("deserializer"); + } + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + return readerContext.getUserCodeClassLoader(); + } + }); + + KafkaRecordEmitter recordEmitter = new KafkaRecordEmitter<>(deserializationSchema); + return new KafkaSourceReader<>( + elementsQueue, + new KafkaSourceFetcherManager( + elementsQueue, + () -> + new KafkaPartitionSplitReaderWrapper( + readerSpecificProperties, + readerContext, + kafkaSourceReaderMetrics, + kafkaClusterId), + (ignore) -> {}), + recordEmitter, + toConfiguration(readerSpecificProperties), + readerContext, + kafkaSourceReaderMetrics); + } + + /** + * In metadata change, we need to reset the availability helper since the number of Kafka source + * readers could have changed. + */ + private void completeAndResetAvailabilityHelper() { + CompletableFuture cachedPreviousFuture = availabilityHelper.getAvailableFuture(); + availabilityHelper = new MultipleFuturesAvailabilityHelper(clusterReaderMap.size()); + syncAvailabilityHelperWithReaders(); + + // We cannot immediately complete the previous future here. We must complete it only when + // the new readers have finished handling the split assignment. Completing the future too + // early can cause WakeupException (implicitly woken up by invocation to pollNext()) if the + // reader has not finished resolving the positions of the splits, as seen in flaky unit test + // errors. There is no error handling for WakeupException in SplitReader's + // handleSplitChanges. + availabilityHelper + .getAvailableFuture() + .whenComplete( + (ignore, t) -> { + restartingReaders.set(false); + cachedPreviousFuture.complete(null); + }); + } + + private void syncAvailabilityHelperWithReaders() { + int i = 0; + for (String kafkaClusterId : clusterReaderMap.navigableKeySet()) { + availabilityHelper.anyOf(i, clusterReaderMap.get(kafkaClusterId).isAvailable()); + i++; + } + } + + private void closeAllReadersAndClearState() { + for (Map.Entry> entry : clusterReaderMap.entrySet()) { + try { + logger.info( + "Closing sub reader in reader {} for cluster: {}", + readerContext.getIndexOfSubtask(), + entry.getKey()); + entry.getValue().close(); + kafkaClusterMetricGroupManager.close(entry.getKey()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + clusterReaderMap.clear(); + clustersProperties.clear(); + } + + static Configuration toConfiguration(Properties props) { + Configuration config = new Configuration(); + props.stringPropertyNames().forEach(key -> config.setString(key, props.getProperty(key))); + return config; + } + + private InputStatus logAndReturnInputStatus(InputStatus inputStatus) { + if (InputStatus.END_OF_INPUT.equals(inputStatus)) { + logger.info( + "inputStatus={}, subtaskIndex={}", + inputStatus, + readerContext.getIndexOfSubtask()); + } else { + logger.trace( + "inputStatus={}, subtaskIndex={}", + inputStatus, + readerContext.getIndexOfSubtask()); + } + + return inputStatus; + } + + @VisibleForTesting + public MultipleFuturesAvailabilityHelper getAvailabilityHelper() { + return availabilityHelper; + } + + @VisibleForTesting + public boolean isActivelyConsumingSplits() { + return isActivelyConsumingSplits; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java new file mode 100644 index 000000000..cc250eace --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics; +import org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** This extends to Kafka Partition Split Reader to wrap split ids with the cluster name. */ +@Internal +public class KafkaPartitionSplitReaderWrapper extends KafkaPartitionSplitReader + implements AutoCloseable { + private final String kafkaClusterId; + + public KafkaPartitionSplitReaderWrapper( + Properties props, + SourceReaderContext context, + KafkaSourceReaderMetrics kafkaSourceReaderMetrics, + String kafkaClusterId) { + super(props, context, kafkaSourceReaderMetrics); + this.kafkaClusterId = kafkaClusterId; + } + + @Override + public RecordsWithSplitIds> fetch() throws IOException { + return new WrappedRecordsWithSplitIds(super.fetch(), kafkaClusterId); + } + + private static final class WrappedRecordsWithSplitIds + implements RecordsWithSplitIds> { + + private final RecordsWithSplitIds> delegate; + private final String kafkaClusterId; + + public WrappedRecordsWithSplitIds( + RecordsWithSplitIds> delegate, + String kafkaClusterId) { + this.delegate = delegate; + this.kafkaClusterId = kafkaClusterId; + } + + @Nullable + @Override + public String nextSplit() { + String nextSplit = delegate.nextSplit(); + if (nextSplit == null) { + return nextSplit; + } else { + return kafkaClusterId + "-" + nextSplit; + } + } + + @Nullable + @Override + public ConsumerRecord nextRecordFromSplit() { + return delegate.nextRecordFromSplit(); + } + + @Override + public Set finishedSplits() { + return delegate.finishedSplits().stream() + .map(finishedSplit -> kafkaClusterId + "-" + finishedSplit) + .collect(Collectors.toSet()); + } + + @Override + public void recycle() { + delegate.recycle(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java new file mode 100644 index 000000000..25ef25b99 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; + +import com.google.common.base.MoreObjects; + +import java.util.Objects; + +/** Split that wraps {@link KafkaPartitionSplit} with Kafka cluster information. */ +@Internal +public class DynamicKafkaSourceSplit extends KafkaPartitionSplit { + + private final String kafkaClusterId; + private final KafkaPartitionSplit kafkaPartitionSplit; + + public DynamicKafkaSourceSplit(String kafkaClusterId, KafkaPartitionSplit kafkaPartitionSplit) { + super( + kafkaPartitionSplit.getTopicPartition(), + kafkaPartitionSplit.getStartingOffset(), + kafkaPartitionSplit.getStoppingOffset().orElse(NO_STOPPING_OFFSET)); + this.kafkaClusterId = kafkaClusterId; + this.kafkaPartitionSplit = kafkaPartitionSplit; + } + + @Override + public String splitId() { + return kafkaClusterId + "-" + kafkaPartitionSplit.splitId(); + } + + public String getKafkaClusterId() { + return kafkaClusterId; + } + + public KafkaPartitionSplit getKafkaPartitionSplit() { + return kafkaPartitionSplit; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("kafkaClusterId", kafkaClusterId) + .add("kafkaPartitionSplit", kafkaPartitionSplit) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + DynamicKafkaSourceSplit that = (DynamicKafkaSourceSplit) o; + return Objects.equals(kafkaClusterId, that.kafkaClusterId) + && Objects.equals(kafkaPartitionSplit, that.kafkaPartitionSplit); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), kafkaClusterId, kafkaPartitionSplit); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java new file mode 100644 index 000000000..852894f8b --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import com.google.common.io.ByteStreams; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** (De)serializes the {@link DynamicKafkaSourceSplit}. */ +@Internal +public class DynamicKafkaSourceSplitSerializer + implements SimpleVersionedSerializer { + + private static final int VERSION_1 = 1; + + private final KafkaPartitionSplitSerializer kafkaPartitionSplitSerializer; + + public DynamicKafkaSourceSplitSerializer() { + this.kafkaPartitionSplitSerializer = new KafkaPartitionSplitSerializer(); + } + + @Override + public int getVersion() { + return VERSION_1; + } + + @Override + public byte[] serialize(DynamicKafkaSourceSplit split) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + out.writeUTF(split.getKafkaClusterId()); + out.writeInt(kafkaPartitionSplitSerializer.getVersion()); + out.write(kafkaPartitionSplitSerializer.serialize(split.getKafkaPartitionSplit())); + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public DynamicKafkaSourceSplit deserialize(int version, byte[] serialized) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + String kafkaClusterId = in.readUTF(); + int kafkaPartitionSplitSerializerVersion = in.readInt(); + KafkaPartitionSplit kafkaPartitionSplit = + kafkaPartitionSplitSerializer.deserialize( + kafkaPartitionSplitSerializerVersion, ByteStreams.toByteArray(in)); + return new DynamicKafkaSourceSplit(kafkaClusterId, kafkaPartitionSplit); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java new file mode 100644 index 000000000..0e29576cc --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.source; + +import org.apache.flink.annotation.Internal; + +import javax.annotation.Nonnull; + +import java.util.Properties; + +/** Utility class for modify Kafka properties. */ +@Internal +public class KafkaPropertiesUtil { + + private KafkaPropertiesUtil() {} + + public static void copyProperties(@Nonnull Properties from, @Nonnull Properties to) { + for (String key : from.stringPropertyNames()) { + to.setProperty(key, from.getProperty(key)); + } + } + + /** + * client.id is used for Kafka server side logging, see + * https://docs.confluent.io/platform/current/installation/configuration/consumer-configs.html#consumerconfigs_client.id + * + *

Set client id prefix to avoid mbean collision warning logs. There are multiple instances + * of the AdminClient/KafkaConsumer so each instance requires a different client id (i.e. also + * per cluster). + * + *

Flink internally configures the clientId, making this the only way to customize the Kafka + * client id parameter. + * + *

If this is not done, we will encounter warning logs of the form: + * + *

WARN org.apache.kafka.common.utils.AppInfoParser [] - Error registering AppInfo mbean + * javax.management.InstanceAlreadyExistsException: + * kafka.consumer:type=app-info,id=null-enumerator-consumer + * + *

WARN org.apache.kafka.common.utils.AppInfoParser [] - Error registering AppInfo mbean + * javax.management.InstanceAlreadyExistsException: + * kafka.admin.client:type=app-info,id=null-enumerator-admin-client + */ + public static void setClientIdPrefix(Properties properties, String kafkaClusterId) { + String userClientIdPrefix = + properties.getProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key()); + properties.setProperty( + KafkaSourceOptions.CLIENT_ID_PREFIX.key(), + userClientIdPrefix + "-" + kafkaClusterId); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java index 1375d9f60..70c435ee3 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java @@ -42,7 +42,7 @@ public KafkaSourceEnumState( this.initialDiscoveryFinished = initialDiscoveryFinished; } - KafkaSourceEnumState( + public KafkaSourceEnumState( Set assignPartitions, Set unassignedInitialPartitions, boolean initialDiscoveryFinished) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java new file mode 100644 index 000000000..4ea1bd7dc --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java @@ -0,0 +1,694 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.metadata.SingleClusterTopicMetadataService; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.testutils.DynamicKafkaSourceExternalContextFactory; +import org.apache.flink.connector.kafka.testutils.MockKafkaMetadataService; +import org.apache.flink.connector.kafka.testutils.TwoKafkaContainers; +import org.apache.flink.connector.kafka.testutils.YamlFileMetadataService; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; +import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestContext; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; +import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; +import org.apache.flink.streaming.connectors.kafka.KafkaTestBase; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.TestLogger; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup.DYNAMIC_KAFKA_SOURCE_METRIC_GROUP; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration tests for {@link org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSource}. + */ +public class DynamicKafkaSourceITTest extends TestLogger { + + private static final String TOPIC = "DynamicKafkaSourceITTest"; + private static final int NUM_PARTITIONS = 3; + private static final int NUM_RECORDS_PER_SPLIT = 5; + + private static KafkaTestBase.KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata0; + private static KafkaTestBase.KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata1; + MiniClusterWithClientResource miniClusterResource; + static InMemoryReporter reporter; + + @TempDir File testDir; + + @Nested + @TestInstance(TestInstance.Lifecycle.PER_CLASS) + class DynamicKafkaSourceSpecificTests { + @BeforeAll + void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + DynamicKafkaSourceTestHelper.createTopic(TOPIC, NUM_PARTITIONS, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + TOPIC, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT); + + kafkaClusterTestEnvMetadata0 = + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0); + kafkaClusterTestEnvMetadata1 = + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(1); + } + + @BeforeEach + void beforeEach() throws Exception { + reporter = InMemoryReporter.create(); + miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(2) + .setConfiguration( + reporter.addToConfiguration(new Configuration())) + .build()); + miniClusterResource.before(); + } + + @AfterEach + void afterEach() { + reporter.close(); + miniClusterResource.after(); + } + + @AfterAll + void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + void testBasicMultiClusterRead() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + Properties properties = new Properties(); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService( + Collections.singleton( + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds( + mockKafkaMetadataService.getAllStreams().stream() + .map(KafkaStream::getStreamId) + .collect(Collectors.toSet())) + .setKafkaMetadataService(mockKafkaMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + CloseableIterator iterator = stream.executeAndCollect(); + List results = new ArrayList<>(); + while (results.size() + < DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS + * NUM_PARTITIONS + * NUM_RECORDS_PER_SPLIT + && iterator.hasNext()) { + results.add(iterator.next()); + } + + iterator.close(); + + // check that all test records have been consumed + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range( + 0, + DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS + * NUM_PARTITIONS + * NUM_RECORDS_PER_SPLIT) + .boxed() + .collect(Collectors.toList())); + } + + @Test + void testSingleClusterTopicMetadataService() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + Properties properties = new Properties(); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + + KafkaMetadataService kafkaMetadataService = + new SingleClusterTopicMetadataService( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + kafkaClusterTestEnvMetadata0.getStandardProperties()); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds( + // use topics as stream ids + Collections.singleton(TOPIC)) + .setKafkaMetadataService(kafkaMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + CloseableIterator iterator = stream.executeAndCollect(); + List results = new ArrayList<>(); + while (results.size() < NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT && iterator.hasNext()) { + results.add(iterator.next()); + } + + iterator.close(); + + // check that all test records have been consumed + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range(0, NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT) + .boxed() + .collect(Collectors.toList())); + } + + @Test + void testMigrationUsingFileMetadataService() throws Throwable { + // setup topics on two clusters + String fixedTopic = "test-file-metadata-service"; + DynamicKafkaSourceTestHelper.createTopic(fixedTopic, NUM_PARTITIONS); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.setParallelism(2); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty(CommonClientConfigs.GROUP_ID_CONFIG, "dynamic-kafka-src"); + + // create new metadata file to consume from 1 cluster + String testStreamId = "test-file-metadata-service-stream"; + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds(Collections.singleton(testStreamId)) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + List results = new ArrayList<>(); + + AtomicInteger latestValueOffset = + new AtomicInteger( + DynamicKafkaSourceTestHelper.produceToKafka( + 0, fixedTopic, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0)); + + try (CloseableIterator iterator = stream.executeAndCollect()) { + CommonTestUtils.waitUtil( + () -> { + try { + results.add(iterator.next()); + + // trigger metadata update to consume from two clusters + if (results.size() == NUM_RECORDS_PER_SPLIT) { + latestValueOffset.set( + DynamicKafkaSourceTestHelper.produceToKafka( + 0, + fixedTopic, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + latestValueOffset.get())); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata(0), + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata(1))); + } + + // trigger another metadata update to remove old cluster + if (results.size() == latestValueOffset.get()) { + latestValueOffset.set( + DynamicKafkaSourceTestHelper.produceToKafka( + 1, + fixedTopic, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + latestValueOffset.get())); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata(1))); + } + } catch (NoSuchElementException e) { + // swallow and wait + } catch (IOException e) { + throw new UncheckedIOException(e); + } catch (Throwable e) { + throw new RuntimeException(e); + } + + // we will produce 3x + return results.size() == NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT * 3; + }, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + } + + // verify no data loss / duplication in metadata changes + // cluster0 contains 0-10 + // cluster 1 contains 10-30 + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range(0, NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT * 3) + .boxed() + .collect(Collectors.toList())); + } + + @Test + void testStreamPatternSubscriber() throws Throwable { + DynamicKafkaSourceTestHelper.createTopic(0, "stream-pattern-test-1", NUM_PARTITIONS); + int lastValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 0, "stream-pattern-test-1", NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0); + DynamicKafkaSourceTestHelper.createTopic(0, "stream-pattern-test-2", NUM_PARTITIONS); + lastValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 0, + "stream-pattern-test-2", + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + lastValueOffset); + DynamicKafkaSourceTestHelper.createTopic(1, "stream-pattern-test-3", NUM_PARTITIONS); + final int totalRecords = + DynamicKafkaSourceTestHelper.produceToKafka( + 1, + "stream-pattern-test-3", + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + lastValueOffset); + + // create new metadata file to consume from 1 cluster + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + + Set kafkaStreams = + getKafkaStreams( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + kafkaClusterTestEnvMetadata0.getStandardProperties(), + ImmutableSet.of("stream-pattern-test-1", "stream-pattern-test-2")); + + writeClusterMetadataToFile(metadataFile, kafkaStreams); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty(CommonClientConfigs.GROUP_ID_CONFIG, "dynamic-kafka-src"); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamPattern(Pattern.compile("stream-pattern-test-.+")) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + List results = new ArrayList<>(); + + try (CloseableIterator iterator = stream.executeAndCollect()) { + CommonTestUtils.waitUtil( + () -> { + try { + Integer record = iterator.next(); + results.add(record); + + // add third stream that matches the regex + if (results.size() == NUM_RECORDS_PER_SPLIT) { + kafkaStreams.add( + getKafkaStream( + kafkaClusterTestEnvMetadata1 + .getKafkaClusterId(), + kafkaClusterTestEnvMetadata1 + .getStandardProperties(), + "stream-pattern-test-3")); + writeClusterMetadataToFile(metadataFile, kafkaStreams); + } + } catch (NoSuchElementException e) { + // swallow + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + return results.size() == totalRecords; + }, + Duration.ofSeconds(15), + "Could not obtain the required records within the timeout"); + } + // verify no data loss / duplication in metadata changes + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range(0, totalRecords).boxed().collect(Collectors.toList())); + } + + @Test + void testMetricsLifecycleManagement() throws Throwable { + // setup topics on two clusters + String fixedTopic = "test-metrics-lifecycle-mgmt"; + DynamicKafkaSourceTestHelper.createTopic(fixedTopic, NUM_PARTITIONS); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + env.setRestartStrategy(RestartStrategies.noRestart()); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty( + CommonClientConfigs.GROUP_ID_CONFIG, "testMetricsLifecycleManagement"); + + // create new metadata file to consume from 1 cluster + String testStreamId = "test-file-metadata-service-stream"; + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds(Collections.singleton(testStreamId)) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + + int latestValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 0, fixedTopic, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0); + List results = new ArrayList<>(); + try (CloseableIterator iterator = stream.executeAndCollect()) { + while (results.size() < latestValueOffset && iterator.hasNext()) { + results.add(iterator.next()); + } + + assertThat(results) + .containsOnlyOnceElementsOf( + IntStream.range(0, latestValueOffset) + .boxed() + .collect(Collectors.toList())); + + // should contain cluster 0 metrics + assertThat(findMetrics(reporter, DYNAMIC_KAFKA_SOURCE_METRIC_GROUP)) + .allSatisfy( + metricName -> + assertThat(metricName) + .containsPattern( + ".*" + + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP + + "\\.kafkaCluster\\.kafka-cluster-0.*")); + + // setup test data for cluster 1 and stop consuming from cluster 0 + latestValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 1, + fixedTopic, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + latestValueOffset); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(1))); + while (results.size() < latestValueOffset && iterator.hasNext()) { + results.add(iterator.next()); + } + + // cluster 0 is not being consumed from, metrics should not appear + assertThat(findMetrics(reporter, DYNAMIC_KAFKA_SOURCE_METRIC_GROUP)) + .allSatisfy( + metricName -> + assertThat(metricName) + .doesNotContainPattern( + ".*" + + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP + + "\\.kafkaCluster\\.kafka-cluster-0.*")); + + assertThat(findMetrics(reporter, DYNAMIC_KAFKA_SOURCE_METRIC_GROUP)) + .allSatisfy( + metricName -> + assertThat(metricName) + .containsPattern( + ".*" + + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP + + "\\.kafkaCluster\\.kafka-cluster-1.*")); + } + } + + private void writeClusterMetadataToFile(File metadataFile, Set kafkaStreams) + throws IOException { + List streamMetadataList = new ArrayList<>(); + for (KafkaStream kafkaStream : kafkaStreams) { + List clusterMetadataList = + new ArrayList<>(); + + for (Map.Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + YamlFileMetadataService.StreamMetadata.ClusterMetadata clusterMetadata = + new YamlFileMetadataService.StreamMetadata.ClusterMetadata(); + clusterMetadata.setClusterId(entry.getKey()); + clusterMetadata.setBootstrapServers( + entry.getValue() + .getProperties() + .getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG)); + clusterMetadata.setTopics(new ArrayList<>(entry.getValue().getTopics())); + clusterMetadataList.add(clusterMetadata); + } + + YamlFileMetadataService.StreamMetadata streamMetadata = + new YamlFileMetadataService.StreamMetadata(); + streamMetadata.setStreamId(kafkaStream.getStreamId()); + streamMetadata.setClusterMetadataList(clusterMetadataList); + streamMetadataList.add(streamMetadata); + } + + YamlFileMetadataService.saveToYaml(streamMetadataList, metadataFile); + } + + private void writeClusterMetadataToFile( + File metadataFile, + String streamId, + String topic, + List kafkaClusterTestEnvMetadataList) + throws IOException { + List clusterMetadata = + kafkaClusterTestEnvMetadataList.stream() + .map( + KafkaClusterTestEnvMetadata -> + new YamlFileMetadataService.StreamMetadata + .ClusterMetadata( + KafkaClusterTestEnvMetadata.getKafkaClusterId(), + KafkaClusterTestEnvMetadata + .getBrokerConnectionStrings(), + ImmutableList.of(topic))) + .collect(Collectors.toList()); + YamlFileMetadataService.StreamMetadata streamMetadata = + new YamlFileMetadataService.StreamMetadata(streamId, clusterMetadata); + YamlFileMetadataService.saveToYaml( + Collections.singletonList(streamMetadata), metadataFile); + } + + private Set findMetrics(InMemoryReporter inMemoryReporter, String groupPattern) { + Optional groups = inMemoryReporter.findGroup(groupPattern); + assertThat(groups).isPresent(); + return inMemoryReporter.getMetricsByGroup(groups.get()).keySet().stream() + .map(metricName -> groups.get().getMetricIdentifier(metricName)) + .collect(Collectors.toSet()); + } + + private Set getKafkaStreams( + String kafkaClusterId, Properties properties, Collection topics) { + return topics.stream() + .map(topic -> getKafkaStream(kafkaClusterId, properties, topic)) + .collect(Collectors.toSet()); + } + + private KafkaStream getKafkaStream( + String kafkaClusterId, Properties properties, String topic) { + return new KafkaStream( + topic, + Collections.singletonMap( + kafkaClusterId, + new ClusterMetadata(Collections.singleton(topic), properties))); + } + } + + /** Integration test based on connector testing framework. */ + @Nested + class IntegrationTests extends SourceTestSuiteBase { + @TestSemantics + CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; + + // Defines test environment on Flink MiniCluster + @SuppressWarnings("unused") + @TestEnv + MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + + @TestExternalSystem + DefaultContainerizedExternalSystem twoKafkas = + DefaultContainerizedExternalSystem.builder() + .fromContainer(new TwoKafkaContainers()) + .build(); + + @SuppressWarnings("unused") + @TestContext + DynamicKafkaSourceExternalContextFactory twoClusters = + new DynamicKafkaSourceExternalContextFactory( + twoKafkas.getContainer().getKafka0(), + twoKafkas.getContainer().getKafka1(), + Collections.emptyList()); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java new file mode 100644 index 000000000..66caec4c7 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.source.enumerator.AssignmentStatus; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.TopicPartitionAndAssignmentStatus; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Test; + +import java.util.Properties; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * A test {@link + * org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumStateSerializer}. + */ +public class DynamicKafkaSourceEnumStateSerializerTest { + + @Test + public void testSerde() throws Exception { + DynamicKafkaSourceEnumStateSerializer dynamicKafkaSourceEnumStateSerializer = + new DynamicKafkaSourceEnumStateSerializer(); + + Properties propertiesForCluster0 = new Properties(); + propertiesForCluster0.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "cluster0:9092"); + Properties propertiesForCluster1 = new Properties(); + propertiesForCluster1.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "cluster1:9092"); + + Set kafkaStreams = + ImmutableSet.of( + new KafkaStream( + "stream0", + ImmutableMap.of( + "cluster0", + new ClusterMetadata( + ImmutableSet.of("topic0", "topic1"), + propertiesForCluster0), + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic2", "topic3"), + propertiesForCluster1))), + new KafkaStream( + "stream1", + ImmutableMap.of( + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic4", "topic5"), + propertiesForCluster1)))); + + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState = + new DynamicKafkaSourceEnumState( + kafkaStreams, + ImmutableMap.of( + "cluster0", + new KafkaSourceEnumState( + ImmutableSet.of( + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic0", 0), + AssignmentStatus.ASSIGNED), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic1", 1), + AssignmentStatus.UNASSIGNED_INITIAL)), + true), + "cluster1", + new KafkaSourceEnumState( + ImmutableSet.of( + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic2", 0), + AssignmentStatus.UNASSIGNED_INITIAL), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic3", 1), + AssignmentStatus.UNASSIGNED_INITIAL), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic4", 2), + AssignmentStatus.UNASSIGNED_INITIAL), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic5", 3), + AssignmentStatus.UNASSIGNED_INITIAL)), + false))); + + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumStateAfterSerde = + dynamicKafkaSourceEnumStateSerializer.deserialize( + 1, + dynamicKafkaSourceEnumStateSerializer.serialize( + dynamicKafkaSourceEnumState)); + + assertThat(dynamicKafkaSourceEnumState) + .usingRecursiveComparison() + .isEqualTo(dynamicKafkaSourceEnumStateAfterSerde); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java new file mode 100644 index 000000000..05046d406 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java @@ -0,0 +1,964 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSourceOptions; +import org.apache.flink.connector.kafka.dynamic.source.GetMetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSetSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.testutils.MockKafkaMetadataService; +import org.apache.flink.mock.Whitebox; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; + +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** A test for {@link DynamicKafkaSourceEnumerator}. */ +public class DynamicKafkaSourceEnumeratorTest { + private static final int NUM_SUBTASKS = 3; + private static final String TOPIC = "DynamicKafkaSourceEnumeratorTest"; + private static final int NUM_SPLITS_PER_CLUSTER = 3; + private static final int NUM_RECORDS_PER_SPLIT = 5; + + @BeforeAll + public static void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + DynamicKafkaSourceTestHelper.createTopic(TOPIC, NUM_SPLITS_PER_CLUSTER, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + TOPIC, NUM_SPLITS_PER_CLUSTER, NUM_RECORDS_PER_SPLIT); + } + + @AfterAll + public static void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + public void testStartupWithoutContinuousDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).isEmpty(); + assertThat(context.getOneTimeCallables()) + .as("A one time partition discovery callable should have been scheduled") + .hasSize(1); + } + } + + @Test + public void testStartupWithContinuousDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + assertThat(context.getOneTimeCallables()).isEmpty(); + assertThat(context.getPeriodicCallables()) + .as("A one time partition discovery callable should have been scheduled") + .hasSize(1); + } + } + + @Test + public void testStartupWithKafkaMetadataServiceFailure_noPeriodicDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, new MockKafkaMetadataService(true), (properties) -> {})) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).isEmpty(); + assertThat(context.getOneTimeCallables()) + .as("A one time partition discovery callable should have been scheduled") + .hasSize(1); + assertThatThrownBy(() -> runAllOneTimeCallables(context)) + .as( + "Exception expected since periodic discovery is disabled and metadata is required for setting up the job") + .hasRootCause(new RuntimeException("Mock exception")); + } + } + + @Test + public void testStartupWithKafkaMetadataServiceFailure_withContinuousDiscovery() + throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + new MockKafkaMetadataService(true), + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).hasSize(1); + assertThatThrownBy(() -> context.runPeriodicCallable(0)) + .as("Exception expected since there is no state") + .hasRootCause(new RuntimeException("Mock exception")); + } + } + + @Test + public void + testStartupWithKafkaMetadataServiceFailure_withContinuousDiscoveryAndCheckpointState() + throws Throwable { + // init enumerator with checkpoint state + final DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState = getCheckpointState(); + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + new MockKafkaMetadataService(true), + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + dynamicKafkaSourceEnumState, + new TestKafkaEnumContextProxyFactory())) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).hasSize(1); + // no exception + context.runPeriodicCallable(0); + + assertThatThrownBy(() -> context.runPeriodicCallable(0)) + .hasRootCause(new RuntimeException("Mock exception")); + } + } + + @Test + public void testHandleMetadataServiceError() throws Throwable { + int failureThreshold = 5; + + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + Integer.toString(failureThreshold)); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService( + Collections.singleton(DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))); + + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + mockKafkaMetadataService, + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + new DynamicKafkaSourceEnumState(), + new TestKafkaEnumContextProxyFactory())) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).hasSize(1); + context.runPeriodicCallable(0); + + // init splits + runAllOneTimeCallables(context); + + // swap to exceptional metadata service + mockKafkaMetadataService.setThrowException(true); + + for (int i = 0; i < failureThreshold; i++) { + context.runPeriodicCallable(0); + } + + for (int i = 0; i < 2; i++) { + assertThatThrownBy(() -> context.runPeriodicCallable(0)) + .hasRootCause(new RuntimeException("Mock exception")); + // Need to reset internal throwable reference after each invocation of + // runPeriodicCallable, + // since the context caches the previous exceptions indefinitely + AtomicReference errorInWorkerThread = + (AtomicReference) + Whitebox.getInternalState(context, "errorInWorkerThread"); + errorInWorkerThread.set(null); + } + + mockKafkaMetadataService.setThrowException(false); + assertThatCode(() -> context.runPeriodicCallable(0)) + .as("Exception counter should have been reset") + .doesNotThrowAnyException(); + } + } + + @Test + public void testKafkaMetadataServiceDiscovery() throws Throwable { + KafkaStream kafkaStreamWithOneCluster = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + kafkaStreamWithOneCluster + .getClusterMetadataMap() + .remove(DynamicKafkaSourceTestHelper.getKafkaClusterId(1)); + + KafkaStream kafkaStreamWithTwoClusters = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService(Collections.singleton(kafkaStreamWithOneCluster)); + + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + mockKafkaMetadataService, + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + context.runPeriodicCallable(0); + + // 1 callable for main enumerator and 2 for the sub enumerators since we have 2 clusters + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + + assertThat(context.getSplitsAssignmentSequence()) + .as("no splits should be assigned yet since there are no readers") + .isEmpty(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithOneCluster); + int currentNumSplits = context.getSplitsAssignmentSequence().size(); + + // no changes to splits + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + assertThat(context.getSplitsAssignmentSequence()).hasSize(currentNumSplits); + + // mock metadata change + mockKafkaMetadataService.setKafkaStreams( + Collections.singleton(kafkaStreamWithTwoClusters)); + + // changes should have occurred here + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + assertThat(context.getSplitsAssignmentSequence()) + .as( + "1 additional split assignment since there was 1 metadata update that caused a change") + .hasSize(currentNumSplits + 1); + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithTwoClusters); + } + } + + @Test + public void testReaderRegistrationAfterSplitDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + assertThat(context.getSplitsAssignmentSequence()) + .as("no splits should be assigned yet since there are no readers") + .isEmpty(); + assertThat(context.getSplitsAssignmentSequence()) + .as("no readers have registered yet") + .isEmpty(); + assertThat(context.getSentSourceEvent()).as("no readers have registered yet").isEmpty(); + + // initialize readers 0 and 2 + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + assertThat(context.getSentSourceEvent().keySet()) + .as("reader 0 and 2 should have only received the source event") + .containsExactlyInAnyOrder(0, 2); + Set allReadersThatReceivedSplits = + context.getSplitsAssignmentSequence().stream() + .flatMap( + splitAssignment -> + splitAssignment.assignment().keySet().stream()) + .collect(Collectors.toSet()); + assertThat(allReadersThatReceivedSplits) + .as("reader 0 and 2 should hve only received splits") + .containsExactlyInAnyOrder(0, 2); + + // initialize readers 1 + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + assertThat(context.getSentSourceEvent().keySet()) + .as("all readers should have received get metadata update event") + .containsExactlyInAnyOrder(0, 1, 2); + + for (List sourceEventsPerReader : context.getSentSourceEvent().values()) { + assertThat(sourceEventsPerReader) + .as("there should have been only 1 source event per reader") + .hasSize(1); + } + + // should have all splits assigned by now + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + } + } + + @Test + public void testReaderRegistrationBeforeSplitDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + assertThat(context.getSplitsAssignmentSequence()) + .as("readers should not be assigned yet since there are no splits") + .isEmpty(); + + // 1 callable for main enumerator and 3 for the sub enumerators since we have 3 clusters + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + } + } + + @Test + public void testSnapshotState() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + DynamicKafkaSourceEnumState stateBeforeSplitAssignment = enumerator.snapshotState(-1); + assertThat( + stateBeforeSplitAssignment.getClusterEnumeratorStates().values() + .stream() + .map(subState -> subState.assignedPartitions().stream()) + .count()) + .as("no readers registered, so state should be empty") + .isZero(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + assertThat(context.getSplitsAssignmentSequence()) + .as("readers should not be assigned yet since there are no splits") + .isEmpty(); + + // 1 callable for main enumerator and 3 for the sub enumerators since we have 3 clusters + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + + DynamicKafkaSourceEnumState stateAfterSplitAssignment = enumerator.snapshotState(-1); + + assertThat( + stateAfterSplitAssignment.getClusterEnumeratorStates().values().stream() + .flatMap(enumState -> enumState.assignedPartitions().stream()) + .count()) + .isEqualTo( + NUM_SPLITS_PER_CLUSTER + * DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + } + } + + @Test + public void testStartupWithCheckpointState() throws Throwable { + // init enumerator with checkpoint state + final DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState = getCheckpointState(); + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + new MockKafkaMetadataService( + Collections.singleton( + DynamicKafkaSourceTestHelper.getKafkaStream( + TOPIC))), + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + dynamicKafkaSourceEnumState, + new TestKafkaEnumContextProxyFactory())) { + // start and check callables + enumerator.start(); + assertThat(context.getPeriodicCallables()).isEmpty(); + assertThat(context.getOneTimeCallables()) + .as( + "3 one time callables should have been scheduled. 1 for main enumerator and then 2 for each underlying enumerator") + .hasSize(1 + DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // initialize all readers and do split assignment + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + runAllOneTimeCallables(context); + + assertThat(context.getSentSourceEvent()).as("3 readers registered").hasSize(3); + for (List sourceEventsReceived : context.getSentSourceEvent().values()) { + assertThat(sourceEventsReceived) + .as("each reader should have sent 1 source event") + .hasSize(1); + } + + assertThat(context.getSplitsAssignmentSequence()) + .as( + "there should not be new splits and we don't assign previously assigned splits at startup and there is no metadata/split changes") + .isEmpty(); + } + + // test with periodic discovery enabled + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + new MockKafkaMetadataService( + Collections.singleton( + DynamicKafkaSourceTestHelper.getKafkaStream( + TOPIC))), + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + dynamicKafkaSourceEnumState, + new TestKafkaEnumContextProxyFactory())) { + enumerator.start(); + assertThat(context.getPeriodicCallables()) + .as("A periodic time partition discovery callable should have been scheduled") + .hasSize(1); + assertThat(context.getOneTimeCallables()) + .as( + "0 one time callables for main enumerator and 2 one time callables for each underlying enumerator should have been scheduled") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + // checkpoint state should have triggered split assignment + assertThat(context.getSplitsAssignmentSequence()) + .as( + "There is no split assignment since there are no new splits that are not contained in state") + .isEmpty(); + } + } + + @Test + public void testAddSplitsBack() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + enumerator.start(); + + runAllOneTimeCallables(context); + + Map> readerAssignmentsBeforeFailure = + getReaderAssignments(context); + assertThat(context.getSplitsAssignmentSequence()) + .as("we only expect splits have been assigned 2 times") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // simulate failures + context.unregisterReader(0); + enumerator.addSplitsBack( + context.getSplitsAssignmentSequence().get(0).assignment().get(0), 0); + context.unregisterReader(2); + enumerator.addSplitsBack( + context.getSplitsAssignmentSequence().get(0).assignment().get(2), 2); + assertThat(context.getSplitsAssignmentSequence()) + .as("Splits assignment should be unchanged") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // mock reader recovery + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + assertThat(getReaderAssignments(context)) + .containsAllEntriesOf(readerAssignmentsBeforeFailure); + assertThat(context.getSplitsAssignmentSequence()) + .as( + "the readers came back up, so there should be 2 additional split assignments in the sequence") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS + 2); + } + } + + @Test + public void testEnumeratorDoesNotAssignDuplicateSplitsInMetadataUpdate() throws Throwable { + KafkaStream kafkaStreamWithOneCluster = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + kafkaStreamWithOneCluster + .getClusterMetadataMap() + .remove(DynamicKafkaSourceTestHelper.getKafkaClusterId(1)); + + KafkaStream kafkaStreamWithTwoClusters = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService(Collections.singleton(kafkaStreamWithOneCluster)); + + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + mockKafkaMetadataService, + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + new DynamicKafkaSourceEnumState(), + new TestKafkaEnumContextProxyFactory())) { + + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + enumerator.start(); + + // run all discovery + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithOneCluster); + + // trigger metadata change + mockKafkaMetadataService.setKafkaStreams( + Collections.singleton(kafkaStreamWithTwoClusters)); + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithTwoClusters); + + Map splitAssignmentFrequencyMap = new HashMap<>(); + for (SplitsAssignment splitsAssignmentStep : + context.getSplitsAssignmentSequence()) { + for (List assignments : + splitsAssignmentStep.assignment().values()) { + for (DynamicKafkaSourceSplit assignment : assignments) { + splitAssignmentFrequencyMap.put( + assignment.splitId(), + splitAssignmentFrequencyMap.getOrDefault(assignment.splitId(), 0) + + 1); + } + } + } + + assertThat(splitAssignmentFrequencyMap.values()) + .as("all splits should have been assigned once") + .allMatch(count -> count == 1); + } + } + + @Test + public void testInitExceptionNonexistingKafkaCluster() { + Properties fakeProperties = new Properties(); + fakeProperties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "fake-cluster:8080"); + MockKafkaMetadataService mockKafkaMetadataServiceWithUnavailableCluster = + new MockKafkaMetadataService( + ImmutableSet.of( + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC), + new KafkaStream( + "fake-stream", + Collections.singletonMap( + "fake-cluster", + new ClusterMetadata( + Collections.singleton("fake-topic"), + fakeProperties))))); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator(context, mockKafkaMetadataServiceWithUnavailableCluster)) { + enumerator.start(); + + runAllOneTimeCallables(context); + } catch (Throwable throwable) { + assertThat(throwable).hasRootCauseInstanceOf(KafkaException.class); + } + } + + @Test + public void testEnumeratorErrorPropagation() { + Properties fakeProperties = new Properties(); + fakeProperties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "fake-cluster:8080"); + KafkaStream fakeStream = + new KafkaStream( + "fake-stream", + Collections.singletonMap( + "fake-cluster", + new ClusterMetadata( + Collections.singleton("fake-topic"), fakeProperties))); + + MockKafkaMetadataService mockKafkaMetadataServiceWithUnavailableCluster = + new MockKafkaMetadataService( + ImmutableSet.of( + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC), fakeStream)); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator(context, mockKafkaMetadataServiceWithUnavailableCluster)) { + enumerator.start(); + + runAllOneTimeCallables(context); + } catch (Throwable throwable) { + assertThat(throwable).hasRootCauseInstanceOf(KafkaException.class); + } + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context) { + return createEnumerator( + context, + new MockKafkaMetadataService( + Collections.singleton(DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))), + (properties) -> {}); + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context, + KafkaMetadataService kafkaMetadataService) { + return createEnumerator(context, kafkaMetadataService, (properties) -> {}); + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context, + Consumer applyPropertiesConsumer) { + return createEnumerator( + context, + new MockKafkaMetadataService( + Collections.singleton(DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))), + applyPropertiesConsumer); + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context, + KafkaMetadataService kafkaMetadataService, + Consumer applyPropertiesConsumer) { + Properties properties = new Properties(); + applyPropertiesConsumer.accept(properties); + properties.putIfAbsent(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.putIfAbsent( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + return new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + kafkaMetadataService, + context, + OffsetsInitializer.earliest(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + new DynamicKafkaSourceEnumState(), + new TestKafkaEnumContextProxyFactory()); + } + + private void mockRegisterReaderAndSendReaderStartupEvent( + MockSplitEnumeratorContext context, + DynamicKafkaSourceEnumerator enumerator, + int reader) { + context.registerReader(new ReaderInfo(reader, "location " + reader)); + enumerator.addReader(reader); + // readers send source event at startup + enumerator.handleSourceEvent(reader, new GetMetadataUpdateEvent()); + } + + private void verifyAllSplitsHaveBeenAssigned( + List> splitsAssignmentSequence, + KafkaStream kafkaStream) { + Map> clusterTopicMap = new HashMap<>(); + for (Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + clusterTopicMap + .computeIfAbsent(entry.getKey(), unused -> new HashSet<>()) + .addAll(entry.getValue().getTopics()); + } + + Set splitsAssigned = + splitsAssignmentSequence.stream() + .flatMap( + splitsAssignment -> + splitsAssignment.assignment().values().stream() + .flatMap(Collection::stream)) + .collect(Collectors.toSet()); + + assertThat(splitsAssignmentSequence).isNotEmpty(); + + Map> clusterToTopicPartition = new HashMap<>(); + for (SplitsAssignment split : splitsAssignmentSequence) { + for (Entry> assignments : + split.assignment().entrySet()) { + for (DynamicKafkaSourceSplit assignment : assignments.getValue()) { + clusterToTopicPartition + .computeIfAbsent(assignment.getKafkaClusterId(), key -> new HashSet<>()) + .add(assignment.getKafkaPartitionSplit().getTopicPartition()); + } + } + } + + assertThat(splitsAssigned) + .hasSize(NUM_SPLITS_PER_CLUSTER * clusterTopicMap.keySet().size()); + + // verify correct clusters + for (String kafkaClusterId : clusterTopicMap.keySet()) { + assertThat(clusterToTopicPartition) + .as("All Kafka clusters must be assigned in the splits.") + .containsKey(kafkaClusterId); + } + + // verify topic partitions + Set assignedTopicPartitionSet = + clusterToTopicPartition.values().stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); + for (Set topics : clusterTopicMap.values()) { + for (String topic : topics) { + Set expectedTopicPartitions = new HashSet<>(); + for (int i = 0; i < NUM_SPLITS_PER_CLUSTER; i++) { + expectedTopicPartitions.add(new TopicPartition(topic, i)); + } + assertThat(assignedTopicPartitionSet) + .as("splits must contain all topics and 2 partitions per topic") + .containsExactlyInAnyOrderElementsOf(expectedTopicPartitions); + } + } + } + + private Map> getReaderAssignments( + MockSplitEnumeratorContext context) { + Map> readerToSplits = new HashMap<>(); + for (SplitsAssignment split : + context.getSplitsAssignmentSequence()) { + for (Entry> assignments : + split.assignment().entrySet()) { + readerToSplits + .computeIfAbsent(assignments.getKey(), key -> new HashSet<>()) + .addAll(assignments.getValue()); + } + } + return readerToSplits; + } + + private static void runAllOneTimeCallables(MockSplitEnumeratorContext context) + throws Throwable { + while (!context.getOneTimeCallables().isEmpty()) { + context.runNextOneTimeCallable(); + } + } + + private DynamicKafkaSourceEnumState getCheckpointState(KafkaStream kafkaStream) + throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + new MockKafkaMetadataService(Collections.singleton(kafkaStream)), + (properties) -> {})) { + enumerator.start(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned(context.getSplitsAssignmentSequence(), kafkaStream); + + return enumerator.snapshotState(-1); + } + } + + private DynamicKafkaSourceEnumState getCheckpointState() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + + return enumerator.snapshotState(-1); + } + } + + private static class TestKafkaEnumContextProxyFactory + implements StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory { + @Override + public StoppableKafkaEnumContextProxy create( + SplitEnumeratorContext enumContext, + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService) { + return new TestKafkaEnumContextProxy( + kafkaClusterId, + kafkaMetadataService, + (MockSplitEnumeratorContext) enumContext); + } + } + + private static class TestKafkaEnumContextProxy extends StoppableKafkaEnumContextProxy { + + private final SplitEnumeratorContext enumContext; + + public TestKafkaEnumContextProxy( + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService, + MockSplitEnumeratorContext enumContext) { + super(kafkaClusterId, kafkaMetadataService, enumContext); + this.enumContext = enumContext; + } + + /** + * Schedule periodic callables under the coordinator executor, so we can use {@link + * MockSplitEnumeratorContext} to invoke the callable (split assignment) on demand to test + * the integration of KafkaSourceEnumerator. + */ + @Override + public void callAsync( + Callable callable, + BiConsumer handler, + long initialDelay, + long period) { + enumContext.callAsync( + wrapCallAsyncCallable(callable), + wrapCallAsyncCallableHandler(handler), + initialDelay, + period); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java new file mode 100644 index 000000000..694c95c65 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.testutils.MockKafkaMetadataService; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.mock.Whitebox; +import org.apache.flink.util.FlinkRuntimeException; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.common.errors.TimeoutException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.Collections; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.Assertions.fail; + +/** A test for {@link StoppableKafkaEnumContextProxy}. */ +public class StoppableKafkaEnumContextProxyTest { + + private static final String ACTIVE_KAFKA_CLUSTER = "mock-kafka-cluster"; + private static final String INACTIVE_KAFKA_CLUSTER = "mock-inactive-kafka-cluster"; + + private volatile boolean throwExceptionFromMainCallable; + + @BeforeEach + public void beforeEach() { + throwExceptionFromMainCallable = true; + } + + @AfterAll + public static void afterAll() throws Exception {} + + @Test + public void testOneTimeCallableErrorHandling() throws Throwable { + try (MockSplitEnumeratorContext enumContext = + new MockSplitEnumeratorContext<>(2); + StoppableKafkaEnumContextProxy enumContextProxy = + createStoppableKafkaEnumContextProxy(enumContext)) { + + AtomicBoolean isCallbackInvoked = new AtomicBoolean(); + setupKafkaTopicPartitionDiscoveryMockCallable(enumContextProxy, isCallbackInvoked); + + CommonTestUtils.waitUtil( + () -> enumContext.getOneTimeCallables().size() == 1, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + + // not running the next periodic callable, since StoppableKafkaEnumContextProxy has the + // callable that periodically schedules the proxy task. So the proxy task (e.g. split + // discovery) is a one time callable in the context of source coordinator. + enumContext.runNextOneTimeCallable(); + assertThat(isCallbackInvoked) + .as("callback should be skipped upon swallowing the error.") + .isFalse(); + } + } + + @Test + public void testPeriodicCallableErrorHandling() throws Throwable { + try (MockSplitEnumeratorContext enumContext = + new MockSplitEnumeratorContext<>(2); + StoppableKafkaEnumContextProxy enumContextProxy = + createStoppableKafkaEnumContextProxy(enumContext)) { + + AtomicBoolean isCallbackInvoked = new AtomicBoolean(); + setupKafkaTopicPartitionDiscoveryMockCallable(enumContextProxy, isCallbackInvoked); + + CommonTestUtils.waitUtil( + () -> enumContext.getOneTimeCallables().size() == 1, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + + enumContext.runNextOneTimeCallable(); + assertThat(isCallbackInvoked) + .as("callback should be skipped upon swallowing the error.") + .isFalse(); + } + } + + @Test + public void testPeriodicCallableThrowsExceptionOnActiveCluster() throws Throwable { + try (MockSplitEnumeratorContext enumContext = + new MockSplitEnumeratorContext<>(2); + StoppableKafkaEnumContextProxy enumContextProxy = + createStoppableKafkaEnumContextProxy(enumContext, ACTIVE_KAFKA_CLUSTER)) { + + AtomicBoolean isCallbackInvoked = new AtomicBoolean(); + setupKafkaTopicPartitionDiscoveryMockCallable(enumContextProxy, isCallbackInvoked); + + CommonTestUtils.waitUtil( + () -> enumContext.getOneTimeCallables().size() == 1, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + + assertThatThrownBy(() -> runNextOneTimeCallableAndCatchException(enumContext)) + .isExactlyInstanceOf(FlinkRuntimeException.class); + assertThat(isCallbackInvoked) + .as("error callback should be invoked since splits have not been assigned yet.") + .isTrue(); + } + } + + private StoppableKafkaEnumContextProxy createStoppableKafkaEnumContextProxy( + SplitEnumeratorContext enumContext) { + return createStoppableKafkaEnumContextProxy(enumContext, INACTIVE_KAFKA_CLUSTER); + } + + private StoppableKafkaEnumContextProxy createStoppableKafkaEnumContextProxy( + SplitEnumeratorContext enumContext, String contextKafkaCluster) { + + KafkaStream mockStream = + new KafkaStream( + "mock-stream", + ImmutableMap.of( + ACTIVE_KAFKA_CLUSTER, + new ClusterMetadata( + ImmutableSet.of("mock-topic"), new Properties()))); + + return new StoppableKafkaEnumContextProxy( + contextKafkaCluster, + new MockKafkaMetadataService(Collections.singleton(mockStream)), + enumContext); + } + + // this modeled after `KafkaSourceEnumerator` topic partition subscription to throw the same + // exceptions + private void setupKafkaTopicPartitionDiscoveryMockCallable( + StoppableKafkaEnumContextProxy enumContextProxy, AtomicBoolean isCallbackInvoked) { + enumContextProxy.callAsync( + () -> { + if (throwExceptionFromMainCallable) { + // mock Kafka Exception + throw new TimeoutException("Kafka server timed out"); + } else { + // ignore output + return null; + } + }, + (res, t) -> { + isCallbackInvoked.set(true); + if (t != null) { + throw new FlinkRuntimeException(t); + } + }, + 0, + 1000); + } + + private void runNextOneTimeCallableAndCatchException(MockSplitEnumeratorContext enumContext) + throws Throwable { + try { + enumContext.runNextOneTimeCallable(); + fail("TimeoutException should have been thrown"); + } catch (TimeoutException e) { + // catch only Kafka Timeout exceptions since it will be rethrown by + // `MockSplitEnumeratorContext` + AtomicReference errorInMainThread = + (AtomicReference) + Whitebox.getInternalState(enumContext, "errorInMainThread"); + AtomicReference errorInWorkerThread = + (AtomicReference) + Whitebox.getInternalState(enumContext, "errorInWorkerThread"); + + assertThat(errorInMainThread.get()) + .as("Should be error in main executor thread for async io") + .isNotNull(); + assertThat(errorInWorkerThread.get()) + .as( + "Should not be error in worker thread that corresponds to source coordinator thread") + .isNull(); + } finally { + // reset MockSplitEnumeratorContext error state + Whitebox.setInternalState( + enumContext, "errorInMainThread", new AtomicReference()); + Whitebox.setInternalState( + enumContext, "errorInWorkerThread", new AtomicReference()); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java new file mode 100644 index 000000000..4e1fcf0ce --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.metadata; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.metadata.SingleClusterTopicMetadataService; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; +import org.apache.flink.streaming.connectors.kafka.KafkaTestBase; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; + +class SingleClusterTopicMetadataServiceTest { + + private static final String TOPIC0 = "SingleClusterTopicMetadataServiceTest-1"; + private static final String TOPIC1 = "SingleClusterTopicMetadataServiceTest-2"; + + private static KafkaMetadataService kafkaMetadataService; + private static KafkaTestBase.KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata0; + + @BeforeAll + static void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + DynamicKafkaSourceTestHelper.createTopic(TOPIC0, 3); + DynamicKafkaSourceTestHelper.createTopic(TOPIC1, 3); + + kafkaClusterTestEnvMetadata0 = + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0); + + kafkaMetadataService = + new SingleClusterTopicMetadataService( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + kafkaClusterTestEnvMetadata0.getStandardProperties()); + } + + @AfterAll + static void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + void getAllStreams() { + Set allStreams = kafkaMetadataService.getAllStreams(); + assertThat(allStreams) + .as("stream names should be equal to topic names") + .containsExactlyInAnyOrder( + new KafkaStream( + TOPIC0, + ImmutableMap.of( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + new ClusterMetadata( + Collections.singleton(TOPIC0), + kafkaClusterTestEnvMetadata0 + .getStandardProperties()))), + new KafkaStream( + TOPIC1, + Collections.singletonMap( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + new ClusterMetadata( + Collections.singleton(TOPIC1), + kafkaClusterTestEnvMetadata0 + .getStandardProperties())))); + } + + @Test + void describeStreams() { + Map streamMap = + kafkaMetadataService.describeStreams(Collections.singleton(TOPIC1)); + assertThat(streamMap) + .containsExactlyInAnyOrderEntriesOf( + ImmutableMap.of( + TOPIC1, + new KafkaStream( + TOPIC1, + Collections.singletonMap( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + new ClusterMetadata( + Collections.singleton(TOPIC1), + kafkaClusterTestEnvMetadata0 + .getStandardProperties()))))); + + assertThatCode( + () -> + kafkaMetadataService.describeStreams( + Collections.singleton("unknown-stream"))) + .as("the stream topic cannot be found in kafka and we rethrow") + .hasRootCauseInstanceOf(UnknownTopicOrPartitionException.class); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java new file mode 100644 index 000000000..71feeb8a7 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.metrics; + +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +import static org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup.DYNAMIC_KAFKA_SOURCE_METRIC_GROUP; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * A test for {@link + * org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup}. + */ +public class KafkaClusterMetricGroupTest { + + private static MetricListener metricListener; + private static InternalSourceReaderMetricGroup mockInternalSourceReaderMetricGroup; + private static KafkaClusterMetricGroup kafkaClusterMetricGroup; + + @BeforeEach + public void beforeEach() { + metricListener = new MetricListener(); + mockInternalSourceReaderMetricGroup = + InternalSourceReaderMetricGroup.mock(metricListener.getMetricGroup()); + kafkaClusterMetricGroup = + new KafkaClusterMetricGroup( + mockInternalSourceReaderMetricGroup.addGroup( + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP), + mockInternalSourceReaderMetricGroup, + "broker-bootstrap-server:443"); + } + + @Test + public void testGetAllVariables() { + // metric variables are wrapped in <...> + assertThat(kafkaClusterMetricGroup.getAllVariables()) + .as( + "variables should contain Kafka cluster info to distinguish multiple sub KafkaSourceReaders") + .containsEntry( + "<" + KafkaClusterMetricGroup.KAFKA_CLUSTER_GROUP_NAME + ">", + "broker-bootstrap-server:443"); + } + + @Test + public void testGetScopeComponents() { + assertThat(kafkaClusterMetricGroup.getScopeComponents()) + .as("scope components contains previously attached scope component") + .contains(DYNAMIC_KAFKA_SOURCE_METRIC_GROUP); + } + + @Test + public void testSetPendingRecordsGauge() { + kafkaClusterMetricGroup.setPendingRecordsGauge(() -> 5L); + + // these identifiers should be attached to distinguish distinguish multiple sub + // KafkaSourceReaders + Optional> pendingRecordsGauge = + metricListener.getGauge( + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP, + "kafkaCluster", + "broker-bootstrap-server:443", + "pendingRecords"); + + assertThat(pendingRecordsGauge.get().getValue()).isEqualTo(5L); + } + + @Test + public void testGetIOMetricGroup() { + assertThat(kafkaClusterMetricGroup.getIOMetricGroup()) + .isEqualTo(mockInternalSourceReaderMetricGroup.getIOMetricGroup()); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java new file mode 100644 index 000000000..5094e0151 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java @@ -0,0 +1,347 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.reader; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.connector.testutils.source.reader.SourceReaderTestBase; +import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; +import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; + +import com.google.common.collect.ImmutableList; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.powermock.reflect.Whitebox; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Test for {@link org.apache.flink.connector.kafka.dynamic.source.reader.DynamicKafkaSourceReader}. + */ +public class DynamicKafkaSourceReaderTest extends SourceReaderTestBase { + private static final String TOPIC = "DynamicKafkaSourceReaderTest"; + + // we are testing two clusters and SourceReaderTestBase expects there to be a total of 10 splits + private static final int NUM_SPLITS_PER_CLUSTER = 5; + + private static String kafkaClusterId0; + private static String kafkaClusterId1; + + @BeforeAll + static void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + + DynamicKafkaSourceTestHelper.createTopic(TOPIC, NUM_SPLITS_PER_CLUSTER, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + TOPIC, NUM_SPLITS_PER_CLUSTER, NUM_RECORDS_PER_SPLIT); + kafkaClusterId0 = DynamicKafkaSourceTestHelper.getKafkaClusterId(0); + kafkaClusterId1 = DynamicKafkaSourceTestHelper.getKafkaClusterId(1); + } + + @AfterAll + static void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + void testHandleSourceEventWithRemovedMetadataAtStartup() throws Exception { + TestingReaderContext context = new TestingReaderContext(); + try (DynamicKafkaSourceReader reader = createReaderWithoutStart(context)) { + // mock restoring state from Flink runtime + List splits = + getSplits( + getNumSplits(), + NUM_RECORDS_PER_SPLIT, + Boundedness.CONTINUOUS_UNBOUNDED); + reader.addSplits(splits); + + // start reader + reader.start(); + KafkaStream kafkaStream = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + + // remove cluster 0 + kafkaStream.getClusterMetadataMap().remove(kafkaClusterId0); + reader.handleSourceEvents(new MetadataUpdateEvent(Collections.singleton(kafkaStream))); + + List splitsWithoutCluster0 = + splits.stream() + .filter(split -> !split.getKafkaClusterId().equals(kafkaClusterId0)) + .collect(Collectors.toList()); + assertThat(reader.snapshotState(-1)) + .as("The splits should not contain any split related to cluster 0") + .containsExactlyInAnyOrderElementsOf(splitsWithoutCluster0); + } + } + + @Test + void testNoSubReadersInputStatus() throws Exception { + try (DynamicKafkaSourceReader reader = + (DynamicKafkaSourceReader) createReader()) { + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + InputStatus inputStatus = reader.pollNext(readerOutput); + assertEquals( + InputStatus.NOTHING_AVAILABLE, + inputStatus, + "nothing available since there are no sub readers created, there could be sub readers created in the future"); + + // notify that this reader will not be assigned anymore splits + reader.notifyNoMoreSplits(); + + inputStatus = reader.pollNext(readerOutput); + assertEquals( + InputStatus.END_OF_INPUT, + inputStatus, + "there will not be any more input from this reader since there are no splits"); + } + } + + @Test + void testNotifyNoMoreSplits() throws Exception { + TestingReaderContext context = new TestingReaderContext(); + try (DynamicKafkaSourceReader reader = createReaderWithoutStart(context)) { + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + reader.start(); + + // Splits assigned + List splits = + getSplits(getNumSplits(), NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED); + reader.addSplits(splits); + + // Send no more splits + reader.notifyNoMoreSplits(); + + // Send metadata + MetadataUpdateEvent metadata = + DynamicKafkaSourceTestHelper.getMetadataUpdateEvent(TOPIC); + reader.handleSourceEvents(metadata); + + // Check consistency + InputStatus status; + do { + status = reader.pollNext(readerOutput); + } while (status != InputStatus.END_OF_INPUT); + + assertThat(readerOutput.getEmittedRecords()) + .hasSize(getNumSplits() * NUM_RECORDS_PER_SPLIT); + } + } + + @Test + void testAvailabilityFutureUpdates() throws Exception { + TestingReaderContext context = new TestingReaderContext(); + try (DynamicKafkaSourceReader reader = createReaderWithoutStart(context)) { + CompletableFuture futureAtInit = reader.isAvailable(); + assertThat(reader.isActivelyConsumingSplits()).isFalse(); + assertThat(futureAtInit) + .as("future is not complete at fresh startup since no readers are created") + .isNotDone(); + assertThat(getAvailabilityHelperSize(reader)).isZero(); + + reader.start(); + MetadataUpdateEvent metadata = + DynamicKafkaSourceTestHelper.getMetadataUpdateEvent(TOPIC); + reader.handleSourceEvents(metadata); + List splits = + getSplits( + getNumSplits(), + NUM_RECORDS_PER_SPLIT, + Boundedness.CONTINUOUS_UNBOUNDED); + reader.addSplits(splits); + CompletableFuture futureAfterSplitAssignment = reader.isAvailable(); + + assertThat(futureAtInit) + .as( + "New future should have been produced since metadata triggers reader creation") + .isNotSameAs(futureAfterSplitAssignment); + assertThat(getAvailabilityHelperSize(reader)).isEqualTo(2); + + // remove cluster 0 + KafkaStream kafkaStream = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + kafkaStream.getClusterMetadataMap().remove(kafkaClusterId0); + reader.handleSourceEvents(new MetadataUpdateEvent(Collections.singleton(kafkaStream))); + + CompletableFuture futureAfterRemovingCluster0 = reader.isAvailable(); + assertThat(futureAfterRemovingCluster0) + .as("There should new future since the metadata has changed") + .isNotSameAs(futureAfterSplitAssignment); + assertThat(getAvailabilityHelperSize(reader)).isEqualTo(1); + } + } + + private int getAvailabilityHelperSize(DynamicKafkaSourceReader reader) { + return ((CompletableFuture[]) + Whitebox.getInternalState( + reader.getAvailabilityHelper(), "futuresToCombine")) + .length; + } + + @Test + void testReaderMetadataChangeWhenOneTopicChanges() throws Exception { + try (DynamicKafkaSourceReader reader = + (DynamicKafkaSourceReader) createReader()) { + + // splits with offsets + DynamicKafkaSourceSplit cluster0Split = + new DynamicKafkaSourceSplit( + DynamicKafkaSourceTestHelper.getKafkaClusterId(0), + new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 10)); + DynamicKafkaSourceSplit cluster1Split = + new DynamicKafkaSourceSplit( + DynamicKafkaSourceTestHelper.getKafkaClusterId(1), + new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 10)); + reader.addSplits(ImmutableList.of(cluster0Split, cluster1Split)); + + // metadata change with a topic changing + KafkaStream kafkaStream = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + Set topicsForCluster1 = + kafkaStream.getClusterMetadataMap().get(kafkaClusterId1).getTopics(); + topicsForCluster1.clear(); + topicsForCluster1.add("new topic"); + reader.handleSourceEvents(new MetadataUpdateEvent(Collections.singleton(kafkaStream))); + // same split but earlier offset + DynamicKafkaSourceSplit newCluster0Split = + new DynamicKafkaSourceSplit( + kafkaClusterId0, + new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 10)); + // new split + DynamicKafkaSourceSplit newCluster1Split = + new DynamicKafkaSourceSplit( + kafkaClusterId1, + new KafkaPartitionSplit(new TopicPartition("new topic", 0), 10)); + reader.addSplits(ImmutableList.of(newCluster0Split, newCluster1Split)); + + List assignedSplits = reader.snapshotState(-1); + + assertThat(assignedSplits) + .as( + "The new split for cluster 1 should be assigned and split for cluster 0 should retain offset 10") + .containsExactlyInAnyOrder(cluster0Split, newCluster1Split); + } + } + + @Override + protected SourceReader createReader() { + TestingReaderContext context = new TestingReaderContext(); + return startReader(createReaderWithoutStart(context), context); + } + + private DynamicKafkaSourceReader createReaderWithoutStart( + TestingReaderContext context) { + Properties properties = getRequiredProperties(); + return new DynamicKafkaSourceReader<>( + context, + KafkaRecordDeserializationSchema.valueOnly(IntegerDeserializer.class), + properties); + } + + private SourceReader startReader( + DynamicKafkaSourceReader reader, TestingReaderContext context) { + reader.start(); + assertThat(context.getSentEvents()) + .as("Reader sends GetMetadataUpdateEvent at startup") + .hasSize(1); + reader.handleSourceEvents(DynamicKafkaSourceTestHelper.getMetadataUpdateEvent(TOPIC)); + return reader; + } + + private static Properties getRequiredProperties() { + Properties properties = new Properties(); + properties.setProperty( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + properties.setProperty( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + return properties; + } + + @Override + protected List getSplits( + int numSplits, int numRecordsPerSplit, Boundedness boundedness) { + List splits = new ArrayList<>(); + for (int i = 0; i < numSplits; i++) { + splits.add(getSplit(i, numRecordsPerSplit, boundedness)); + } + return splits; + } + + @Override + protected DynamicKafkaSourceSplit getSplit( + int splitId, int numRecords, Boundedness boundedness) { + long stoppingOffset = + boundedness == Boundedness.BOUNDED + ? NUM_RECORDS_PER_SPLIT + : KafkaPartitionSplit.NO_STOPPING_OFFSET; + + String kafkaClusterId; + int splitIdForCluster = splitId % NUM_SPLITS_PER_CLUSTER; + if (splitId < NUM_SPLITS_PER_CLUSTER) { + kafkaClusterId = "kafka-cluster-0"; + } else { + kafkaClusterId = "kafka-cluster-1"; + } + + return new DynamicKafkaSourceSplit( + kafkaClusterId, + new KafkaPartitionSplit( + new TopicPartition(TOPIC, splitIdForCluster), 0L, stoppingOffset)); + } + + @Override + protected long getNextRecordIndex(DynamicKafkaSourceSplit split) { + return split.getKafkaPartitionSplit().getStartingOffset(); + } + + private Map> splitsToClusterTopicMap(List splits) { + Map> clusterTopicMap = new HashMap<>(); + + for (DynamicKafkaSourceSplit split : splits) { + Set topics = + clusterTopicMap.computeIfAbsent( + split.getKafkaClusterId(), (ignore) -> new HashSet<>()); + topics.add(split.getKafkaPartitionSplit().getTopic()); + } + + return clusterTopicMap; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java new file mode 100644 index 000000000..4125219a8 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.dynamic.source.split; + +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; + +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * A test for {@link + * org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplitSerializer}. + */ +public class DynamicKafkaSourceSplitSerializerTest { + + @Test + public void testSerde() throws IOException { + DynamicKafkaSourceSplitSerializer serializer = new DynamicKafkaSourceSplitSerializer(); + DynamicKafkaSourceSplit dynamicKafkaSourceSplit = + new DynamicKafkaSourceSplit( + "test-cluster", + new KafkaPartitionSplit(new TopicPartition("test-topic", 3), 1)); + DynamicKafkaSourceSplit dynamicKafkaSourceSplitAfterSerde = + serializer.deserialize(1, serializer.serialize(dynamicKafkaSourceSplit)); + assertEquals(dynamicKafkaSourceSplit, dynamicKafkaSourceSplitAfterSerde); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java new file mode 100644 index 000000000..e9bc77e83 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java @@ -0,0 +1,263 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSource; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSourceBuilder; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; +import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; +import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; +import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.TopicListing; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URL; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** A external context for {@link DynamicKafkaSource} connector testing framework. */ +public class DynamicKafkaSourceExternalContext implements DataStreamSourceExternalContext { + private static final Logger logger = + LoggerFactory.getLogger(DynamicKafkaSourceExternalContext.class); + private static final int NUM_TEST_RECORDS_PER_SPLIT = 10; + private static final int NUM_PARTITIONS = 1; + + private static final Pattern STREAM_ID_PATTERN = Pattern.compile("stream-[0-9]+"); + private final List connectorJarPaths; + private final Set kafkaStreams = new HashSet<>(); + private final Map clusterPropertiesMap; + private final List splitDataWriters = new ArrayList<>(); + + // add random suffix to alleviate race conditions with Kafka deleting topics + private final long randomTopicSuffix; + + public DynamicKafkaSourceExternalContext( + List bootstrapServerList, List connectorJarPaths) { + this.connectorJarPaths = connectorJarPaths; + Properties propertiesForCluster0 = new Properties(); + propertiesForCluster0.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServerList.get(0)); + Properties propertiesForCluster1 = new Properties(); + propertiesForCluster1.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServerList.get(1)); + + this.clusterPropertiesMap = + ImmutableMap.of( + "cluster0", propertiesForCluster0, "cluster1", propertiesForCluster1); + this.randomTopicSuffix = ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + } + + @Override + public Source createSource(TestingSourceSettings sourceSettings) + throws UnsupportedOperationException { + final DynamicKafkaSourceBuilder builder = DynamicKafkaSource.builder(); + + builder.setStreamPattern(STREAM_ID_PATTERN) + .setKafkaMetadataService(new MockKafkaMetadataService(kafkaStreams)) + .setGroupId("DynamicKafkaSourceExternalContext") + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)); + + if (sourceSettings.getBoundedness().equals(Boundedness.BOUNDED)) { + builder.setBounded(OffsetsInitializer.latest()); + } + + return builder.build(); + } + + @Override + public ExternalSystemSplitDataWriter createSourceSplitDataWriter( + TestingSourceSettings sourceSettings) { + int suffix = splitDataWriters.size(); + List> clusterTopics = setupSplits(String.valueOf(suffix)); + SplitDataWriter splitDataWriter = new SplitDataWriter(clusterPropertiesMap, clusterTopics); + this.splitDataWriters.add(splitDataWriter); + return splitDataWriter; + } + + private List> setupSplits(String suffix) { + KafkaStream kafkaStream = getKafkaStream(suffix + randomTopicSuffix); + logger.info("Setting up splits for {}", kafkaStream); + List> clusterTopics = + kafkaStream.getClusterMetadataMap().entrySet().stream() + .flatMap( + entry -> + entry.getValue().getTopics().stream() + .map(topic -> Tuple2.of(entry.getKey(), topic))) + .collect(Collectors.toList()); + + for (Tuple2 clusterTopic : clusterTopics) { + String cluster = clusterTopic.f0; + String topic = clusterTopic.f1; + KafkaTestEnvironmentImpl.createNewTopic( + topic, NUM_PARTITIONS, 1, clusterPropertiesMap.get(cluster)); + } + + kafkaStreams.add(kafkaStream); + return clusterTopics; + } + + private KafkaStream getKafkaStream(String suffix) { + return new KafkaStream( + "stream-" + suffix, + ImmutableMap.of( + "cluster0", + new ClusterMetadata( + ImmutableSet.of("topic0-" + suffix, "topic1-" + suffix), + clusterPropertiesMap.get("cluster0")), + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic2-" + suffix, "topic3-" + suffix), + clusterPropertiesMap.get("cluster1")))); + } + + @Override + public List generateTestData( + TestingSourceSettings sourceSettings, int splitIndex, long seed) { + return IntStream.range(0, NUM_TEST_RECORDS_PER_SPLIT * NUM_PARTITIONS) + .boxed() + .map(num -> Integer.toString(num)) + .collect(Collectors.toList()); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(String.class); + } + + @Override + public List getConnectorJarPaths() { + return connectorJarPaths; + } + + @Override + public void close() throws Exception { + // need to clear topics + Map> clusterTopics = new HashMap<>(); + for (SplitDataWriter splitDataWriter : splitDataWriters) { + for (Tuple2 clusterTopic : splitDataWriter.getClusterTopics()) { + clusterTopics + .computeIfAbsent(clusterTopic.f0, unused -> new ArrayList<>()) + .add(clusterTopic.f1); + } + } + for (Map.Entry> entry : clusterTopics.entrySet()) { + String cluster = entry.getKey(); + List topics = entry.getValue(); + try (AdminClient adminClient = AdminClient.create(clusterPropertiesMap.get(cluster))) { + adminClient.deleteTopics(topics).all().get(); + CommonTestUtils.waitUtil( + () -> { + try { + return adminClient.listTopics().listings().get().stream() + .map(TopicListing::name) + .noneMatch(topics::contains); + } catch (Exception e) { + logger.warn("Exception caught when listing Kafka topics", e); + return false; + } + }, + Duration.ofSeconds(30), + String.format("Topics %s were not deleted within timeout", topics)); + } + + logger.info("topics {} are deleted from {}", topics, cluster); + } + } + + private static class SplitDataWriter implements ExternalSystemSplitDataWriter { + private final Map clusterPropertiesMap; + private final List> clusterTopics; + + public SplitDataWriter( + Map clusterPropertiesMap, + List> clusterTopics) { + this.clusterPropertiesMap = clusterPropertiesMap; + this.clusterTopics = clusterTopics; + } + + @Override + public void writeRecords(List records) { + int counter = 0; + try { + for (Tuple2 clusterTopic : clusterTopics) { + String cluster = clusterTopic.f0; + String topic = clusterTopic.f1; + List> producerRecords = new ArrayList<>(); + for (int j = 0; j < NUM_PARTITIONS; j++) { + for (int k = 0; k < NUM_TEST_RECORDS_PER_SPLIT; k++) { + if (records.size() <= counter) { + break; + } + + producerRecords.add( + new ProducerRecord<>(topic, j, null, records.get(counter++))); + } + } + + logger.info("Writing producer records: {}", producerRecords); + + DynamicKafkaSourceTestHelper.produceToKafka( + clusterPropertiesMap.get(cluster), + producerRecords, + StringSerializer.class, + StringSerializer.class); + } + } catch (Throwable e) { + throw new RuntimeException("Failed to produce test data", e); + } + } + + @Override + public void close() throws Exception {} + + public List> getClusterTopics() { + return clusterTopics; + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java new file mode 100644 index 000000000..71798e185 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSource; +import org.apache.flink.connector.testframe.external.ExternalContextFactory; + +import com.google.common.collect.ImmutableList; +import org.testcontainers.containers.KafkaContainer; + +import java.net.URL; +import java.util.List; +import java.util.stream.Collectors; + +/** A external context factory for {@link DynamicKafkaSource} connector testing framework. */ +public class DynamicKafkaSourceExternalContextFactory + implements ExternalContextFactory { + + private final KafkaContainer kafkaContainer0; + private final KafkaContainer kafkaContainer1; + private final List connectorJars; + + public DynamicKafkaSourceExternalContextFactory( + KafkaContainer kafkaContainer0, + KafkaContainer kafkaContainer1, + List connectorJars) { + this.kafkaContainer0 = kafkaContainer0; + this.kafkaContainer1 = kafkaContainer1; + this.connectorJars = connectorJars; + } + + @Override + public DynamicKafkaSourceExternalContext createExternalContext(String testName) { + return new DynamicKafkaSourceExternalContext( + ImmutableList.of( + getBootstrapServers(kafkaContainer0), getBootstrapServers(kafkaContainer1)), + connectorJars); + } + + private static String getBootstrapServers(KafkaContainer kafkaContainer) { + final String internalEndpoints = + kafkaContainer.getNetworkAliases().stream() + .map(host -> String.join(":", host, Integer.toString(9092))) + .collect(Collectors.joining(",")); + return String.join(",", kafkaContainer.getBootstrapServers(), internalEndpoints); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java new file mode 100644 index 000000000..18854cf90 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import com.google.common.collect.ImmutableMap; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** A mock in-memory implementation of {@link KafkaMetadataService}. */ +public class MockKafkaMetadataService implements KafkaMetadataService { + + private Set kafkaStreams; + private Set kafkaClusterIds; + private boolean throwException = false; + + public MockKafkaMetadataService(boolean throwException) { + this.throwException = throwException; + } + + public MockKafkaMetadataService(Set kafkaStreams) { + setKafkaStreams(kafkaStreams); + } + + public void setKafkaStreams(Set kafkaStreams) { + this.kafkaStreams = kafkaStreams; + this.kafkaClusterIds = + kafkaStreams.stream() + .flatMap( + kafkaStream -> + kafkaStream.getClusterMetadataMap().keySet().stream()) + .collect(Collectors.toSet()); + } + + public void setThrowException(boolean throwException) { + this.throwException = throwException; + } + + private void checkAndThrowException() { + if (throwException) { + throw new RuntimeException("Mock exception"); + } + } + + @Override + public Set getAllStreams() { + checkAndThrowException(); + return kafkaStreams; + } + + @Override + public Map describeStreams(Collection streamIds) { + checkAndThrowException(); + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (KafkaStream stream : getAllStreams()) { + if (streamIds.contains(stream.getStreamId())) { + builder.put(stream.getStreamId(), stream); + } + } + + return builder.build(); + } + + @Override + public boolean isClusterActive(String kafkaClusterId) { + checkAndThrowException(); + return kafkaClusterIds.contains(kafkaClusterId); + } + + @Override + public void close() throws Exception {} +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java new file mode 100644 index 000000000..7085a480a --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.util.DockerImageVersions; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.utility.DockerImageName; + +/** Wraps 2 Kafka containers into one for test utilities that only accept one container. */ +public class TwoKafkaContainers extends GenericContainer { + private final KafkaContainer kafka0; + private final KafkaContainer kafka1; + + public TwoKafkaContainers() { + DockerImageName dockerImageName = DockerImageName.parse(DockerImageVersions.KAFKA); + this.kafka0 = new KafkaContainer(dockerImageName); + this.kafka1 = new KafkaContainer(dockerImageName); + } + + @Override + public boolean isRunning() { + return kafka0.isRunning() && kafka1.isRunning(); + } + + @Override + public void start() { + kafka0.start(); + kafka1.start(); + } + + @Override + public void stop() { + kafka0.stop(); + kafka1.stop(); + } + + public KafkaContainer getKafka0() { + return kafka0; + } + + public KafkaContainer getKafka1() { + return kafka1; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java new file mode 100644 index 000000000..32839f379 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java @@ -0,0 +1,361 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.DumperOptions; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.TypeDescription; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.Yaml; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.constructor.Constructor; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.Node; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.SequenceNode; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.Tag; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.representer.Representer; + +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.clients.CommonClientConfigs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.time.Duration; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Reads metadata from yaml file and lazily refreshes periodically. This implementation assumes that + * specified topics exist in the clusters that are contained in the yaml metadata. Therefore, topic + * is used as the stream name. This is designed to integrate with K8s configmap and cluster + * migration. + * + *

Files must be of the form: + * + *

{@code
+ * - streamId: stream0
+ *   clusterMetadataList:
+ *     - clusterId: cluster0
+ *       bootstrapServers: bootstrap-server-0:443
+ *       topics:
+ *         - topic0
+ *         - topic1
+ *     - clusterId: cluster1
+ *       bootstrapServers: bootstrap-server-1:443
+ *       topics:
+ *         - topic2
+ *         - topic3
+ * - streamId: stream1
+ *   clusterMetadataList:
+ *     - clusterId: cluster2
+ *       bootstrapServers: bootstrap-server-2:443
+ *       topics:
+ *         - topic4
+ *         - topic5
+ * }
+ * + *

Typically, usage will look like: first consuming from one cluster, second adding new cluster + * and consuming from both clusters, and third consuming from only from the new cluster after all + * data from the old cluster has been read. + */ +public class YamlFileMetadataService implements KafkaMetadataService { + private static final Logger logger = LoggerFactory.getLogger(YamlFileMetadataService.class); + private final String metadataFilePath; + private final Duration refreshInterval; + private Instant lastRefresh; + // current metadata should be accessed from #getAllStreams() + private transient Set streamMetadata; + private transient Yaml yaml; + + /** + * Constructs a metadata service based on cluster information stored in a file. + * + * @param metadataFilePath location of the metadata file + * @param metadataTtl ttl of metadata that controls how often to refresh + */ + public YamlFileMetadataService(String metadataFilePath, Duration metadataTtl) { + this.metadataFilePath = metadataFilePath; + this.refreshInterval = metadataTtl; + this.lastRefresh = Instant.MIN; + } + + /** + * {@inheritDoc} + * + *

This obtains the all stream metadata and enforces the ttl configuration on the metadata. + */ + @Override + public Set getAllStreams() { + refreshIfNeeded(); + return streamMetadata; + } + + /** {@inheritDoc} */ + @Override + public Map describeStreams(Collection streamIds) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + Set streams = getAllStreams(); + for (KafkaStream stream : streams) { + if (streamIds.contains(stream.getStreamId())) { + builder.put(stream.getStreamId(), stream); + } + } + + return builder.build(); + } + + /** {@inheritDoc} */ + @Override + public boolean isClusterActive(String kafkaClusterId) { + return getAllStreams().stream() + .flatMap(kafkaStream -> kafkaStream.getClusterMetadataMap().keySet().stream()) + .anyMatch(cluster -> cluster.equals(kafkaClusterId)); + } + + /** {@inheritDoc} */ + @Override + public void close() throws Exception {} + + /** + * A utility method for writing metadata in the expected yaml format. + * + * @param streamMetadata list of {@link StreamMetadata} + * @param metadataFile the metadata {@link File} + */ + public static void saveToYaml(List streamMetadata, File metadataFile) + throws IOException { + logger.debug("Writing stream infos to file: {}", streamMetadata); + Yaml yaml = initYamlParser(); + FileWriter fileWriter = new FileWriter(metadataFile, false); + yaml.dump(streamMetadata, fileWriter); + fileWriter.close(); + } + + /** + * A utility method for writing metadata in the expected yaml format. + * + * @param kafkaStreams list of {@link KafkaStream} + * @param metadataFile the metadata {@link File} + */ + public static void saveToYamlFromKafkaStreams(List kafkaStreams, File metadataFile) + throws IOException { + saveToYaml( + kafkaStreams.stream() + .map(YamlFileMetadataService::convertToStreamMetadata) + .collect(Collectors.toList()), + metadataFile); + } + + private static StreamMetadata convertToStreamMetadata(KafkaStream kafkaStream) { + return new StreamMetadata( + kafkaStream.getStreamId(), + kafkaStream.getClusterMetadataMap().entrySet().stream() + .map( + entry -> + new StreamMetadata.ClusterMetadata( + entry.getKey(), + entry.getValue() + .getProperties() + .getProperty( + CommonClientConfigs + .BOOTSTRAP_SERVERS_CONFIG), + new ArrayList<>(entry.getValue().getTopics()))) + .collect(Collectors.toList())); + } + + private void refreshIfNeeded() { + Instant now = Instant.now(); + try { + if (now.isAfter(lastRefresh.plus(refreshInterval.toMillis(), ChronoUnit.MILLIS))) { + streamMetadata = parseFile(); + lastRefresh = now; + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @VisibleForTesting + Set parseFile() throws IOException { + if (yaml == null) { + yaml = initYamlParser(); + } + + List streamMetadataList = + yaml.load(Files.newInputStream(Paths.get(metadataFilePath))); + if (logger.isDebugEnabled()) { + logger.debug( + "Input stream of metadata file has size: {}", + Files.newInputStream(Paths.get(metadataFilePath)).available()); + } + Set kafkaStreams = new HashSet<>(); + + for (StreamMetadata streamMetadata : streamMetadataList) { + Map clusterMetadataMap = new HashMap<>(); + + for (StreamMetadata.ClusterMetadata clusterMetadata : + streamMetadata.getClusterMetadataList()) { + final String kafkaClusterId; + if (clusterMetadata.getClusterId() != null) { + kafkaClusterId = clusterMetadata.getClusterId(); + } else { + kafkaClusterId = clusterMetadata.getBootstrapServers(); + } + + Properties properties = new Properties(); + properties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, + clusterMetadata.getBootstrapServers()); + clusterMetadataMap.put( + kafkaClusterId, + new ClusterMetadata( + new HashSet<>(clusterMetadata.getTopics()), properties)); + } + + kafkaStreams.add(new KafkaStream(streamMetadata.getStreamId(), clusterMetadataMap)); + } + + logger.debug("From {} loaded metadata: {}", metadataFilePath, kafkaStreams); + return kafkaStreams; + } + + private static Yaml initYamlParser() { + Representer representer = new Representer(); + representer.addClassTag(StreamMetadata.class, Tag.MAP); + TypeDescription typeDescription = new TypeDescription(StreamMetadata.class); + representer.addTypeDescription(typeDescription); + representer.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK); + return new Yaml(new ListConstructor<>(StreamMetadata.class), representer); + } + + /** A custom constructor is required to read yaml lists at the root. */ + private static class ListConstructor extends Constructor { + private final Class clazz; + + public ListConstructor(final Class clazz) { + this.clazz = clazz; + } + + @Override + protected Object constructObject(final Node node) { + if (node instanceof SequenceNode && isRootNode(node)) { + ((SequenceNode) node).setListType(clazz); + } + return super.constructObject(node); + } + + private boolean isRootNode(final Node node) { + return node.getStartMark().getIndex() == 0; + } + } + + /** Internal class for snake yaml parsing. A mutable, no arg, public class is required. */ + public static class StreamMetadata { + + private String streamId; + private List clusterMetadataList; + + public StreamMetadata() {} + + public StreamMetadata(String streamId, List clusterMetadataList) { + this.streamId = streamId; + this.clusterMetadataList = clusterMetadataList; + } + + public String getStreamId() { + return streamId; + } + + public void setStreamId(String streamId) { + this.streamId = streamId; + } + + public List getClusterMetadataList() { + return clusterMetadataList; + } + + public void setClusterMetadataList(List clusterMetadata) { + this.clusterMetadataList = clusterMetadata; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("streamId", streamId) + .add("clusterMetadataList", clusterMetadataList) + .toString(); + } + + /** Information to connect to a particular cluster. */ + public static class ClusterMetadata { + private String clusterId; + private String bootstrapServers; + private List topics; + + public ClusterMetadata() {} + + public ClusterMetadata(String clusterId, String bootstrapServers, List topics) { + this.clusterId = clusterId; + this.bootstrapServers = bootstrapServers; + this.topics = topics; + } + + public String getClusterId() { + return clusterId; + } + + public void setClusterId(String clusterId) { + this.clusterId = clusterId; + } + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public List getTopics() { + return topics; + } + + public void setTopics(List topics) { + this.topics = topics; + } + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java new file mode 100644 index 000000000..f0012d181 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.io.Resources; +import org.apache.kafka.clients.CommonClientConfigs; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.time.Duration; +import java.util.Properties; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; + +/** A test class for {@link YamlFileMetadataService}. */ +public class YamlFileMetadataServiceTest { + + @Test + public void testParseFile() throws IOException { + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService( + Resources.getResource("stream-metadata.yaml").getPath(), Duration.ZERO); + Set kafkaStreams = yamlFileMetadataService.parseFile(); + + Properties propertiesForCluster0 = new Properties(); + propertiesForCluster0.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "bootstrap-server-0:443"); + Properties propertiesForCluster1 = new Properties(); + propertiesForCluster1.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "bootstrap-server-1:443"); + Properties propertiesForCluster2 = new Properties(); + propertiesForCluster2.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "bootstrap-server-2:443"); + + assertThat(kafkaStreams) + .containsExactlyInAnyOrderElementsOf( + ImmutableSet.of( + new KafkaStream( + "stream0", + ImmutableMap.of( + "cluster0", + new ClusterMetadata( + ImmutableSet.of("topic0", "topic1"), + propertiesForCluster0), + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic2", "topic3"), + propertiesForCluster1))), + new KafkaStream( + "stream1", + ImmutableMap.of( + "cluster2", + new ClusterMetadata( + ImmutableSet.of("topic4", "topic5"), + propertiesForCluster2))))); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java new file mode 100644 index 000000000..c6ecfd061 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.StringSerializer; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** Brings up multiple kafka clusters and provides utilities to setup test data. */ +public class DynamicKafkaSourceTestHelper extends KafkaTestBase { + public static final int NUM_KAFKA_CLUSTERS = 2; + + public static void setup() throws Throwable { + setNumKafkaClusters(NUM_KAFKA_CLUSTERS); + prepare(); + } + + public static void tearDown() throws Exception { + shutDownServices(); + } + + public static KafkaClusterTestEnvMetadata getKafkaClusterTestEnvMetadata(int kafkaClusterIdx) { + return kafkaClusters.get(kafkaClusterIdx); + } + + public static MetadataUpdateEvent getMetadataUpdateEvent(String topic) { + return new MetadataUpdateEvent(Collections.singleton(getKafkaStream(topic))); + } + + public static String getKafkaClusterId(int kafkaClusterIdx) { + return kafkaClusters.get(kafkaClusterIdx).getKafkaClusterId(); + } + + /** Stream is a topic across multiple clusters. */ + public static KafkaStream getKafkaStream(String topic) { + Map clusterMetadataMap = new HashMap<>(); + for (int i = 0; i < NUM_KAFKA_CLUSTERS; i++) { + KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = + getKafkaClusterTestEnvMetadata(i); + + Set topics = new HashSet<>(); + topics.add(topic); + + ClusterMetadata clusterMetadata = + new ClusterMetadata( + topics, kafkaClusterTestEnvMetadata.getStandardProperties()); + clusterMetadataMap.put( + kafkaClusterTestEnvMetadata.getKafkaClusterId(), clusterMetadata); + } + + return new KafkaStream(topic, clusterMetadataMap); + } + + public static void createTopic(String topic, int numPartitions, int replicationFactor) { + for (int i = 0; i < NUM_KAFKA_CLUSTERS; i++) { + createTopic(i, topic, numPartitions, replicationFactor); + } + } + + public static void createTopic(String topic, int numPartitions) { + createTopic(topic, numPartitions, 1); + } + + public static void createTopic(int kafkaClusterIdx, String topic, int numPartitions) { + createTopic(kafkaClusterIdx, topic, numPartitions, 1); + } + + private static void createTopic( + int kafkaClusterIdx, String topic, int numPartitions, int replicationFactor) { + kafkaClusters + .get(kafkaClusterIdx) + .getKafkaTestEnvironment() + .createTestTopic(topic, numPartitions, replicationFactor); + } + + /** Produces [0, numPartitions*numRecordsPerSplit) range of records to the specified topic. */ + public static List> produceToKafka( + String topic, int numPartitions, int numRecordsPerSplit) throws Throwable { + List> records = new ArrayList<>(); + + int counter = 0; + for (int kafkaClusterIdx = 0; kafkaClusterIdx < NUM_KAFKA_CLUSTERS; kafkaClusterIdx++) { + String kafkaClusterId = getKafkaClusterId(kafkaClusterIdx); + List> recordsForCluster = new ArrayList<>(); + for (int part = 0; part < numPartitions; part++) { + for (int i = 0; i < numRecordsPerSplit; i++) { + recordsForCluster.add( + new ProducerRecord<>( + topic, + part, + topic + "-" + part, + counter++, + Collections.singleton( + new RecordHeader( + "flink.kafka-cluster-name", + kafkaClusterId.getBytes( + StandardCharsets.UTF_8))))); + } + } + + produceToKafka(kafkaClusterIdx, recordsForCluster); + records.addAll(recordsForCluster); + } + + return records; + } + + /** + * Produces [recordValueStartingOffset, recordValueStartingOffset + + * numPartitions*numRecordsPerSplit) range of records to the specified topic and cluster. + */ + public static int produceToKafka( + int kafkaClusterIdx, + String topic, + int numPartitions, + int numRecordsPerSplit, + int recordValueStartingOffset) + throws Throwable { + int counter = recordValueStartingOffset; + String kafkaClusterId = getKafkaClusterId(kafkaClusterIdx); + List> recordsForCluster = new ArrayList<>(); + for (int part = 0; part < numPartitions; part++) { + for (int i = 0; i < numRecordsPerSplit; i++) { + recordsForCluster.add( + new ProducerRecord<>( + topic, + part, + topic + "-" + part, + counter++, + Collections.singleton( + new RecordHeader( + "flink.kafka-cluster-name", + kafkaClusterId.getBytes(StandardCharsets.UTF_8))))); + } + } + + produceToKafka(kafkaClusterIdx, recordsForCluster); + + return counter; + } + + public static void produceToKafka( + int kafkaClusterIdx, Collection> records) + throws Throwable { + produceToKafka(kafkaClusterIdx, records, StringSerializer.class, IntegerSerializer.class); + } + + public static void produceToKafka( + int id, + Collection> records, + Class> keySerializerClass, + Class> + valueSerializerClass) + throws Throwable { + produceToKafka( + kafkaClusters.get(id).getStandardProperties(), + records, + keySerializerClass, + valueSerializerClass); + } + + public static void produceToKafka( + Properties clusterProperties, + Collection> records, + Class> keySerializerClass, + Class> + valueSerializerClass) + throws Throwable { + Properties props = new Properties(); + props.putAll(clusterProperties); + props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass.getName()); + props.setProperty( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName()); + + AtomicReference sendingError = new AtomicReference<>(); + Callback callback = + (metadata, exception) -> { + if (exception != null) { + if (!sendingError.compareAndSet(null, exception)) { + sendingError.get().addSuppressed(exception); + } + } + }; + try (KafkaProducer producer = new KafkaProducer<>(props)) { + for (ProducerRecord record : records) { + producer.send(record, callback).get(); + } + } + if (sendingError.get() != null) { + throw sendingError.get(); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index b4fd54ab2..deafb7d6d 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -32,6 +32,7 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLogger; +import com.google.common.base.MoreObjects; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; @@ -83,6 +84,7 @@ public abstract class KafkaTestBase extends TestLogger { public static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class); public static final int NUMBER_OF_KAFKA_SERVERS = 1; + private static int numKafkaClusters = 1; public static String brokerConnectionStrings; @@ -92,6 +94,8 @@ public abstract class KafkaTestBase extends TestLogger { public static KafkaTestEnvironment kafkaServer; + public static List kafkaClusters = new ArrayList<>(); + @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder(); public static Properties secureProps = new Properties(); @@ -108,7 +112,7 @@ public static void prepare() throws Exception { LOG.info(" Starting KafkaTestBase "); LOG.info("-------------------------------------------------------------------------"); - startClusters(false); + startClusters(false, numKafkaClusters); } @AfterClass @@ -140,11 +144,21 @@ public static void startClusters() throws Exception { KafkaTestEnvironment.createConfig().setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS)); } - public static void startClusters(boolean secureMode) throws Exception { + public static void startClusters(boolean secureMode, int numKafkaClusters) throws Exception { startClusters( - KafkaTestEnvironment.createConfig() - .setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS) - .setSecureMode(secureMode)); + KafkaTestEnvironment.createConfig().setSecureMode(secureMode), numKafkaClusters); + } + + public static void startClusters( + KafkaTestEnvironment.Config environmentConfig, int numKafkaClusters) throws Exception { + for (int i = 0; i < numKafkaClusters; i++) { + startClusters(environmentConfig); + KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = + new KafkaClusterTestEnvMetadata( + i, kafkaServer, standardProps, brokerConnectionStrings, secureProps); + kafkaClusters.add(kafkaClusterTestEnvMetadata); + LOG.info("Created Kafka cluster with configuration: {}", kafkaClusterTestEnvMetadata); + } } public static void startClusters(KafkaTestEnvironment.Config environmentConfig) @@ -183,6 +197,13 @@ public static void shutdownClusters() throws Exception { if (kafkaServer != null) { kafkaServer.shutdown(); } + + if (kafkaClusters != null && !kafkaClusters.isEmpty()) { + for (KafkaClusterTestEnvMetadata value : kafkaClusters) { + value.getKafkaTestEnvironment().shutdown(); + } + kafkaClusters.clear(); + } } // ------------------------------------------------------------------------ @@ -338,4 +359,62 @@ private String formatElements(List elements) { return String.format("elements: <%s>", elements); } } + + public static void setNumKafkaClusters(int size) { + numKafkaClusters = size; + } + + /** Metadata generated by this test utility. */ + public static class KafkaClusterTestEnvMetadata { + + private final String kafkaClusterId; + private final KafkaTestEnvironment kafkaTestEnvironment; + private final Properties standardProperties; + private final String brokerConnectionStrings; + private final Properties secureProperties; + + private KafkaClusterTestEnvMetadata( + int kafkaClusterIdx, + KafkaTestEnvironment kafkaTestEnvironment, + Properties standardProperties, + String brokerConnectionStrings, + Properties secureProperties) { + this.kafkaClusterId = "kafka-cluster-" + kafkaClusterIdx; + this.kafkaTestEnvironment = kafkaTestEnvironment; + this.standardProperties = standardProperties; + this.brokerConnectionStrings = brokerConnectionStrings; + this.secureProperties = secureProperties; + } + + public String getKafkaClusterId() { + return kafkaClusterId; + } + + public KafkaTestEnvironment getKafkaTestEnvironment() { + return kafkaTestEnvironment; + } + + public Properties getStandardProperties() { + return standardProperties; + } + + public String getBrokerConnectionStrings() { + return brokerConnectionStrings; + } + + public Properties getSecureProperties() { + return secureProperties; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("kafkaClusterId", kafkaClusterId) + .add("kafkaTestEnvironment", kafkaTestEnvironment) + .add("standardProperties", standardProperties) + .add("brokerConnectionStrings", brokerConnectionStrings) + .add("secureProperties", secureProperties) + .toString(); + } + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java index ee38e8501..6687cd525 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java @@ -39,6 +39,8 @@ public abstract class KafkaTestEnvironment { /** Configuration class for {@link KafkaTestEnvironment}. */ public static class Config { + + private int numKafkaClusters = 1; private int kafkaServersNumber = 1; private Properties kafkaServerProperties = null; private boolean secureMode = false; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index c5bc3b003..b776cad22 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -169,8 +169,13 @@ private void tryDelete(AdminClient adminClient, String topic) throws Exception { @Override public void createTestTopic( String topic, int numberOfPartitions, int replicationFactor, Properties properties) { + createNewTopic(topic, numberOfPartitions, replicationFactor, getStandardProperties()); + } + + public static void createNewTopic( + String topic, int numberOfPartitions, int replicationFactor, Properties properties) { LOG.info("Creating topic {}", topic); - try (AdminClient adminClient = AdminClient.create(getStandardProperties())) { + try (AdminClient adminClient = AdminClient.create(properties)) { NewTopic topicObj = new NewTopic(topic, numberOfPartitions, (short) replicationFactor); adminClient.createTopics(Collections.singleton(topicObj)).all().get(); CommonTestUtils.waitUtil( diff --git a/flink-connector-kafka/src/test/resources/log4j2-test.properties b/flink-connector-kafka/src/test/resources/log4j2-test.properties index 863665cf4..3d901331b 100644 --- a/flink-connector-kafka/src/test/resources/log4j2-test.properties +++ b/flink-connector-kafka/src/test/resources/log4j2-test.properties @@ -36,3 +36,6 @@ logger.zookeeper.name = org.apache.zookeeper logger.zookeeper.level = OFF logger.I0Itec.name = org.I0Itec logger.I0Itec.level = OFF + +logger.splitreader.name = org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader +logger.splitreader.level = DEBUG diff --git a/flink-connector-kafka/src/test/resources/stream-metadata.yaml b/flink-connector-kafka/src/test/resources/stream-metadata.yaml new file mode 100644 index 000000000..9502351b5 --- /dev/null +++ b/flink-connector-kafka/src/test/resources/stream-metadata.yaml @@ -0,0 +1,19 @@ +- streamId: stream0 + clusterMetadataList: + - clusterId: cluster0 + bootstrapServers: bootstrap-server-0:443 + topics: + - topic0 + - topic1 + - clusterId: cluster1 + bootstrapServers: bootstrap-server-1:443 + topics: + - topic2 + - topic3 +- streamId: stream1 + clusterMetadataList: + - clusterId: cluster2 + bootstrapServers: bootstrap-server-2:443 + topics: + - topic4 + - topic5 diff --git a/pom.xml b/pom.xml index b70e98c20..f3131759c 100644 --- a/pom.xml +++ b/pom.xml @@ -70,6 +70,7 @@ under the License. 2.12.7 1.1.10.5 1.11.3 + 32.1.2-jre false 1.17.0 @@ -271,6 +272,12 @@ under the License. 3.3.2 + + com.google.guava + guava + ${guava.version} + + org.apache.flink diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml index 2048fd186..7d1f9fd71 100644 --- a/tools/maven/checkstyle.xml +++ b/tools/maven/checkstyle.xml @@ -224,10 +224,6 @@ This file is based on the checkstyle file of Apache Beam. - - - - + +# Dynamic Kafka Source _`Experimental`_ + +Flink provides an [Apache Kafka](https://kafka.apache.org) connector for reading data from Kafka topics from one or more Kafka clusters. +The Dynamic Kafka connector discovers the clusters and topics using a Kafka metadata service and can achieve reading in a dynamic fashion, facilitating changes in +topics and/or clusters, without requiring a job restart. This is especially useful when you need to read a new Kafka cluster/topic and/or stop reading +an existing Kafka cluster/topic (cluster migration/failover/other infrastructure changes) and when you need direct integration with Hybrid Source. The solution +makes these operations automated so that they are transparent to Kafka consumers. + +## Dependency + +For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). + +{{< connector_artifact flink-connector-kafka 3.1.0 >}} + +Flink's streaming connectors are not part of the binary distribution. +See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). + +## Dynamic Kafka Source +{{< hint info >}} +This part describes the Dynamic Kafka Source based on the new +[data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. +{{< /hint >}} + +### Usage + +Dynamic Kafka Source provides a builder class to initialize the DynamicKafkaSource. The code snippet +below shows how to build a DynamicKafkaSource to consume messages from the earliest offset of the +stream "input-stream" and deserialize only the value of the +ConsumerRecord as a string, using "MyKafkaMetadataService" to resolve the cluster(s) and topic(s) +corresponding to "input-stream". + +{{< tabs "DynamicKafkaSource" >}} +{{< tab "Java" >}} +```java + +DynamicKafkaSource source = DynamicKafkaSource.builder() + .setKafkaMetadataService(new MyKafkaMetadataService()) + .setStreamIds(Collections.singleton("input-stream")) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) + .setProperties(properties) + .build(); + + env.fromSource(source, WatermarkStrategy.noWatermarks(), "Dynamic Kafka Source"); +``` +{{< /tab >}} +{{< /tabs >}} +The following properties are **required** for building a DynamicKafkaSource: + +The Kafka metadata service, configured by setKafkaMetadataService(KafkaMetadataService) +The stream ids to subscribe, see the following Kafka stream subscription section for more details. +Deserializer to parse Kafka messages, see the [Kafka Source Documentation]({{< ref "docs/connectors/datastream/kafka" >}}#deserializer) for more details. + +### Kafka Stream Subscription +The Dynamic Kafka Source provides 2 ways of subscribing to Kafka stream(s). +* A set of Kafka stream ids. For example: + {{< tabs "DynamicKafkaSource#setStreamIds" >}} + {{< tab "Java" >}} + ```java + DynamicKafkaSource.builder().setStreamIds(Set.of("stream-a", "stream-b")); + ``` + {{< /tab >}} + {{< /tabs >}} +* A regex pattern that subscribes to all Kafka stream ids that match the provided regex. For example: + {{< tabs "DynamicKafkaSource#setStreamPattern" >}} + {{< tab "Java" >}} + ```java + DynamicKafkaSource.builder().setStreamPattern(Pattern.of("stream.*")); + ``` + {{< /tab >}} + {{< /tabs >}} + +### Kafka Metadata Service + +An interface is provided to resolve the logical Kafka stream(s) into the corresponding physical +topic(s) and cluster(s). Typically, these implementations are based on services that align well +with internal Kafka infrastructure--if that is not available, an in-memory implementation +would also work. An example of in-memory implementation can be found in our tests. + +This source achieves its dynamic characteristic by periodically polling this Kafka metadata service +for any changes to the Kafka stream(s) and reconciling the reader tasks to subscribe to the new +Kafka metadata returned by the service. For example, in the case of a Kafka migration, the source would +swap from one cluster to the new cluster when the service makes that change in the Kafka stream metadata. + +### Additional Properties +There are configuration options in DynamicKafkaSourceOptions that can be configured in the properties through the builder: + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequiredDefaultTypeDescription
stream-metadata-discovery-interval-ms
required-1LongThe interval in milliseconds for the source to discover the changes in stream metadata. A non-positive value disables the stream metadata discovery.
stream-metadata-discovery-failure-threshold
required1IntegerThe number of consecutive failures before letting the exception from Kafka metadata service discovery trigger jobmanager failure and global failover. The default is one to at least catch startup failures.
+ + +In addition to this list, see the [regular Kafka connector]({{< ref "docs/connectors/datastream/kafka" >}}#additional-properties) for +a list of applicable properties. + +### Metrics + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ScopeMetricsUser VariablesDescriptionType
OperatorcurrentEmitEventTimeLagn/aThe time span from the record event timestamp to the time the record is emitted by the source connector¹: currentEmitEventTimeLag = EmitTime - EventTime.Gauge
watermarkLagn/aThe time span that the watermark lags behind the wall clock time: watermarkLag = CurrentTime - WatermarkGauge
sourceIdleTimen/aThe time span that the source has not processed any record: sourceIdleTime = CurrentTime - LastRecordProcessTimeGauge
pendingRecordsn/aThe number of records that have not been fetched by the source. e.g. the available records after the consumer offset in a Kafka partition.Gauge
kafkaClustersCountn/aThe total number of Kafka clusters read by this reader.Gauge
+ +In addition to this list, see the [regular Kafka connector]({{< ref "docs/connectors/datastream/kafka" >}}#monitoring) for +the KafkaSourceReader metrics that are also reported. + +### Additional Details + +For additional details on deserialization, event time and watermarks, idleness, consumer offset +committing, security, and more, you can refer to the [Kafka Source documentation]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-source). This is possible because the +Dynamic Kafka Source leverages components of the Kafka Source, and the implementation will be +discussed in the next section. + +### Behind the Scene +{{< hint info >}} +If you are interested in how Kafka source works under the design of new data source API, you may +want to read this part as a reference. For details about the new data source API, +[documentation of data source]({{< ref "docs/dev/datastream/sources.md" >}}) and +FLIP-27 +provide more descriptive discussions. +{{< /hint >}} + + +Under the abstraction of the new data source API, Dynamic Kafka Source consists of the following components: +#### Source Split +A source split in Dynamic Kafka Source represents a partition of a Kafka topic, with cluster information. It +consists of: +* A Kafka cluster id that can be resolved by the Kafka metadata service. +* A Kafka Source Split (TopicPartition, starting offset, stopping offset). + +You can check the class `DynamicKafkaSourceSplit` for more details. + +#### Split Enumerator + +This enumerator is responsible for discovering and assigning splits from one or more clusters. At startup, the +enumerator will discover metadata belonging to the Kafka stream ids. Using the metadata, it can +initialize KafkaSourceEnumerators to handle the functions of assigning splits to the readers. In addition, +source events will be sent to the source reader to reconcile the metadata. This enumerator has the ability to poll the +KafkaMetadataService, periodically for stream discovery. In addition, restarting enumerators when metadata changes involve +clearing outdated metrics since clusters may be removed and so should their metrics. + +#### Source Reader + +This reader is responsible for reading from one or more clusters and using the KafkaSourceReader to fetch +records from topics and clusters based on the metadata. When new metadata is discovered by the enumerator, +the reader will reconcile metadata changes to possibly restart the KafkaSourceReader to read from the new +set of topics and clusters. + +#### Kafka Metadata Service + +This interface represents the source of truth for the current metadata for the configured Kafka stream ids. +Metadata that is removed in between polls is considered non-active (e.g. removing a cluster from the +return value, means that a cluster is non-active and should not be read from). The cluster metadata +contains an immutable Kafka cluster id, the set of topics, and properties needed to connect to the +Kafka cluster. + +#### FLIP 246 + +To understand more behind the scenes, please read [FLIP-246](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320) +for more details and discussion. diff --git a/docs/content.zh/docs/connectors/table/dynamic-kafka.md b/docs/content.zh/docs/connectors/table/dynamic-kafka.md deleted file mode 100644 index f00a2f226..000000000 --- a/docs/content.zh/docs/connectors/table/dynamic-kafka.md +++ /dev/null @@ -1,141 +0,0 @@ ---- -title: Kafka -weight: 3 -type: docs -aliases: - - /zh/dev/connectors/dynamic-kafka.html ---- - - -# Dynamic Kafka Source _`Experimental`_ - -Flink provides an [Apache Kafka](https://kafka.apache.org) connector for reading data from and -writing data to Kafka topics from one or more Kafka clusters. This connector achieves this in a dynamic -fashion, without requiring a job restart, using a Kafka metadata service to facilitate changes in -topics and/or clusters. This is especially useful in transparent Kafka cluster addition/removal without -Flink job restart, transparent Kafka topic addition/removal without Flink job restart, and direct integration -with Hybrid Source. - -## Dependency - -For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). - -{{< connector_artifact flink-connector-kafka 3.1.0 >}} - -Flink's streaming connectors are not part of the binary distribution. -See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). - -## Dynamic Kafka Source -{{< hint info >}} -This part describes the Dynamic Kafka Source based on the new -[data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. -{{< /hint >}} - -## Usage - -Dynamic Kafka Source provides a builder class to initialize the DynamicKafkaSource. The code snippet -below shows how to build a DynamicKafkaSource to consume messages from the earliest offset of the -stream "input-stream" and deserialize only the value of the -ConsumerRecord as a string, using "MyKafkaMetadataService" to resolve the cluster(s) and topic(s) -corresponding to "input-stream". - -{{< tabs "KafkaSource" >}} -{{< tab "Java" >}} -```java - -DynamicKafkaSource source = DynamicKafkaSource.builder() - .setKafkaMetadataService(new MyKafkaMetadataService()) - .setStreamIds(Collections.singleton("input-stream")) - .setStartingOffsets(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) - .setDeserializer(new SimpleStringSchema()) - .setProperties(properties) - .build(); - -env.fromSource(source, WatermarkStrategy.noWatermarks(), "Kafka Source"); -``` -{{< /tab >}} -{{< /tabs >}} - -### Kafka Metadata Service - -An interface is provided to resolve the logical Kafka stream(s) into the corresponding physical -topic(s) and cluster(s). Typically, these implementations are based on services that align well -with internal Kafka infrastructure--if that is not available, an in-memory implementation -would also work. An example of in-memory implementation can be found in our tests. - -This source achieves its dynamic characteristic by periodically polling this Kafka metadata service -for any changes to the Kafka stream(s) and reconciling the reader tasks to subscribe to the new -Kafka metadata returned by the service. For example, in the case of a Kafka migration, the source would -swap from one cluster to the new cluster when the service makes that change in the Kafka stream metadata. - -### Additional Details - -For additional details on deserialization, event time and watermarks, idleness, consumer offset -committing, security, and more, you can refer to the Kafka Source documentation. This is possible because the -Dynamic Kafka Source leverages components of the Kafka Source, and the implementation will be -discussed in the next section. - -### Behind the Scene -{{< hint info >}} -If you are interested in how Kafka source works under the design of new data source API, you may -want to read this part as a reference. For details about the new data source API, -[documentation of data source]({{< ref "docs/dev/datastream/sources.md" >}}) and -FLIP-27 -provide more descriptive discussions. -{{< /hint >}} - - -Under the abstraction of the new data source API, Dynamic Kafka Source consists of the following components: -#### Source Split -A source split in Dynamic Kafka Source represents a partition of a Kafka topic, with cluster information. It -consists of: -* A Kafka cluster id that can be resolved by the Kafka metadata service. -* A Kafka Source Split (TopicPartition, starting offset, stopping offset). - -You can check the class `DynamicKafkaSourceSplit` for more details. - -#### Split Enumerator - -This enumerator is responsible for discovering and assigning splits from 1+ cluster. At startup, the -enumerator will discover metadata belonging to the Kafka stream ids. Using the metadata, it can -initialize KafkaSourceEnumerators to handle the functions of assigning splits to the readers. In addition, -source events will be sent to the source reader to reconcile the metadata. This enumerator has the ability to poll the -KafkaMetadataService, periodically for stream discovery. In addition, restarting enumerators when metadata changes involve -clearing outdated metrics since clusters may be removed and so should their metrics. - -#### Source Reader - -This reader is responsible for reading from 1+ clusters and using the KafkaSourceReader to fetch -records from topics and clusters based on the metadata. When new metadata is discovered by the enumerator, -the reader will reconcile metadata changes to possibly restart the KafkaSourceReader to read from the new -set of topics and clusters. - -#### Kafka Metadata Service - -This interface represents the source of truth for the current metadata for the configured Kafka stream ids. -Metadata that is removed in between polls is considered non-active (e.g. removing a cluster from the -return value, means that a cluster is non-active and should not be read from). The cluster metadata -contains an immutable Kafka cluster id, the set of topics, and properties needed to connect to the -Kafka cluster. - -#### FLIP 246 - -To understand more behind the scenes, please read [FLIP-246](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320) -for more details and discussion. diff --git a/docs/content/docs/connectors/datastream/dynamic-kafka.md b/docs/content/docs/connectors/datastream/dynamic-kafka.md index 903a26d6b..4c6e38fc3 100644 --- a/docs/content/docs/connectors/datastream/dynamic-kafka.md +++ b/docs/content/docs/connectors/datastream/dynamic-kafka.md @@ -1,5 +1,5 @@ --- -title: Kafka +title: Dynamic Kafka weight: 3 type: docs aliases: @@ -26,12 +26,11 @@ under the License. # Dynamic Kafka Source _`Experimental`_ -Flink provides an [Apache Kafka](https://kafka.apache.org) connector for reading data from and -writing data to Kafka topics from one or more Kafka clusters. This connector achieves this in a dynamic -fashion, without requiring a job restart, using a Kafka metadata service to facilitate changes in -topics and/or clusters. This is especially useful in transparent Kafka cluster addition/removal without -Flink job restart, transparent Kafka topic addition/removal without Flink job restart, and direct integration -with Hybrid Source. +Flink provides an [Apache Kafka](https://kafka.apache.org) connector for reading data from Kafka topics from one or more Kafka clusters. +The Dynamic Kafka connector discovers the clusters and topics using a Kafka metadata service and can achieve reading in a dynamic fashion, facilitating changes in +topics and/or clusters, without requiring a job restart. This is especially useful when you need to read a new Kafka cluster/topic and/or stop reading +an existing Kafka cluster/topic (cluster migration/failover/other infrastructure changes) and when you need direct integration with Hybrid Source. The solution +makes these operations automated so that they are transparent to Kafka consumers. ## Dependency @@ -48,7 +47,7 @@ This part describes the Dynamic Kafka Source based on the new [data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. {{< /hint >}} -## Usage +### Usage Dynamic Kafka Source provides a builder class to initialize the DynamicKafkaSource. The code snippet below shows how to build a DynamicKafkaSource to consume messages from the earliest offset of the @@ -56,22 +55,46 @@ stream "input-stream" and deserialize only the value of the ConsumerRecord as a string, using "MyKafkaMetadataService" to resolve the cluster(s) and topic(s) corresponding to "input-stream". -{{< tabs "KafkaSource" >}} +{{< tabs "DynamicKafkaSource" >}} {{< tab "Java" >}} ```java DynamicKafkaSource source = DynamicKafkaSource.builder() .setKafkaMetadataService(new MyKafkaMetadataService()) .setStreamIds(Collections.singleton("input-stream")) - .setStartingOffsets(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) - .setDeserializer(new SimpleStringSchema()) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) .setProperties(properties) .build(); -env.fromSource(source, WatermarkStrategy.noWatermarks(), "Kafka Source"); +env.fromSource(source, WatermarkStrategy.noWatermarks(), "Dynamic Kafka Source"); ``` {{< /tab >}} {{< /tabs >}} +The following properties are **required** for building a DynamicKafkaSource: + +The Kafka metadata service, configured by setKafkaMetadataService(KafkaMetadataService) +The stream ids to subscribe, see the following Kafka stream subscription section for more details. +Deserializer to parse Kafka messages, see the [Kafka Source Documentation]({{< ref "docs/connectors/datastream/kafka" >}}#deserializer) for more details. + +### Kafka Stream Subscription +The Dynamic Kafka Source provides 2 ways of subscribing to Kafka stream(s). +* A set of Kafka stream ids. For example: + {{< tabs "DynamicKafkaSource#setStreamIds" >}} + {{< tab "Java" >}} + ```java + DynamicKafkaSource.builder().setStreamIds(Set.of("stream-a", "stream-b")); + ``` + {{< /tab >}} + {{< /tabs >}} +* A regex pattern that subscribes to all Kafka stream ids that match the provided regex. For example: + {{< tabs "DynamicKafkaSource#setStreamPattern" >}} + {{< tab "Java" >}} + ```java + DynamicKafkaSource.builder().setStreamPattern(Pattern.of("stream.*")); + ``` + {{< /tab >}} + {{< /tabs >}} ### Kafka Metadata Service @@ -85,10 +108,94 @@ for any changes to the Kafka stream(s) and reconciling the reader tasks to subsc Kafka metadata returned by the service. For example, in the case of a Kafka migration, the source would swap from one cluster to the new cluster when the service makes that change in the Kafka stream metadata. +### Additional Properties +There are configuration options in DynamicKafkaSourceOptions that can be configured in the properties through the builder: + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequiredDefaultTypeDescription
stream-metadata-discovery-interval-ms
required-1LongThe interval in milliseconds for the source to discover the changes in stream metadata. A non-positive value disables the stream metadata discovery.
stream-metadata-discovery-failure-threshold
required1IntegerThe number of consecutive failures before letting the exception from Kafka metadata service discovery trigger jobmanager failure and global failover. The default is one to at least catch startup failures.
+ + +In addition to this list, see the [regular Kafka connector]({{< ref "docs/connectors/datastream/kafka" >}}#additional-properties) for +a list of applicable properties. + +### Metrics + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ScopeMetricsUser VariablesDescriptionType
OperatorcurrentEmitEventTimeLagn/aThe time span from the record event timestamp to the time the record is emitted by the source connector¹: currentEmitEventTimeLag = EmitTime - EventTime.Gauge
watermarkLagn/aThe time span that the watermark lags behind the wall clock time: watermarkLag = CurrentTime - WatermarkGauge
sourceIdleTimen/aThe time span that the source has not processed any record: sourceIdleTime = CurrentTime - LastRecordProcessTimeGauge
pendingRecordsn/aThe number of records that have not been fetched by the source. e.g. the available records after the consumer offset in a Kafka partition.Gauge
kafkaClustersCountn/aThe total number of Kafka clusters read by this reader.Gauge
+ +In addition to this list, see the [regular Kafka connector]({{< ref "docs/connectors/datastream/kafka" >}}#monitoring) for +the KafkaSourceReader metrics that are also reported. + ### Additional Details For additional details on deserialization, event time and watermarks, idleness, consumer offset -committing, security, and more, you can refer to the Kafka Source documentation. This is possible because the +committing, security, and more, you can refer to the [Kafka Source documentation]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-source). This is possible because the Dynamic Kafka Source leverages components of the Kafka Source, and the implementation will be discussed in the next section. @@ -113,7 +220,7 @@ You can check the class `DynamicKafkaSourceSplit` for more details. #### Split Enumerator -This enumerator is responsible for discovering and assigning splits from 1+ cluster. At startup, the +This enumerator is responsible for discovering and assigning splits from one or more clusters. At startup, the enumerator will discover metadata belonging to the Kafka stream ids. Using the metadata, it can initialize KafkaSourceEnumerators to handle the functions of assigning splits to the readers. In addition, source events will be sent to the source reader to reconcile the metadata. This enumerator has the ability to poll the @@ -122,7 +229,7 @@ clearing outdated metrics since clusters may be removed and so should their metr #### Source Reader -This reader is responsible for reading from 1+ clusters and using the KafkaSourceReader to fetch +This reader is responsible for reading from one or more clusters and using the KafkaSourceReader to fetch records from topics and clusters based on the metadata. When new metadata is discovered by the enumerator, the reader will reconcile metadata changes to possibly restart the KafkaSourceReader to read from the new set of topics and clusters. diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java index 074798ce8..bdecaf390 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java @@ -39,7 +39,7 @@ private DynamicKafkaSourceOptions() {} .longType() .defaultValue(-1L) .withDescription( - "The interval in milliseconds for the sink to discover " + "The interval in milliseconds for the source to discover " + "the changes in stream metadata. A non-positive value disables the stream metadata discovery."); public static final ConfigOption STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD = @@ -49,7 +49,7 @@ private DynamicKafkaSourceOptions() {} .withDescription( "The number of consecutive failures before letting the exception from Kafka metadata service discovery " + "trigger jobmanager failure and global failover. The default is one to at least catch startup " - + "failures. This is only implemented for the source"); + + "failures."); @Internal public static T getOption( From fd7b46c5431ba9e384543f7fe7a1ad9f45030f2b Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Fri, 26 Jan 2024 13:00:30 +0100 Subject: [PATCH 254/322] [release] Update weekly CI to check against new release branch `v3.1` --- .github/workflows/weekly.yml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index e6bf27dda..850ca5b27 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -38,10 +38,16 @@ jobs: jdk: '8, 11, 17, 21', branch: main }, { - flink: 1.17.1, + flink: 1.17.2, + branch: v3.1 + }, { + flink: 1.18.1, + branch: v3.1 + }, { + flink: 1.17.2, branch: v3.0 }, { - flink: 1.18.0, + flink: 1.18.1, branch: v3.0 }] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils From 68ac980b777593f4a14489e93684f3b4e9a1922b Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Fri, 26 Jan 2024 13:02:26 +0100 Subject: [PATCH 255/322] [hotfix] Remove weekly test against 1.19-SNAPSHOT since that's currently broken and tracked under FLINK-34193 --- .github/workflows/weekly.yml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index 850ca5b27..8d23c79de 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -33,10 +33,6 @@ jobs: flink: 1.18-SNAPSHOT, jdk: '8, 11, 17', branch: main - }, { - flink: 1.19-SNAPSHOT, - jdk: '8, 11, 17, 21', - branch: main }, { flink: 1.17.2, branch: v3.1 From a468c2b766acf8d8cdaa4f64122497632cdaaf39 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Fri, 26 Jan 2024 13:04:03 +0100 Subject: [PATCH 256/322] [hotfix] Add missing JDK17 test for 1.18 against v3.1 --- .github/workflows/weekly.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index 8d23c79de..21462eb7e 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -38,6 +38,7 @@ jobs: branch: v3.1 }, { flink: 1.18.1, + jdk: '8, 11, 17', branch: v3.1 }, { flink: 1.17.2, From 624f0a13c81dcbfc5f6c2b05a088a23e4c04e1f8 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Fri, 26 Jan 2024 13:43:46 +0100 Subject: [PATCH 257/322] [hotfix] Update copyright year to 2024 --- NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/NOTICE b/NOTICE index 1914984d3..a0ed01945 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Flink Kafka Connector -Copyright 2014-2023 The Apache Software Foundation +Copyright 2014-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From abf4563e0342abe25dc28bb6b5457bb971381f61 Mon Sep 17 00:00:00 2001 From: Hang Ruan Date: Tue, 30 Jan 2024 09:53:58 +0800 Subject: [PATCH 258/322] [hotfix] Update copyright year to 2024 (#82) --- flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE b/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE index 1ca013b7b..926976da9 100644 --- a/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE +++ b/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-kafka -Copyright 2014-2023 The Apache Software Foundation +Copyright 2014-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From ab356b4d0232b47ae6b7b507b3a9f1ccca862b98 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Wed, 7 Feb 2024 11:03:02 +0100 Subject: [PATCH 259/322] [hotfix] Make the upsert-kafka artifacts point to the regular kafka artifacts in the documentation --- docs/content.zh/docs/connectors/table/upsert-kafka.md | 2 +- docs/content/docs/connectors/table/upsert-kafka.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/upsert-kafka.md b/docs/content.zh/docs/connectors/table/upsert-kafka.md index a7e571fd8..9612de0b8 100644 --- a/docs/content.zh/docs/connectors/table/upsert-kafka.md +++ b/docs/content.zh/docs/connectors/table/upsert-kafka.md @@ -38,7 +38,7 @@ Upsert Kafka 连接器支持以 upsert 方式从 Kafka topic 中读取数据并 依赖 ------------ -{{< sql_download_table "upsert-kafka" >}} +{{< sql_connector_download_table "kafka" >}} Upsert Kafka 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Upsert Kafka 连接器。 diff --git a/docs/content/docs/connectors/table/upsert-kafka.md b/docs/content/docs/connectors/table/upsert-kafka.md index 612376402..814ff9bcb 100644 --- a/docs/content/docs/connectors/table/upsert-kafka.md +++ b/docs/content/docs/connectors/table/upsert-kafka.md @@ -47,7 +47,7 @@ key will fall into the same partition. Dependencies ------------ -{{< sql_connector_download_table "upsert-kafka" >}} +{{< sql_connector_download_table "kafka" >}} The Upsert Kafka connector is not part of the binary distribution. See how to link with it for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). From 6d0ffcd7f4bfa58a8ede1404d4c082614be4c753 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Wed, 7 Feb 2024 11:12:13 +0100 Subject: [PATCH 260/322] [hotfix] Update shortcode in Kafka Datastream documentation so that the download links appear --- docs/content.zh/docs/connectors/datastream/dynamic-kafka.md | 2 +- docs/content.zh/docs/connectors/datastream/kafka.md | 2 +- docs/content/docs/connectors/datastream/dynamic-kafka.md | 2 +- docs/content/docs/connectors/datastream/kafka.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md b/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md index e46adca36..39adb936c 100644 --- a/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md +++ b/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md @@ -36,7 +36,7 @@ makes these operations automated so that they are transparent to Kafka consumers For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). -{{< connector_artifact flink-connector-kafka 3.1.0 >}} +{{< connector_artifact flink-connector-kafka kafka >}} Flink's streaming connectors are not part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). diff --git a/docs/content.zh/docs/connectors/datastream/kafka.md b/docs/content.zh/docs/connectors/datastream/kafka.md index 35fd281f8..9c54aec7d 100644 --- a/docs/content.zh/docs/connectors/datastream/kafka.md +++ b/docs/content.zh/docs/connectors/datastream/kafka.md @@ -36,7 +36,7 @@ Apache Flink 集成了通用的 Kafka 连接器,它会尽力与 Kafka client 当前 Kafka client 向后兼容 0.10.0 或更高版本的 Kafka broker。 有关 Kafka 兼容性的更多细节,请参考 [Kafka 官方文档](https://kafka.apache.org/protocol.html#protocol_compatibility)。 -{{< connector_artifact flink-connector-kafka 3.0.0 >}} +{{< connector_artifact flink-connector-kafka kafka >}} 如果使用 Kafka source,```flink-connector-base``` 也需要包含在依赖中: diff --git a/docs/content/docs/connectors/datastream/dynamic-kafka.md b/docs/content/docs/connectors/datastream/dynamic-kafka.md index 4c6e38fc3..08fa2401e 100644 --- a/docs/content/docs/connectors/datastream/dynamic-kafka.md +++ b/docs/content/docs/connectors/datastream/dynamic-kafka.md @@ -36,7 +36,7 @@ makes these operations automated so that they are transparent to Kafka consumers For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). -{{< connector_artifact flink-connector-kafka 3.1.0 >}} +{{< connector_artifact flink-connector-kafka kafka >}} Flink's streaming connectors are not part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). diff --git a/docs/content/docs/connectors/datastream/kafka.md b/docs/content/docs/connectors/datastream/kafka.md index 422ed9e36..0ab35af6a 100644 --- a/docs/content/docs/connectors/datastream/kafka.md +++ b/docs/content/docs/connectors/datastream/kafka.md @@ -36,7 +36,7 @@ The version of the client it uses may change between Flink releases. Modern Kafka clients are backwards compatible with broker versions 0.10.0 or later. For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). -{{< connector_artifact flink-connector-kafka 3.0.0 >}} +{{< connector_artifact flink-connector-kafka kafka >}} Flink's streaming connectors are not part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). From cfb275b478ff97e9105c5ffaf20224f59a89ebd7 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Fri, 9 Feb 2024 08:52:36 +0100 Subject: [PATCH 261/322] [FLINK-34244] Update Confluent Platform to 7.4.4. This closes #81 * Make sure that all tests use the central DockerImageVersions * Update Confluent Platform to 7.4.4 --- .../apache/flink/tests/util/kafka/KafkaSinkE2ECase.java | 2 +- .../apache/flink/tests/util/kafka/KafkaSourceE2ECase.java | 2 +- .../tests/util/kafka/SQLClientSchemaRegistryITCase.java | 2 +- .../apache/flink/tests/util/kafka/SmokeKafkaITCase.java | 2 +- .../kafka/sink/FlinkKafkaInternalProducerITCase.java | 2 +- .../flink/connector/kafka/sink/KafkaSinkITCase.java | 4 ++-- .../connector/kafka/sink/KafkaTransactionLogITCase.java | 2 +- .../flink/connector/kafka/sink/KafkaWriterITCase.java | 2 +- .../flink/connector/kafka/source/KafkaSourceITCase.java | 2 +- .../connector/kafka/testutils}/DockerImageVersions.java | 8 +++++--- .../connector/kafka/testutils/TwoKafkaContainers.java | 2 -- .../connectors/kafka/KafkaTestEnvironmentImpl.java | 2 +- .../internals/metrics/KafkaMetricMutableWrapperTest.java | 2 +- .../connectors/kafka/table/KafkaTableTestBase.java | 2 +- pom.xml | 2 +- 15 files changed, 19 insertions(+), 19 deletions(-) rename {flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/test => flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils}/DockerImageVersions.java (89%) diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java index b22e8a382..e18c035b0 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java @@ -19,6 +19,7 @@ package org.apache.flink.tests.util.kafka; import org.apache.flink.connector.kafka.sink.testutils.KafkaSinkExternalContextFactory; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment; import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem; import org.apache.flink.connector.testframe.junit.annotations.TestContext; @@ -28,7 +29,6 @@ import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.test.resources.ResourceTestUtils; -import org.apache.flink.util.DockerImageVersions; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.utility.DockerImageName; diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java index 4a036df20..1a2ac1f24 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java @@ -18,6 +18,7 @@ package org.apache.flink.tests.util.kafka; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContextFactory; import org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment; import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem; @@ -28,7 +29,6 @@ import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.test.resources.ResourceTestUtils; -import org.apache.flink.util.DockerImageVersions; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.utility.DockerImageName; diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java index a18976b18..721cf59d7 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java @@ -19,12 +19,12 @@ package org.apache.flink.tests.util.kafka; import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.testframe.container.FlinkContainers; import org.apache.flink.connector.testframe.container.TestcontainersSettings; import org.apache.flink.test.resources.ResourceTestUtils; import org.apache.flink.test.util.SQLJobSubmission; import org.apache.flink.tests.util.kafka.containers.SchemaRegistryContainer; -import org.apache.flink.tests.util.kafka.test.DockerImageVersions; import io.confluent.kafka.schemaregistry.avro.AvroSchema; import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java index a4d0c002e..b6102ef92 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java @@ -62,8 +62,8 @@ import java.util.UUID; import java.util.stream.Collectors; +import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; /** smoke test for the kafka connectors. */ diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java index 69e9f19f7..22795e7fe 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java @@ -47,8 +47,8 @@ import java.util.function.Consumer; import java.util.stream.Collectors; +import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java index fda9d6fa3..9cc80518a 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java @@ -32,6 +32,7 @@ import org.apache.flink.configuration.StateBackendOptions; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.sink.testutils.KafkaSinkExternalContextFactory; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.kafka.testutils.KafkaUtil; import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem; @@ -57,7 +58,6 @@ import org.apache.flink.test.util.TestUtils; import org.apache.flink.testutils.junit.SharedObjects; import org.apache.flink.testutils.junit.SharedReference; -import org.apache.flink.util.DockerImageVersions; import org.apache.flink.util.TestLogger; import org.apache.kafka.clients.CommonClientConfigs; @@ -101,8 +101,8 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; +import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java index 84d732dbf..1497c9bb9 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java @@ -44,8 +44,8 @@ import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.Ongoing; import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.PrepareAbort; import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.PrepareCommit; +import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link KafkaTransactionLog} to retrieve abortable Kafka transactions. */ diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index 811ffa207..41c266336 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -69,9 +69,9 @@ import java.util.function.Consumer; import java.util.stream.IntStream; +import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java index e37aefe0e..38ef80d51 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java @@ -29,6 +29,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContextFactory; import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv; import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; @@ -48,7 +49,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; -import org.apache.flink.util.DockerImageVersions; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerRecord; diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/test/DockerImageVersions.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DockerImageVersions.java similarity index 89% rename from flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/test/DockerImageVersions.java rename to flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DockerImageVersions.java index dcc82921f..2d262963b 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/test/DockerImageVersions.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DockerImageVersions.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.tests.util.kafka.test; +package org.apache.flink.connector.kafka.testutils; /** * Utility class for defining the image names and versions of Docker containers used during the Java @@ -24,7 +24,9 @@ */ public class DockerImageVersions { - public static final String KAFKA = "confluentinc/cp-kafka:7.2.2"; + public static final String KAFKA = "confluentinc/cp-kafka:7.4.4"; - public static final String SCHEMA_REGISTRY = "confluentinc/cp-schema-registry:7.2.2"; + public static final String SCHEMA_REGISTRY = "confluentinc/cp-schema-registry:7.4.4"; + + public static final String ZOOKEEPER = "zookeeper:3.4.14"; } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java index 7085a480a..6c5036a9b 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java @@ -18,8 +18,6 @@ package org.apache.flink.connector.kafka.testutils; -import org.apache.flink.util.DockerImageVersions; - import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.utility.DockerImageName; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index b776cad22..431e2924b 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -21,6 +21,7 @@ import org.apache.flink.connector.kafka.source.KafkaSource; import org.apache.flink.connector.kafka.source.KafkaSourceBuilder; import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.kafka.testutils.KafkaUtil; import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.streaming.api.datastream.DataStream; @@ -28,7 +29,6 @@ import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; -import org.apache.flink.util.DockerImageVersions; import org.apache.commons.collections.list.UnmodifiableList; import org.apache.kafka.clients.admin.AdminClient; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java index 269a1552b..c9948e00c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java @@ -42,8 +42,8 @@ import java.util.function.Function; import java.util.stream.Stream; +import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.util.DockerImageVersions.KAFKA; @Testcontainers @ExtendWith(TestLoggerExtension.class) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java index f80a54fc9..cffe2d6c0 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java @@ -19,10 +19,10 @@ package org.apache.flink.streaming.connectors.kafka.table; import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.test.util.AbstractTestBase; -import org.apache.flink.util.DockerImageVersions; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClient; diff --git a/pom.xml b/pom.xml index a05835215..155437679 100644 --- a/pom.xml +++ b/pom.xml @@ -53,7 +53,7 @@ under the License. 1.17.0 3.4.0 3.7.2 - 7.2.2 + 7.4.4 2.15.2 4.13.2 From b8328ab55e2bcf026ef82e35cebbb1d867cfb18f Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Thu, 8 Feb 2024 23:16:44 +0800 Subject: [PATCH 262/322] [FLINK-34192] Update to be compatible with updated SinkV2 interfaces --- .github/workflows/push_pr.yml | 2 + flink-connector-kafka/pom.xml | 4 + .../kafka/sink/KafkaWriterITCase.java | 149 +++++++++--------- .../kafka/table/KafkaTableTestUtils.java | 16 +- 4 files changed, 91 insertions(+), 80 deletions(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index d57c01812..00e2f788d 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -30,6 +30,8 @@ jobs: include: - flink: 1.18.1 jdk: '8, 11, 17' + - flink: 1.19-SNAPSHOT + jdk: '8, 11, 17, 21' uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 40d6a9f3b..6510b9c8f 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -144,6 +144,10 @@ under the License. org.slf4j slf4j-api + + io.dropwizard.metrics + metrics-core + test
diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index 41c266336..c9eceb982 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -27,9 +27,11 @@ import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.OperatorMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.metrics.testutils.MetricListener; import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.ProxyMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.UserCodeClassLoader; @@ -58,7 +60,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Optional; @@ -84,7 +85,7 @@ public class KafkaWriterITCase { private static final Network NETWORK = Network.newNetwork(); private static final String KAFKA_METRIC_WITH_GROUP_NAME = "KafkaProducer.incoming-byte-total"; private static final SinkWriter.Context SINK_WRITER_CONTEXT = new DummySinkWriterContext(); - private String topic; + private static String topic; private MetricListener metricListener; private TriggerTimeService timeService; @@ -130,11 +131,8 @@ public void testNotRegisterMetrics(DeliveryGuarantee guarantee) throws Exception @Test public void testIncreasingRecordBasedCounters() throws Exception { - final OperatorIOMetricGroup operatorIOMetricGroup = - UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); - final InternalSinkWriterMetricGroup metricGroup = - InternalSinkWriterMetricGroup.mock( - metricListener.getMetricGroup(), operatorIOMetricGroup); + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + try (final KafkaWriter writer = createWriterWithConfiguration( getKafkaClientConfiguration(), DeliveryGuarantee.NONE, metricGroup)) { @@ -167,13 +165,9 @@ public void testIncreasingRecordBasedCounters() throws Exception { @Test public void testCurrentSendTimeMetric() throws Exception { - final InternalSinkWriterMetricGroup metricGroup = - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()); try (final KafkaWriter writer = createWriterWithConfiguration( - getKafkaClientConfiguration(), - DeliveryGuarantee.AT_LEAST_ONCE, - metricGroup)) { + getKafkaClientConfiguration(), DeliveryGuarantee.AT_LEAST_ONCE)) { final Optional> currentSendTime = metricListener.getGauge("currentSendTime"); assertThat(currentSendTime.isPresent()).isTrue(); @@ -199,16 +193,12 @@ public void testCurrentSendTimeMetric() throws Exception { void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - SinkInitContext sinkInitContext = - new SinkInitContext( - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), - timeService, - null); + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); - final Counter numRecordsOutErrors = - sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); triggerProducerException(writer, properties); @@ -228,16 +218,12 @@ void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - SinkInitContext sinkInitContext = - new SinkInitContext( - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), - timeService, - null); + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); - final Counter numRecordsOutErrors = - sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); triggerProducerException(writer, properties); @@ -259,10 +245,8 @@ void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); SinkInitContext sinkInitContext = - new SinkInitContext( - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), - timeService, - null); + new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); + final KafkaWriter writer = createWriterWithConfiguration( properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); @@ -289,16 +273,12 @@ void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { void testCloseAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - SinkInitContext sinkInitContext = - new SinkInitContext( - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), - timeService, - null); + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); - final Counter numRecordsOutErrors = - sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); triggerProducerException(writer, properties); @@ -334,7 +314,7 @@ public void testMetadataPublisher() throws Exception { createWriterWithConfiguration( getKafkaClientConfiguration(), DeliveryGuarantee.AT_LEAST_ONCE, - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + createSinkWriterMetricGroup(), meta -> metadataList.add(meta.toString()))) { List expected = new ArrayList<>(); for (int i = 0; i < 100; i++) { @@ -518,17 +498,15 @@ private void assertKafkaMetricNotPresent( } private KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee) { - return createWriterWithConfiguration( - config, - guarantee, - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup())); + Properties config, DeliveryGuarantee guarantee) throws IOException { + return createWriterWithConfiguration(config, guarantee, createSinkWriterMetricGroup()); } private KafkaWriter createWriterWithConfiguration( Properties config, DeliveryGuarantee guarantee, - SinkWriterMetricGroup sinkWriterMetricGroup) { + SinkWriterMetricGroup sinkWriterMetricGroup) + throws IOException { return createWriterWithConfiguration(config, guarantee, sinkWriterMetricGroup, null); } @@ -536,27 +514,37 @@ private KafkaWriter createWriterWithConfiguration( Properties config, DeliveryGuarantee guarantee, SinkWriterMetricGroup sinkWriterMetricGroup, - @Nullable Consumer metadataConsumer) { - return new KafkaWriter<>( - guarantee, - config, - "test-prefix", - new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer), - new DummyRecordSerializer(), - new DummySchemaContext(), - Collections.emptyList()); + @Nullable Consumer metadataConsumer) + throws IOException { + KafkaSink kafkaSink = + KafkaSink.builder() + .setKafkaProducerConfig(config) + .setDeliveryGuarantee(guarantee) + .setTransactionalIdPrefix("test-prefix") + .setRecordSerializer(new DummyRecordSerializer()) + .build(); + return (KafkaWriter) + kafkaSink.createWriter( + new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer)); } private KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) { - return new KafkaWriter<>( - guarantee, - config, - "test-prefix", - sinkInitContext, - new DummyRecordSerializer(), - new DummySchemaContext(), - Collections.emptyList()); + Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) + throws IOException { + KafkaSink kafkaSink = + KafkaSink.builder() + .setKafkaProducerConfig(config) + .setDeliveryGuarantee(guarantee) + .setTransactionalIdPrefix("test-prefix") + .setRecordSerializer(new DummyRecordSerializer()) + .build(); + return (KafkaWriter) kafkaSink.createWriter(sinkInitContext); + } + + private SinkWriterMetricGroup createSinkWriterMetricGroup() { + DummyOperatorMetricGroup operatorMetricGroup = + new DummyOperatorMetricGroup(metricListener.getMetricGroup()); + return InternalSinkWriterMetricGroup.wrap(operatorMetricGroup); } private static Properties getKafkaClientConfiguration() { @@ -632,7 +620,7 @@ public Optional> metadataConsumer() { } } - private class DummyRecordSerializer implements KafkaRecordSerializationSchema { + private static class DummyRecordSerializer implements KafkaRecordSerializationSchema { @Override public ProducerRecord serialize( Integer element, KafkaSinkContext context, Long timestamp) { @@ -644,28 +632,33 @@ public ProducerRecord serialize( } } - private static class DummySchemaContext implements SerializationSchema.InitializationContext { - + private static class DummySinkWriterContext implements SinkWriter.Context { @Override - public MetricGroup getMetricGroup() { - throw new UnsupportedOperationException("Not implemented."); + public long currentWatermark() { + return 0; } @Override - public UserCodeClassLoader getUserCodeClassLoader() { - throw new UnsupportedOperationException("Not implemented."); + public Long timestamp() { + return null; } } - private static class DummySinkWriterContext implements SinkWriter.Context { - @Override - public long currentWatermark() { - return 0; + private static class DummyOperatorMetricGroup extends ProxyMetricGroup + implements OperatorMetricGroup { + + private final OperatorIOMetricGroup operatorIOMetricGroup; + + public DummyOperatorMetricGroup(MetricGroup parentMetricGroup) { + super(parentMetricGroup); + this.operatorIOMetricGroup = + UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup() + .getIOMetricGroup(); } @Override - public Long timestamp() { - return null; + public OperatorIOMetricGroup getIOMetricGroup() { + return operatorIOMetricGroup; } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java index 793d8da78..e4a5ba628 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java @@ -41,6 +41,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.HamcrestCondition.matching; @@ -98,8 +99,11 @@ public static void waitingExpectedResults( Collections.sort(expected); CommonTestUtils.waitUtil( () -> { - List actual = TestValuesTableFactory.getResults(sinkName); - Collections.sort(actual); + List actual = + TestValuesTableFactory.getResults(sinkName).stream() + .map(KafkaTableTestUtils::rowToString) + .sorted() + .collect(Collectors.toList()); return expected.equals(actual); }, timeout, @@ -124,4 +128,12 @@ public static void comparedWithKeyAndOrder( matching(TableTestMatchers.deepEqualTo(expectedData.get(key), false))); } } + + private static String rowToString(Object o) { + if (o instanceof Row) { + return ((Row) o).toString(); + } else { + return o.toString(); + } + } } From 2606a8256d0e25da19ebce4f92cd426b5bf63f7c Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Thu, 8 Feb 2024 23:17:23 +0800 Subject: [PATCH 263/322] [FLINK-34193] Remove usage of Flink-Shaded Jackson and Snakeyaml in flink-connector-kafka --- flink-connector-kafka/pom.xml | 7 +++++++ .../testutils/YamlFileMetadataService.java | 17 ++++++++--------- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 6510b9c8f..529d92527 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -171,6 +171,13 @@ under the License. test + + org.yaml + snakeyaml + 1.31 + test + + org.apache.flink flink-test-utils diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java index 32839f379..524f72431 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java @@ -23,20 +23,19 @@ import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; -import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.DumperOptions; -import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.TypeDescription; -import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.Yaml; -import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.constructor.Constructor; -import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.Node; -import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.SequenceNode; -import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.Tag; -import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.representer.Representer; - import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableMap; import org.apache.kafka.clients.CommonClientConfigs; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.yaml.snakeyaml.DumperOptions; +import org.yaml.snakeyaml.TypeDescription; +import org.yaml.snakeyaml.Yaml; +import org.yaml.snakeyaml.constructor.Constructor; +import org.yaml.snakeyaml.nodes.Node; +import org.yaml.snakeyaml.nodes.SequenceNode; +import org.yaml.snakeyaml.nodes.Tag; +import org.yaml.snakeyaml.representer.Representer; import java.io.File; import java.io.FileWriter; From 489dd7bebdd89ef69a699b87bb3fada04a04b87f Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Tue, 13 Feb 2024 09:48:29 +0100 Subject: [PATCH 264/322] [hotfix] Test against Flink 1.19-SNAPSHOT for `main` and Weekly builds --- .github/workflows/push_pr.yml | 2 +- .github/workflows/weekly.yml | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 00e2f788d..20a666606 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -39,7 +39,7 @@ jobs: python_test: strategy: matrix: - flink: [ 1.17.2, 1.18.1 ] + flink: [ 1.17.2, 1.18.1, 1.19-SNAPSHOT ] uses: apache/flink-connector-shared-utils/.github/workflows/python_ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} \ No newline at end of file diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index 21462eb7e..0aa3bec2c 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -33,6 +33,10 @@ jobs: flink: 1.18-SNAPSHOT, jdk: '8, 11, 17', branch: main + }, { + flink: 1.19-SNAPSHOT, + jdk: '8, 11, 17', + branch: main }, { flink: 1.17.2, branch: v3.1 From 15f2662eccf461d9d539ed87a78c9851cd17fa43 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Tue, 13 Feb 2024 09:50:22 +0100 Subject: [PATCH 265/322] [hotfix] Add JDK21 for 1.19-SNAPSHOT --- .github/workflows/weekly.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index 0aa3bec2c..aaa729fd4 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -35,7 +35,7 @@ jobs: branch: main }, { flink: 1.19-SNAPSHOT, - jdk: '8, 11, 17', + jdk: '8, 11, 17, 21', branch: main }, { flink: 1.17.2, From 897001d5682a0708042d59be81a10485ffd0dde7 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Thu, 11 Apr 2024 10:15:31 +0200 Subject: [PATCH 266/322] [FLINK-35007] Add support for Flink 1.19 (#90) * [FLINK-35007] Add support for Flink 1.19 This also includes dropping the weekly tests for the `v3.0` branch, since `v3.1` has been released and that's our main version going forward. * [FLINK-35007] Remove unused test class that relied on removed Internal class * [FLINK-35007][ci] Copy old `flink-conf.yaml` to make sure that all Python tests work for Flink 1.x releases --- .github/workflows/push_pr.yml | 4 +- .github/workflows/weekly.yml | 12 +- .../kafka/testutils/DataGenerators.java | 29 -- flink-python/dev/integration_test.sh | 18 + flink-python/pyflink/flink-conf.yaml | 311 ++++++++++++++++++ 5 files changed, 338 insertions(+), 36 deletions(-) create mode 100644 flink-python/pyflink/flink-conf.yaml diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 20a666606..7f30c691e 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -30,7 +30,7 @@ jobs: include: - flink: 1.18.1 jdk: '8, 11, 17' - - flink: 1.19-SNAPSHOT + - flink: 1.19.0 jdk: '8, 11, 17, 21' uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: @@ -39,7 +39,7 @@ jobs: python_test: strategy: matrix: - flink: [ 1.17.2, 1.18.1, 1.19-SNAPSHOT ] + flink: [ 1.17.2, 1.18.1, 1.19.0 ] uses: apache/flink-connector-shared-utils/.github/workflows/python_ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} \ No newline at end of file diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index aaa729fd4..f24769ae9 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -37,6 +37,10 @@ jobs: flink: 1.19-SNAPSHOT, jdk: '8, 11, 17, 21', branch: main + }, { + flink: 1.20-SNAPSHOT, + jdk: '8, 11, 17, 21', + branch: main }, { flink: 1.17.2, branch: v3.1 @@ -45,11 +49,9 @@ jobs: jdk: '8, 11, 17', branch: v3.1 }, { - flink: 1.17.2, - branch: v3.0 - }, { - flink: 1.18.1, - branch: v3.0 + flink: 1.19.0, + branch: v3.1, + jdk: '8, 11, 17', }] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java index 83ee3fb1c..d660bd2f1 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java @@ -18,16 +18,13 @@ package org.apache.flink.streaming.connectors.kafka.testutils; -import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironment; @@ -36,8 +33,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import java.util.Collections; -import java.util.List; import java.util.Properties; import java.util.Random; @@ -210,29 +205,5 @@ public void shutdown() { public Throwable getError() { return this.error; } - - private static class MockTransformation extends Transformation { - public MockTransformation() { - super("MockTransform", BasicTypeInfo.STRING_TYPE_INFO, 1); - } - - @Override - public List> getTransitivePredecessors() { - return null; - } - - @Override - public List> getInputs() { - return Collections.emptyList(); - } - } - - private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment { - - @Override - public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { - return null; - } - } } } diff --git a/flink-python/dev/integration_test.sh b/flink-python/dev/integration_test.sh index 19816725a..45f66482a 100755 --- a/flink-python/dev/integration_test.sh +++ b/flink-python/dev/integration_test.sh @@ -50,5 +50,23 @@ echo "Checking ${FLINK_SOURCE_DIR} for 'pyflink_gateway_server.py'" find "${FLINK_SOURCE_DIR}/flink-python" -name pyflink_gateway_server.py find "${FLINK_SOURCE_DIR}/flink-python/.tox" -name pyflink_gateway_server.py -exec cp "${FLINK_SOURCE_DIR}/flink-python/pyflink/pyflink_gateway_server.py" {} \; +# Copy an empty flink-conf.yaml to conf/ folder, so that all Python tests on Flink 1.x can succeed. +# This needs to be changed when adding support for Flink 2.0 +echo "Checking ${FLINK_SOURCE_DIR} for 'config.yaml'" +find "${FLINK_SOURCE_DIR}/flink-python" -name config.yaml + +# For every occurrence of config.yaml (new YAML file since Flink 1.19), copy in the old flink-conf.yaml so that +# is used over the new config.yaml file. +# +# Because our intention is to copy `flink-conf.yaml` into the same directory as `config.yaml` and not replace it, +# we need to extract the directory from `{}` and then specify the target filename (`flink-conf.yaml`) explicitly. +# Unfortunately, `find`'s `-exec` doesn't directly support manipulating `{}`. So we use a slightly modified shell command +# +# `"${1}"` and `"${2}"` correspond to the first and second arguments after the shell command. +# In this case, `"${1}"` is the path to `flink-conf.yaml` and `"${2}"` is the path to each `config.yaml` found by `find`. +# `$(dirname "${2}")` extracts the directory part of the path to `config.yaml`, and then `/flink-conf.yaml` +# specifies the target filename within that directory. +find "${FLINK_SOURCE_DIR}/flink-python/.tox" -name config.yaml -exec sh -c 'cp "${1}" "$(dirname "${2}")/flink-conf.yaml"' _ "${FLINK_SOURCE_DIR}/flink-python/pyflink/flink-conf.yaml" {} \; + # python test test_all_modules diff --git a/flink-python/pyflink/flink-conf.yaml b/flink-python/pyflink/flink-conf.yaml new file mode 100644 index 000000000..b5aa2794d --- /dev/null +++ b/flink-python/pyflink/flink-conf.yaml @@ -0,0 +1,311 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +# These parameters are required for Java 17 support. +# They can be safely removed when using Java 8/11. +env.java.opts.all: --add-exports=java.base/sun.net.util=ALL-UNNAMED --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.text=ALL-UNNAMED --add-opens=java.base/java.time=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.locks=ALL-UNNAMED + +#============================================================================== +# Common +#============================================================================== + +# The external address of the host on which the JobManager runs and can be +# reached by the TaskManagers and any clients which want to connect. This setting +# is only used in Standalone mode and may be overwritten on the JobManager side +# by specifying the --host parameter of the bin/jobmanager.sh executable. +# In high availability mode, if you use the bin/start-cluster.sh script and setup +# the conf/masters file, this will be taken care of automatically. Yarn +# automatically configure the host name based on the hostname of the node where the +# JobManager runs. + +jobmanager.rpc.address: localhost + +# The RPC port where the JobManager is reachable. + +jobmanager.rpc.port: 6123 + +# The host interface the JobManager will bind to. By default, this is localhost, and will prevent +# the JobManager from communicating outside the machine/container it is running on. +# On YARN this setting will be ignored if it is set to 'localhost', defaulting to 0.0.0.0. +# On Kubernetes this setting will be ignored, defaulting to 0.0.0.0. +# +# To enable this, set the bind-host address to one that has access to an outside facing network +# interface, such as 0.0.0.0. + +jobmanager.bind-host: localhost + + +# The total process memory size for the JobManager. +# +# Note this accounts for all memory usage within the JobManager process, including JVM metaspace and other overhead. + +jobmanager.memory.process.size: 1600m + +# The host interface the TaskManager will bind to. By default, this is localhost, and will prevent +# the TaskManager from communicating outside the machine/container it is running on. +# On YARN this setting will be ignored if it is set to 'localhost', defaulting to 0.0.0.0. +# On Kubernetes this setting will be ignored, defaulting to 0.0.0.0. +# +# To enable this, set the bind-host address to one that has access to an outside facing network +# interface, such as 0.0.0.0. + +taskmanager.bind-host: localhost + +# The address of the host on which the TaskManager runs and can be reached by the JobManager and +# other TaskManagers. If not specified, the TaskManager will try different strategies to identify +# the address. +# +# Note this address needs to be reachable by the JobManager and forward traffic to one of +# the interfaces the TaskManager is bound to (see 'taskmanager.bind-host'). +# +# Note also that unless all TaskManagers are running on the same machine, this address needs to be +# configured separately for each TaskManager. + +taskmanager.host: localhost + +# The total process memory size for the TaskManager. +# +# Note this accounts for all memory usage within the TaskManager process, including JVM metaspace and other overhead. + +taskmanager.memory.process.size: 1728m + +# To exclude JVM metaspace and overhead, please, use total Flink memory size instead of 'taskmanager.memory.process.size'. +# It is not recommended to set both 'taskmanager.memory.process.size' and Flink memory. +# +# taskmanager.memory.flink.size: 1280m + +# The number of task slots that each TaskManager offers. Each slot runs one parallel pipeline. + +taskmanager.numberOfTaskSlots: 1 + +# The parallelism used for programs that did not specify and other parallelism. + +parallelism.default: 1 + +# The default file system scheme and authority. +# +# By default file paths without scheme are interpreted relative to the local +# root file system 'file:///'. Use this to override the default and interpret +# relative paths relative to a different file system, +# for example 'hdfs://mynamenode:12345' +# +# fs.default-scheme + +#============================================================================== +# High Availability +#============================================================================== + +# The high-availability mode. Possible options are 'NONE' or 'zookeeper'. +# +# high-availability.type: zookeeper + +# The path where metadata for master recovery is persisted. While ZooKeeper stores +# the small ground truth for checkpoint and leader election, this location stores +# the larger objects, like persisted dataflow graphs. +# +# Must be a durable file system that is accessible from all nodes +# (like HDFS, S3, Ceph, nfs, ...) +# +# high-availability.storageDir: hdfs:///flink/ha/ + +# The list of ZooKeeper quorum peers that coordinate the high-availability +# setup. This must be a list of the form: +# "host1:clientPort,host2:clientPort,..." (default clientPort: 2181) +# +# high-availability.zookeeper.quorum: localhost:2181 + + +# ACL options are based on https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_BuiltinACLSchemes +# It can be either "creator" (ZOO_CREATE_ALL_ACL) or "open" (ZOO_OPEN_ACL_UNSAFE) +# The default value is "open" and it can be changed to "creator" if ZK security is enabled +# +# high-availability.zookeeper.client.acl: open + +#============================================================================== +# Fault tolerance and checkpointing +#============================================================================== + +# The backend that will be used to store operator state checkpoints if +# checkpointing is enabled. Checkpointing is enabled when execution.checkpointing.interval > 0. +# +# Execution checkpointing related parameters. Please refer to CheckpointConfig and ExecutionCheckpointingOptions for more details. +# +# execution.checkpointing.interval: 3min +# execution.checkpointing.externalized-checkpoint-retention: [DELETE_ON_CANCELLATION, RETAIN_ON_CANCELLATION] +# execution.checkpointing.max-concurrent-checkpoints: 1 +# execution.checkpointing.min-pause: 0 +# execution.checkpointing.mode: [EXACTLY_ONCE, AT_LEAST_ONCE] +# execution.checkpointing.timeout: 10min +# execution.checkpointing.tolerable-failed-checkpoints: 0 +# execution.checkpointing.unaligned: false +# +# Supported backends are 'hashmap', 'rocksdb', or the +# . +# +# state.backend.type: hashmap + +# Directory for checkpoints filesystem, when using any of the default bundled +# state backends. +# +# state.checkpoints.dir: hdfs://namenode-host:port/flink-checkpoints + +# Default target directory for savepoints, optional. +# +# state.savepoints.dir: hdfs://namenode-host:port/flink-savepoints + +# Flag to enable/disable incremental checkpoints for backends that +# support incremental checkpoints (like the RocksDB state backend). +# +# state.backend.incremental: false + +# The failover strategy, i.e., how the job computation recovers from task failures. +# Only restart tasks that may have been affected by the task failure, which typically includes +# downstream tasks and potentially upstream tasks if their produced data is no longer available for consumption. + +jobmanager.execution.failover-strategy: region + +#============================================================================== +# Rest & web frontend +#============================================================================== + +# The port to which the REST client connects to. If rest.bind-port has +# not been specified, then the server will bind to this port as well. +# +#rest.port: 8081 + +# The address to which the REST client will connect to +# +rest.address: localhost + +# Port range for the REST and web server to bind to. +# +#rest.bind-port: 8080-8090 + +# The address that the REST & web server binds to +# By default, this is localhost, which prevents the REST & web server from +# being able to communicate outside of the machine/container it is running on. +# +# To enable this, set the bind address to one that has access to outside-facing +# network interface, such as 0.0.0.0. +# +rest.bind-address: localhost + +# Flag to specify whether job submission is enabled from the web-based +# runtime monitor. Uncomment to disable. + +#web.submit.enable: false + +# Flag to specify whether job cancellation is enabled from the web-based +# runtime monitor. Uncomment to disable. + +#web.cancel.enable: false + +#============================================================================== +# Advanced +#============================================================================== + +# Override the directories for temporary files. If not specified, the +# system-specific Java temporary directory (java.io.tmpdir property) is taken. +# +# For framework setups on Yarn, Flink will automatically pick up the +# containers' temp directories without any need for configuration. +# +# Add a delimited list for multiple directories, using the system directory +# delimiter (colon ':' on unix) or a comma, e.g.: +# /data1/tmp:/data2/tmp:/data3/tmp +# +# Note: Each directory entry is read from and written to by a different I/O +# thread. You can include the same directory multiple times in order to create +# multiple I/O threads against that directory. This is for example relevant for +# high-throughput RAIDs. +# +# io.tmp.dirs: /tmp + +# The classloading resolve order. Possible values are 'child-first' (Flink's default) +# and 'parent-first' (Java's default). +# +# Child first classloading allows users to use different dependency/library +# versions in their application than those in the classpath. Switching back +# to 'parent-first' may help with debugging dependency issues. +# +# classloader.resolve-order: child-first + +# The amount of memory going to the network stack. These numbers usually need +# no tuning. Adjusting them may be necessary in case of an "Insufficient number +# of network buffers" error. The default min is 64MB, the default max is 1GB. +# +# taskmanager.memory.network.fraction: 0.1 +# taskmanager.memory.network.min: 64mb +# taskmanager.memory.network.max: 1gb + +#============================================================================== +# Flink Cluster Security Configuration +#============================================================================== + +# Kerberos authentication for various components - Hadoop, ZooKeeper, and connectors - +# may be enabled in four steps: +# 1. configure the local krb5.conf file +# 2. provide Kerberos credentials (either a keytab or a ticket cache w/ kinit) +# 3. make the credentials available to various JAAS login contexts +# 4. configure the connector to use JAAS/SASL + +# The below configure how Kerberos credentials are provided. A keytab will be used instead of +# a ticket cache if the keytab path and principal are set. + +# security.kerberos.login.use-ticket-cache: true +# security.kerberos.login.keytab: /path/to/kerberos/keytab +# security.kerberos.login.principal: flink-user + +# The configuration below defines which JAAS login contexts + +# security.kerberos.login.contexts: Client,KafkaClient + +#============================================================================== +# ZK Security Configuration +#============================================================================== + +# Below configurations are applicable if ZK ensemble is configured for security + +# Override below configuration to provide custom ZK service name if configured +# zookeeper.sasl.service-name: zookeeper + +# The configuration below must match one of the values set in "security.kerberos.login.contexts" +# zookeeper.sasl.login-context-name: Client + +#============================================================================== +# HistoryServer +#============================================================================== + +# The HistoryServer is started and stopped via bin/historyserver.sh (start|stop) + +# Directory to upload completed jobs to. Add this directory to the list of +# monitored directories of the HistoryServer as well (see below). +#jobmanager.archive.fs.dir: hdfs:///completed-jobs/ + +# The address under which the web-based HistoryServer listens. +#historyserver.web.address: 0.0.0.0 + +# The port under which the web-based HistoryServer listens. +#historyserver.web.port: 8082 + +# Comma separated list of directories to monitor for completed jobs. +#historyserver.archive.fs.dir: hdfs:///completed-jobs/ + +# Interval in milliseconds for refreshing the monitored directories. +#historyserver.archive.fs.refresh-interval: 10000 + From 2f5e3cf1e43ee38d68e47c5c72ec0fd83a904295 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Thu, 11 Apr 2024 12:09:22 +0200 Subject: [PATCH 267/322] [hotfix] Add Java 21 as JDK to test for Flink 1.19.0 weekly runs --- .github/workflows/weekly.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index f24769ae9..1d9094680 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -51,7 +51,7 @@ jobs: }, { flink: 1.19.0, branch: v3.1, - jdk: '8, 11, 17', + jdk: '8, 11, 17, 21', }] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: From c47abb3933b7c1e567a9142c6495038d16d42dd0 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Thu, 11 Apr 2024 13:32:36 +0200 Subject: [PATCH 268/322] [FLINK-34961] Use dedicated CI name for Kafka connector to differentiate it in infra-reports --- .github/workflows/push_pr.yml | 5 ++++- .github/workflows/weekly.yml | 5 ++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 7f30c691e..0f3da2ce6 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -16,7 +16,10 @@ # limitations under the License. ################################################################################ -name: CI +# We need to specify repo related information here since Apache INFRA doesn't differentiate +# between several workflows with the same names while preparing a report for GHA usage +# https://infra-reports.apache.org/#ghactions +name: Flink Connector Kafka CI on: [push, pull_request] concurrency: group: ${{ github.workflow }}-${{ github.ref }} diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index 1d9094680..7ee0f419b 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -16,7 +16,10 @@ # limitations under the License. ################################################################################ -name: Nightly +# We need to specify repo related information here since Apache INFRA doesn't differentiate +# between several workflows with the same names while preparing a report for GHA usage +# https://infra-reports.apache.org/#ghactions +name: Weekly Flink Connector Kafka CI on: schedule: - cron: "0 0 * * 0" From 1c39e3b7495640c9b3784ec672097741c072cebb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 13:43:59 +0200 Subject: [PATCH 269/322] [FLINK-35008] Bump org.apache.commons:commons-compress from 1.25.0 to 1.26.1 (#87) * [FLINK-35008] Bump org.apache.commons:commons-compress from 1.25.0 to 1.26.1 Bumps org.apache.commons:commons-compress from 1.25.0 to 1.26.1. --- updated-dependencies: - dependency-name: org.apache.commons:commons-compress dependency-type: direct:production ... Signed-off-by: dependabot[bot] * [FLINK-35008] Address dependency convergence due to bumped commons-compress --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> --- pom.xml | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 155437679..907bd67a8 100644 --- a/pom.xml +++ b/pom.xml @@ -424,7 +424,14 @@ under the License. org.apache.commons commons-compress - 1.25.0 + 1.26.1 + + + + + commons-io + commons-io + 2.15.1 From 369e7be46a70fd50d68746498aed82105741e7d6 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi <1756620+uce@users.noreply.github.com> Date: Thu, 11 Apr 2024 13:47:46 +0200 Subject: [PATCH 270/322] [FLINK-35038] Bump `org.yaml:snakeyaml` to `2.2` (#93) * Bump org.yaml:snakeyaml from 1.31 to 2.0 in /flink-connector-kafka Bumps [org.yaml:snakeyaml](https://bitbucket.org/snakeyaml/snakeyaml) from 1.31 to 2.0. - [Commits](https://bitbucket.org/snakeyaml/snakeyaml/branches/compare/snakeyaml-2.0..snakeyaml-1.31) --- updated-dependencies: - dependency-name: org.yaml:snakeyaml dependency-type: direct:development ... Signed-off-by: dependabot[bot] * [FLINK-XXXXX] Bump org.yaml:snakeyaml from 2.0 to 2.2 * [FLINK-35038] Fix SnakeYAML usage after version bump SnakeYAML introduced breaking API changes and behavior changes between 1.31 and 2.2. This commit uses the updated APIs and explicitly allows the global tag for StreamMetadata (see changed SnakeYAML behavior in [1]). [1] https://github.com/snakeyaml/snakeyaml/commit/2b8d47c8bcfd402e7a682b7b2674e8d0cb25e522 --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> --- flink-connector-kafka/pom.xml | 2 +- .../kafka/testutils/YamlFileMetadataService.java | 13 ++++++++++--- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 529d92527..2fa9d9ca8 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -174,7 +174,7 @@ under the License. org.yaml snakeyaml - 1.31 + 2.2 test diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java index 524f72431..4a1dab17a 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.yaml.snakeyaml.DumperOptions; +import org.yaml.snakeyaml.LoaderOptions; import org.yaml.snakeyaml.TypeDescription; import org.yaml.snakeyaml.Yaml; import org.yaml.snakeyaml.constructor.Constructor; @@ -252,19 +253,25 @@ Set parseFile() throws IOException { } private static Yaml initYamlParser() { - Representer representer = new Representer(); + DumperOptions dumperOptions = new DumperOptions(); + Representer representer = new Representer(dumperOptions); representer.addClassTag(StreamMetadata.class, Tag.MAP); TypeDescription typeDescription = new TypeDescription(StreamMetadata.class); representer.addTypeDescription(typeDescription); representer.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK); - return new Yaml(new ListConstructor<>(StreamMetadata.class), representer); + LoaderOptions loaderOptions = new LoaderOptions(); + // Allow global tag for StreamMetadata + loaderOptions.setTagInspector( + tag -> tag.getClassName().equals(StreamMetadata.class.getName())); + return new Yaml(new ListConstructor<>(StreamMetadata.class, loaderOptions), representer); } /** A custom constructor is required to read yaml lists at the root. */ private static class ListConstructor extends Constructor { private final Class clazz; - public ListConstructor(final Class clazz) { + public ListConstructor(final Class clazz, final LoaderOptions loaderOptions) { + super(loaderOptions); this.clazz = clazz; } From 00c9c8c74121136a0c1710ac77f307dc53adae99 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ignas=20Dauk=C5=A1as?= Date: Tue, 30 Apr 2024 02:13:37 +0300 Subject: [PATCH 271/322] [FLINK-35228][Connectors/Kafka] Fix: DynamicKafkaSource does not read re-added topic for the same cluster (#97) --- .../DynamicKafkaSourceEnumerator.java | 48 +++--- .../source/DynamicKafkaSourceITTest.java | 144 +++++++++++++++++- .../DynamicKafkaSourceEnumeratorTest.java | 99 ++++++++++++ .../kafka/DynamicKafkaSourceTestHelper.java | 27 ++-- 4 files changed, 284 insertions(+), 34 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java index e14a36d9f..20e8b9238 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java @@ -288,23 +288,34 @@ private void onHandleSubscribedStreamsFetch(Set fetchedKafkaStreams // create enumerators for (Entry> activeClusterTopics : latestClusterTopicsMap.entrySet()) { - final Set activeTopicPartitions = new HashSet<>(); - - if (dynamicKafkaSourceEnumState + KafkaSourceEnumState kafkaSourceEnumState = + dynamicKafkaSourceEnumState .getClusterEnumeratorStates() - .get(activeClusterTopics.getKey()) - != null) { - Set oldTopicPartitions = - dynamicKafkaSourceEnumState - .getClusterEnumeratorStates() - .get(activeClusterTopics.getKey()) - .assignedPartitions(); + .get(activeClusterTopics.getKey()); + + final KafkaSourceEnumState newKafkaSourceEnumState; + if (kafkaSourceEnumState != null) { + final Set activeTopics = activeClusterTopics.getValue(); + // filter out removed topics - for (TopicPartition oldTopicPartition : oldTopicPartitions) { - if (activeClusterTopics.getValue().contains(oldTopicPartition.topic())) { - activeTopicPartitions.add(oldTopicPartition); - } - } + Set activeAssignedPartitions = + kafkaSourceEnumState.assignedPartitions().stream() + .filter(tp -> activeTopics.contains(tp.topic())) + .collect(Collectors.toSet()); + Set activeUnassignedInitialPartitions = + kafkaSourceEnumState.unassignedInitialPartitions().stream() + .filter(tp -> activeTopics.contains(tp.topic())) + .collect(Collectors.toSet()); + + newKafkaSourceEnumState = + new KafkaSourceEnumState( + activeAssignedPartitions, + activeUnassignedInitialPartitions, + kafkaSourceEnumState.initialDiscoveryFinished()); + } else { + newKafkaSourceEnumState = + new KafkaSourceEnumState( + Collections.emptySet(), Collections.emptySet(), false); } // restarts enumerator from state using only the active topic partitions, to avoid @@ -312,12 +323,7 @@ private void onHandleSubscribedStreamsFetch(Set fetchedKafkaStreams createEnumeratorWithAssignedTopicPartitions( activeClusterTopics.getKey(), activeClusterTopics.getValue(), - dynamicKafkaSourceEnumState - .getClusterEnumeratorStates() - .getOrDefault( - activeClusterTopics.getKey(), - new KafkaSourceEnumState( - Collections.emptySet(), Collections.emptySet(), false)), + newKafkaSourceEnumState, clusterProperties.get(activeClusterTopics.getKey())); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java index 4ea1bd7dc..edd867565 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java @@ -82,6 +82,7 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; import static org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup.DYNAMIC_KAFKA_SOURCE_METRIC_GROUP; import static org.assertj.core.api.Assertions.assertThat; @@ -372,6 +373,132 @@ void testMigrationUsingFileMetadataService() throws Throwable { .collect(Collectors.toList())); } + @Test + void testTopicReAddMigrationUsingFileMetadataService() throws Throwable { + // setup topics + int kafkaClusterIdx = 0; + String topic1 = "test-topic-re-add-1"; + String topic2 = "test-topic-re-add-2"; + DynamicKafkaSourceTestHelper.createTopic(kafkaClusterIdx, topic1, NUM_PARTITIONS); + DynamicKafkaSourceTestHelper.createTopic(kafkaClusterIdx, topic2, NUM_PARTITIONS); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.setParallelism(2); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty(CommonClientConfigs.GROUP_ID_CONFIG, "dynamic-kafka-src"); + + // create new metadata file to consume from 1 cluster + String testStreamId = "test-topic-re-add-stream"; + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + ImmutableList.of(topic1), + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata( + kafkaClusterIdx))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds(Collections.singleton(testStreamId)) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + List results = new ArrayList<>(); + + int stage1Records = + DynamicKafkaSourceTestHelper.produceToKafka( + kafkaClusterIdx, topic1, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0); + int stage2Records = + DynamicKafkaSourceTestHelper.produceToKafka( + kafkaClusterIdx, + topic2, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + stage1Records); + + try (CloseableIterator iterator = stream.executeAndCollect()) { + CommonTestUtils.waitUtil( + () -> { + try { + results.add(iterator.next()); + + // switch to second topic after first is read + if (results.size() == stage1Records) { + writeClusterMetadataToFile( + metadataFile, + testStreamId, + ImmutableList.of(topic2), + ImmutableList.of( + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata( + kafkaClusterIdx))); + } + + // re-add first topic again after second is read + // produce another batch to first topic + if (results.size() == stage2Records) { + DynamicKafkaSourceTestHelper.produceToKafka( + kafkaClusterIdx, + topic1, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + stage2Records); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + ImmutableList.of(topic1, topic2), + ImmutableList.of( + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata( + kafkaClusterIdx))); + } + } catch (NoSuchElementException e) { + // swallow and wait + } catch (IOException e) { + throw new UncheckedIOException(e); + } catch (Throwable e) { + throw new RuntimeException(e); + } + + // first batch of topic 1 * 2 + topic 2 + second batch of topic 1 + return results.size() == NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT * 4; + }, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + } + + // verify data + Stream expectedFullRead = + IntStream.range(0, NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT * 3).boxed(); + Stream expectedReRead = + IntStream.range(0, NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT).boxed(); + List expectedResults = + Stream.concat(expectedFullRead, expectedReRead).collect(Collectors.toList()); + assertThat(results).containsExactlyInAnyOrderElementsOf(expectedResults); + } + @Test void testStreamPatternSubscriber() throws Throwable { DynamicKafkaSourceTestHelper.createTopic(0, "stream-pattern-test-1", NUM_PARTITIONS); @@ -621,7 +748,7 @@ private void writeClusterMetadataToFile(File metadataFile, Set kafk private void writeClusterMetadataToFile( File metadataFile, String streamId, - String topic, + List topics, List kafkaClusterTestEnvMetadataList) throws IOException { List clusterMetadata = @@ -633,7 +760,7 @@ private void writeClusterMetadataToFile( KafkaClusterTestEnvMetadata.getKafkaClusterId(), KafkaClusterTestEnvMetadata .getBrokerConnectionStrings(), - ImmutableList.of(topic))) + topics)) .collect(Collectors.toList()); YamlFileMetadataService.StreamMetadata streamMetadata = new YamlFileMetadataService.StreamMetadata(streamId, clusterMetadata); @@ -641,6 +768,19 @@ private void writeClusterMetadataToFile( Collections.singletonList(streamMetadata), metadataFile); } + private void writeClusterMetadataToFile( + File metadataFile, + String streamId, + String topic, + List kafkaClusterTestEnvMetadataList) + throws IOException { + writeClusterMetadataToFile( + metadataFile, + streamId, + ImmutableList.of(topic), + kafkaClusterTestEnvMetadataList); + } + private Set findMetrics(InMemoryReporter inMemoryReporter, String groupPattern) { Optional groups = inMemoryReporter.findGroup(groupPattern); assertThat(groups).isPresent(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java index 3c3a76e8f..861333458 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java @@ -32,6 +32,8 @@ import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSetSubscriber; import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.AssignmentStatus; +import org.apache.flink.connector.kafka.source.enumerator.TopicPartitionAndAssignmentStatus; import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; import org.apache.flink.connector.kafka.testutils.MockKafkaMetadataService; @@ -464,6 +466,91 @@ public void testSnapshotState() throws Throwable { } } + @Test + public void testEnumeratorStateDoesNotContainStaleTopicPartitions() throws Throwable { + final String topic2 = TOPIC + "_2"; + + DynamicKafkaSourceTestHelper.createTopic(topic2, NUM_SPLITS_PER_CLUSTER, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + topic2, NUM_SPLITS_PER_CLUSTER, NUM_RECORDS_PER_SPLIT); + + final Set initialStreams = + Collections.singleton( + new KafkaStream( + TOPIC, + DynamicKafkaSourceTestHelper.getClusterMetadataMap( + 0, TOPIC, topic2))); + + final Set updatedStreams = + Collections.singleton( + new KafkaStream( + TOPIC, + DynamicKafkaSourceTestHelper.getClusterMetadataMap(0, TOPIC))); + + try (MockKafkaMetadataService metadataService = + new MockKafkaMetadataService(initialStreams); + MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + metadataService, + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + context.runPeriodicCallable(0); + + runAllOneTimeCallables(context); + + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + + DynamicKafkaSourceEnumState initialState = enumerator.snapshotState(-1); + + assertThat(getFilteredTopicPartitions(initialState, TOPIC, AssignmentStatus.ASSIGNED)) + .hasSize(2); + assertThat( + getFilteredTopicPartitions( + initialState, TOPIC, AssignmentStatus.UNASSIGNED_INITIAL)) + .hasSize(1); + assertThat(getFilteredTopicPartitions(initialState, topic2, AssignmentStatus.ASSIGNED)) + .hasSize(2); + assertThat( + getFilteredTopicPartitions( + initialState, topic2, AssignmentStatus.UNASSIGNED_INITIAL)) + .hasSize(1); + + // mock metadata change + metadataService.setKafkaStreams(updatedStreams); + + // changes should have occurred here + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + DynamicKafkaSourceEnumState migratedState = enumerator.snapshotState(-1); + + assertThat(getFilteredTopicPartitions(migratedState, TOPIC, AssignmentStatus.ASSIGNED)) + .hasSize(3); + assertThat( + getFilteredTopicPartitions( + migratedState, TOPIC, AssignmentStatus.UNASSIGNED_INITIAL)) + .isEmpty(); + assertThat(getFilteredTopicPartitions(migratedState, topic2, AssignmentStatus.ASSIGNED)) + .isEmpty(); + assertThat( + getFilteredTopicPartitions( + migratedState, topic2, AssignmentStatus.UNASSIGNED_INITIAL)) + .isEmpty(); + } + } + @Test public void testStartupWithCheckpointState() throws Throwable { // init enumerator with checkpoint state @@ -865,6 +952,18 @@ private Map> getReaderAssignments( return readerToSplits; } + private List getFilteredTopicPartitions( + DynamicKafkaSourceEnumState state, String topic, AssignmentStatus assignmentStatus) { + return state.getClusterEnumeratorStates().values().stream() + .flatMap(s -> s.partitions().stream()) + .filter( + partition -> + partition.topicPartition().topic().equals(topic) + && partition.assignmentStatus() == assignmentStatus) + .map(TopicPartitionAndAssignmentStatus::topicPartition) + .collect(Collectors.toList()); + } + private static void runAllOneTimeCallables(MockSplitEnumeratorContext context) throws Throwable { while (!context.getOneTimeCallables().isEmpty()) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java index 0ec02cc0e..8eb0d28c1 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java @@ -32,6 +32,7 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -67,21 +68,25 @@ public static String getKafkaClusterId(int kafkaClusterIdx) { return kafkaClusters.get(kafkaClusterIdx).getKafkaClusterId(); } + public static Map getClusterMetadataMap( + int kafkaClusterIdx, String... topics) { + KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = + getKafkaClusterTestEnvMetadata(kafkaClusterIdx); + + Set topicsSet = new HashSet<>(Arrays.asList(topics)); + + ClusterMetadata clusterMetadata = + new ClusterMetadata(topicsSet, kafkaClusterTestEnvMetadata.getStandardProperties()); + + return Collections.singletonMap( + kafkaClusterTestEnvMetadata.getKafkaClusterId(), clusterMetadata); + } + /** Stream is a topic across multiple clusters. */ public static KafkaStream getKafkaStream(String topic) { Map clusterMetadataMap = new HashMap<>(); for (int i = 0; i < NUM_KAFKA_CLUSTERS; i++) { - KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = - getKafkaClusterTestEnvMetadata(i); - - Set topics = new HashSet<>(); - topics.add(topic); - - ClusterMetadata clusterMetadata = - new ClusterMetadata( - topics, kafkaClusterTestEnvMetadata.getStandardProperties()); - clusterMetadataMap.put( - kafkaClusterTestEnvMetadata.getKafkaClusterId(), clusterMetadata); + clusterMetadataMap.putAll(getClusterMetadataMap(i, topic)); } return new KafkaStream(topic, clusterMetadataMap); From 15d3fbd4e65dae6c334e2386dd337d2bf423c216 Mon Sep 17 00:00:00 2001 From: Danny Cranmer Date: Fri, 7 Jun 2024 11:51:22 +0100 Subject: [PATCH 272/322] [FLINK-35138][Connectors/Kafka] Bump development version to 3.3-SNAPSHOT --- .../flink-end-to-end-tests-common-kafka/pom.xml | 2 +- .../flink-streaming-kafka-test-base/pom.xml | 2 +- .../flink-streaming-kafka-test/pom.xml | 2 +- flink-connector-kafka-e2e-tests/pom.xml | 2 +- flink-connector-kafka/pom.xml | 2 +- flink-python/pom.xml | 2 +- flink-sql-connector-kafka/pom.xml | 2 +- pom.xml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index cc0aa1bd9..c6faf1149 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 3.1-SNAPSHOT + 3.3-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml index fa4e22d56..c393fe80c 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 3.1-SNAPSHOT + 3.3-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml index 2b5dc8cd3..a9dadfa6e 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 3.1-SNAPSHOT + 3.3-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/pom.xml b/flink-connector-kafka-e2e-tests/pom.xml index c9208ed93..00661d0bd 100644 --- a/flink-connector-kafka-e2e-tests/pom.xml +++ b/flink-connector-kafka-e2e-tests/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 3.1-SNAPSHOT + 3.3-SNAPSHOT pom diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 2fa9d9ca8..c16d4d79b 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 3.1-SNAPSHOT + 3.3-SNAPSHOT flink-connector-kafka diff --git a/flink-python/pom.xml b/flink-python/pom.xml index cb3f70caa..978db5f5d 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 3.1-SNAPSHOT + 3.3-SNAPSHOT flink-connector-kafka-python diff --git a/flink-sql-connector-kafka/pom.xml b/flink-sql-connector-kafka/pom.xml index d22899e4f..0f724d1ce 100644 --- a/flink-sql-connector-kafka/pom.xml +++ b/flink-sql-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 3.1-SNAPSHOT + 3.3-SNAPSHOT flink-sql-connector-kafka diff --git a/pom.xml b/pom.xml index 907bd67a8..92e1ebd78 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 3.1-SNAPSHOT + 3.3-SNAPSHOT Flink : Connectors : Kafka : Parent pom 2022 From 4429b78f689b4834a4d0d714b59c46e5faa436e6 Mon Sep 17 00:00:00 2001 From: Jimmyz Date: Thu, 11 Jul 2024 19:39:44 +0800 Subject: [PATCH 273/322] [FLINK-35749] Kafka sink component will lose data when kafka cluster is unavailable for a while (#107) The key problem is in WriterCallback#onCompletion of KafkaWriter: mailboxExecutor.submit( () -> { // Checking for exceptions from previous writes checkAsyncException(); }, "Update error metric"); 'mailboxExecutor.submit' without getting future back will not throw exception, which causes the 'asyncProducerException' is assign to null but the job seems like nothing happened. The fix is using 'mailboxExecutor.execute' instead of 'mailboxExecutor.submit'. --------- Co-authored-by: Jimmy Zhao --- .../86dfd459-67a9-4b26-9b5c-0b0bbf22681a | 12 + .../connector/kafka/sink/KafkaWriter.java | 4 +- .../sink/KafkaWriterFaultToleranceITCase.java | 167 ++++++++++ .../kafka/sink/KafkaWriterITCase.java | 254 +-------------- .../kafka/sink/KafkaWriterTestBase.java | 288 ++++++++++++++++++ 5 files changed, 480 insertions(+), 245 deletions(-) create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java diff --git a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a index d952f0267..4406d5c5f 100644 --- a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a +++ b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a @@ -20,6 +20,12 @@ org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only o * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.KafkaWriterFaultToleranceITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.source.KafkaSourceITCase does not satisfy: only one of the following predicates match:\ @@ -98,6 +104,12 @@ org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only o * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.KafkaWriterFaultToleranceITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.source.KafkaSourceITCase does not satisfy: only one of the following predicates match:\ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java index 0cc16b219..0f4e8755e 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java @@ -449,7 +449,9 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } // Checking for exceptions from previous writes - mailboxExecutor.submit( + // Notice: throwing exception in mailboxExecutor thread is not safe enough for + // triggering global fail over, which has been fixed in [FLINK-31305]. + mailboxExecutor.execute( () -> { // Checking for exceptions from previous writes checkAsyncException(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java new file mode 100644 index 000000000..e045d7dda --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.kafka.common.errors.NetworkException; +import org.apache.kafka.common.errors.TimeoutException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Properties; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode; + +/** Tests for the standalone KafkaWriter in case of fault tolerance. */ +@ExtendWith(TestLoggerExtension.class) +public class KafkaWriterFaultToleranceITCase extends KafkaWriterTestBase { + private static final String INIT_KAFKA_RETRIES = "0"; + private static final String INIT_KAFKA_REQUEST_TIMEOUT_MS = "1000"; + private static final String INIT_KAFKA_MAX_BLOCK_MS = "1000"; + private static final String INIT_KAFKA_DELIVERY_TIMEOUT_MS = "1000"; + + @BeforeAll + public static void beforeAll() { + KAFKA_CONTAINER.start(); + } + + @AfterAll + public static void afterAll() { + KAFKA_CONTAINER.stop(); + } + + @BeforeEach + public void setUp(TestInfo testInfo) { + super.setUp(testInfo); + } + + @Test + void testWriteExceptionWhenKafkaUnavailable() throws Exception { + Properties properties = getPropertiesForSendingFaultTolerance(); + + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup); + + writer.write(1, SINK_WRITER_CONTEXT); + + KAFKA_CONTAINER.stop(); + + try { + writer.getCurrentProducer().flush(); + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .hasRootCauseExactlyInstanceOf(NetworkException.class); + } finally { + KAFKA_CONTAINER.start(); + } + } + + @Test + void testFlushExceptionWhenKafkaUnavailable() throws Exception { + Properties properties = getPropertiesForSendingFaultTolerance(); + + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup); + writer.write(1, SINK_WRITER_CONTEXT); + + KAFKA_CONTAINER.stop(); + try { + assertThatCode(() -> writer.flush(false)) + .hasRootCauseExactlyInstanceOf(NetworkException.class); + } finally { + KAFKA_CONTAINER.start(); + } + } + + @Test + void testCloseExceptionWhenKafkaUnavailable() throws Exception { + Properties properties = getPropertiesForSendingFaultTolerance(); + + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup); + + writer.write(1, SINK_WRITER_CONTEXT); + + KAFKA_CONTAINER.stop(); + + try { + writer.getCurrentProducer().flush(); + // closing producer resource throws exception first + assertThatCode(() -> writer.close()) + .hasRootCauseExactlyInstanceOf(NetworkException.class); + } finally { + KAFKA_CONTAINER.start(); + } + } + + @Test + void testMailboxExceptionWhenKafkaUnavailable() throws Exception { + Properties properties = getPropertiesForSendingFaultTolerance(); + SinkInitContext sinkInitContext = + new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); + + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.AT_LEAST_ONCE, sinkInitContext); + + KAFKA_CONTAINER.stop(); + + writer.write(1, SINK_WRITER_CONTEXT); + + try { + writer.getCurrentProducer().flush(); + + assertThatCode( + () -> { + while (sinkInitContext.getMailboxExecutor().tryYield()) { + // execute all mails + } + }) + .hasRootCauseExactlyInstanceOf(TimeoutException.class); + } finally { + KAFKA_CONTAINER.start(); + } + } + + private Properties getPropertiesForSendingFaultTolerance() { + Properties properties = getKafkaClientConfiguration(); + + // reduce the default vault for test case + properties.setProperty("retries", INIT_KAFKA_RETRIES); + properties.setProperty("request.timeout.ms", INIT_KAFKA_REQUEST_TIMEOUT_MS); + properties.setProperty("max.block.ms", INIT_KAFKA_MAX_BLOCK_MS); + properties.setProperty("delivery.timeout.ms", INIT_KAFKA_DELIVERY_TIMEOUT_MS); + + return properties; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index c9eceb982..0aef938c9 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -17,30 +17,15 @@ package org.apache.flink.connector.kafka.sink; -import org.apache.flink.api.common.operators.ProcessingTimeService; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.connector.sink2.SinkWriter; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.base.DeliveryGuarantee; -import org.apache.flink.connector.base.sink.writer.TestSinkInitContext; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.groups.OperatorIOMetricGroup; -import org.apache.flink.metrics.groups.OperatorMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; -import org.apache.flink.metrics.testutils.MetricListener; -import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; -import org.apache.flink.runtime.metrics.groups.ProxyMetricGroup; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.TestLoggerExtension; -import org.apache.flink.util.UserCodeClassLoader; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.errors.ProducerFencedException; -import org.apache.kafka.common.serialization.ByteArraySerializer; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -49,52 +34,22 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.KafkaContainer; -import org.testcontainers.containers.Network; - -import javax.annotation.Nullable; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; -import java.util.Comparator; import java.util.List; import java.util.Optional; -import java.util.OptionalLong; -import java.util.PriorityQueue; import java.util.Properties; -import java.util.concurrent.ScheduledFuture; -import java.util.function.Consumer; import java.util.stream.IntStream; -import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; -import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode; /** Tests for the standalone KafkaWriter. */ @ExtendWith(TestLoggerExtension.class) -public class KafkaWriterITCase { - - private static final Logger LOG = LoggerFactory.getLogger(KafkaWriterITCase.class); - private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; - private static final Network NETWORK = Network.newNetwork(); - private static final String KAFKA_METRIC_WITH_GROUP_NAME = "KafkaProducer.incoming-byte-total"; - private static final SinkWriter.Context SINK_WRITER_CONTEXT = new DummySinkWriterContext(); - private static String topic; - - private MetricListener metricListener; - private TriggerTimeService timeService; - - private static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) - .withEmbeddedZookeeper() - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); +public class KafkaWriterITCase extends KafkaWriterTestBase { @BeforeAll public static void beforeAll() { @@ -108,9 +63,7 @@ public static void afterAll() { @BeforeEach public void setUp(TestInfo testInfo) { - metricListener = new MetricListener(); - timeService = new TriggerTimeService(); - topic = testInfo.getDisplayName().replaceAll("\\W", ""); + super.setUp(testInfo); } @ParameterizedTest @@ -258,9 +211,14 @@ void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { // to ensure that the exceptional send request has completed writer.getCurrentProducer().flush(); - while (sinkInitContext.getMailboxExecutor().tryYield()) { - // execute all mails - } + assertThatCode( + () -> { + while (sinkInitContext.getMailboxExecutor().tryYield()) { + // execute all mails + } + }) + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) @@ -496,196 +454,4 @@ private void assertKafkaMetricNotPresent( assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME)).isNotPresent(); } } - - private KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee) throws IOException { - return createWriterWithConfiguration(config, guarantee, createSinkWriterMetricGroup()); - } - - private KafkaWriter createWriterWithConfiguration( - Properties config, - DeliveryGuarantee guarantee, - SinkWriterMetricGroup sinkWriterMetricGroup) - throws IOException { - return createWriterWithConfiguration(config, guarantee, sinkWriterMetricGroup, null); - } - - private KafkaWriter createWriterWithConfiguration( - Properties config, - DeliveryGuarantee guarantee, - SinkWriterMetricGroup sinkWriterMetricGroup, - @Nullable Consumer metadataConsumer) - throws IOException { - KafkaSink kafkaSink = - KafkaSink.builder() - .setKafkaProducerConfig(config) - .setDeliveryGuarantee(guarantee) - .setTransactionalIdPrefix("test-prefix") - .setRecordSerializer(new DummyRecordSerializer()) - .build(); - return (KafkaWriter) - kafkaSink.createWriter( - new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer)); - } - - private KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) - throws IOException { - KafkaSink kafkaSink = - KafkaSink.builder() - .setKafkaProducerConfig(config) - .setDeliveryGuarantee(guarantee) - .setTransactionalIdPrefix("test-prefix") - .setRecordSerializer(new DummyRecordSerializer()) - .build(); - return (KafkaWriter) kafkaSink.createWriter(sinkInitContext); - } - - private SinkWriterMetricGroup createSinkWriterMetricGroup() { - DummyOperatorMetricGroup operatorMetricGroup = - new DummyOperatorMetricGroup(metricListener.getMetricGroup()); - return InternalSinkWriterMetricGroup.wrap(operatorMetricGroup); - } - - private static Properties getKafkaClientConfiguration() { - final Properties standardProps = new Properties(); - standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers()); - standardProps.put("group.id", "kafkaWriter-tests"); - standardProps.put("enable.auto.commit", false); - standardProps.put("key.serializer", ByteArraySerializer.class.getName()); - standardProps.put("value.serializer", ByteArraySerializer.class.getName()); - standardProps.put("auto.offset.reset", "earliest"); - return standardProps; - } - - private static class SinkInitContext extends TestSinkInitContext { - - private final SinkWriterMetricGroup metricGroup; - private final ProcessingTimeService timeService; - @Nullable private final Consumer metadataConsumer; - - SinkInitContext( - SinkWriterMetricGroup metricGroup, - ProcessingTimeService timeService, - @Nullable Consumer metadataConsumer) { - this.metricGroup = metricGroup; - this.timeService = timeService; - this.metadataConsumer = metadataConsumer; - } - - @Override - public UserCodeClassLoader getUserCodeClassLoader() { - throw new UnsupportedOperationException("Not implemented."); - } - - @Override - public ProcessingTimeService getProcessingTimeService() { - return timeService; - } - - @Override - public int getSubtaskId() { - return 0; - } - - @Override - public int getNumberOfParallelSubtasks() { - return 1; - } - - @Override - public int getAttemptNumber() { - return 0; - } - - @Override - public SinkWriterMetricGroup metricGroup() { - return metricGroup; - } - - @Override - public OptionalLong getRestoredCheckpointId() { - return OptionalLong.empty(); - } - - @Override - public SerializationSchema.InitializationContext - asSerializationSchemaInitializationContext() { - return null; - } - - @Override - public Optional> metadataConsumer() { - return Optional.ofNullable((Consumer) metadataConsumer); - } - } - - private static class DummyRecordSerializer implements KafkaRecordSerializationSchema { - @Override - public ProducerRecord serialize( - Integer element, KafkaSinkContext context, Long timestamp) { - if (element == null) { - // in general, serializers should be allowed to skip invalid elements - return null; - } - return new ProducerRecord<>(topic, ByteBuffer.allocate(4).putInt(element).array()); - } - } - - private static class DummySinkWriterContext implements SinkWriter.Context { - @Override - public long currentWatermark() { - return 0; - } - - @Override - public Long timestamp() { - return null; - } - } - - private static class DummyOperatorMetricGroup extends ProxyMetricGroup - implements OperatorMetricGroup { - - private final OperatorIOMetricGroup operatorIOMetricGroup; - - public DummyOperatorMetricGroup(MetricGroup parentMetricGroup) { - super(parentMetricGroup); - this.operatorIOMetricGroup = - UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup() - .getIOMetricGroup(); - } - - @Override - public OperatorIOMetricGroup getIOMetricGroup() { - return operatorIOMetricGroup; - } - } - - private static class TriggerTimeService implements ProcessingTimeService { - - private final PriorityQueue> registeredCallbacks = - new PriorityQueue<>(Comparator.comparingLong(o -> o.f0)); - - @Override - public long getCurrentProcessingTime() { - return 0; - } - - @Override - public ScheduledFuture registerTimer( - long time, ProcessingTimeCallback processingTimerCallback) { - registeredCallbacks.add(new Tuple2<>(time, processingTimerCallback)); - return null; - } - - public void trigger() throws Exception { - final Tuple2 registered = registeredCallbacks.poll(); - if (registered == null) { - LOG.warn("Triggered time service but no callback was registered."); - return; - } - registered.f1.onProcessingTime(registered.f0); - } - } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java new file mode 100644 index 000000000..c9dce3670 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java @@ -0,0 +1,288 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.base.sink.writer.TestSinkInitContext; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.OperatorMetricGroup; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.ProxyMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.util.UserCodeClassLoader; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.containers.Network; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.PriorityQueue; +import java.util.Properties; +import java.util.concurrent.ScheduledFuture; +import java.util.function.Consumer; + +import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; + +/** Test base for KafkaWriter. */ +public abstract class KafkaWriterTestBase { + + protected static final Logger LOG = LoggerFactory.getLogger(KafkaWriterTestBase.class); + protected static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; + protected static final Network NETWORK = Network.newNetwork(); + protected static final String KAFKA_METRIC_WITH_GROUP_NAME = + "KafkaProducer.incoming-byte-total"; + protected static final SinkWriter.Context SINK_WRITER_CONTEXT = new DummySinkWriterContext(); + protected static String topic; + + protected MetricListener metricListener; + protected TriggerTimeService timeService; + + protected static final KafkaContainer KAFKA_CONTAINER = + createKafkaContainer(KAFKA, LOG) + .withEmbeddedZookeeper() + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); + + @BeforeEach + public void setUp(TestInfo testInfo) { + metricListener = new MetricListener(); + timeService = new TriggerTimeService(); + topic = testInfo.getDisplayName().replaceAll("\\W", ""); + } + + protected KafkaWriter createWriterWithConfiguration( + Properties config, DeliveryGuarantee guarantee) throws IOException { + return createWriterWithConfiguration(config, guarantee, createSinkWriterMetricGroup()); + } + + protected KafkaWriter createWriterWithConfiguration( + Properties config, + DeliveryGuarantee guarantee, + SinkWriterMetricGroup sinkWriterMetricGroup) + throws IOException { + return createWriterWithConfiguration(config, guarantee, sinkWriterMetricGroup, null); + } + + protected KafkaWriter createWriterWithConfiguration( + Properties config, + DeliveryGuarantee guarantee, + SinkWriterMetricGroup sinkWriterMetricGroup, + @Nullable Consumer metadataConsumer) + throws IOException { + KafkaSink kafkaSink = + KafkaSink.builder() + .setKafkaProducerConfig(config) + .setDeliveryGuarantee(guarantee) + .setTransactionalIdPrefix("test-prefix") + .setRecordSerializer(new DummyRecordSerializer()) + .build(); + return (KafkaWriter) + kafkaSink.createWriter( + new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer)); + } + + protected KafkaWriter createWriterWithConfiguration( + Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) + throws IOException { + KafkaSink kafkaSink = + KafkaSink.builder() + .setKafkaProducerConfig(config) + .setDeliveryGuarantee(guarantee) + .setTransactionalIdPrefix("test-prefix") + .setRecordSerializer(new DummyRecordSerializer()) + .build(); + return (KafkaWriter) kafkaSink.createWriter(sinkInitContext); + } + + protected SinkWriterMetricGroup createSinkWriterMetricGroup() { + DummyOperatorMetricGroup operatorMetricGroup = + new DummyOperatorMetricGroup(metricListener.getMetricGroup()); + return InternalSinkWriterMetricGroup.wrap(operatorMetricGroup); + } + + protected static Properties getKafkaClientConfiguration() { + final Properties standardProps = new Properties(); + standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers()); + standardProps.put("group.id", "kafkaWriter-tests"); + standardProps.put("enable.auto.commit", false); + standardProps.put("key.serializer", ByteArraySerializer.class.getName()); + standardProps.put("value.serializer", ByteArraySerializer.class.getName()); + standardProps.put("auto.offset.reset", "earliest"); + return standardProps; + } + + /** mock sink context for initializing KafkaWriter. */ + protected static class SinkInitContext extends TestSinkInitContext { + + protected final SinkWriterMetricGroup metricGroup; + protected final ProcessingTimeService timeService; + @Nullable protected final Consumer metadataConsumer; + + SinkInitContext( + SinkWriterMetricGroup metricGroup, + ProcessingTimeService timeService, + @Nullable Consumer metadataConsumer) { + this.metricGroup = metricGroup; + this.timeService = timeService; + this.metadataConsumer = metadataConsumer; + } + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + throw new UnsupportedOperationException("Not implemented."); + } + + @Override + public ProcessingTimeService getProcessingTimeService() { + return timeService; + } + + @Override + public int getSubtaskId() { + return 0; + } + + @Override + public int getNumberOfParallelSubtasks() { + return 1; + } + + @Override + public int getAttemptNumber() { + return 0; + } + + @Override + public SinkWriterMetricGroup metricGroup() { + return metricGroup; + } + + @Override + public OptionalLong getRestoredCheckpointId() { + return OptionalLong.empty(); + } + + @Override + public SerializationSchema.InitializationContext + asSerializationSchemaInitializationContext() { + return null; + } + + @Override + public Optional> metadataConsumer() { + return Optional.ofNullable((Consumer) metadataConsumer); + } + } + + /** mock recordSerializer for KafkaSink. */ + protected static class DummyRecordSerializer + implements KafkaRecordSerializationSchema { + @Override + public ProducerRecord serialize( + Integer element, KafkaSinkContext context, Long timestamp) { + if (element == null) { + // in general, serializers should be allowed to skip invalid elements + return null; + } + return new ProducerRecord<>(topic, ByteBuffer.allocate(4).putInt(element).array()); + } + } + + /** + * mock context for KafkaWriter#write(java.lang.Object, + * org.apache.flink.api.connector.sink2.SinkWriter.Context). + */ + protected static class DummySinkWriterContext implements SinkWriter.Context { + @Override + public long currentWatermark() { + return 0; + } + + @Override + public Long timestamp() { + return null; + } + } + + /** mock metrics group for initializing KafkaWriter. */ + protected static class DummyOperatorMetricGroup extends ProxyMetricGroup + implements OperatorMetricGroup { + + private final OperatorIOMetricGroup operatorIOMetricGroup; + + public DummyOperatorMetricGroup(MetricGroup parentMetricGroup) { + super(parentMetricGroup); + this.operatorIOMetricGroup = + UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup() + .getIOMetricGroup(); + } + + @Override + public OperatorIOMetricGroup getIOMetricGroup() { + return operatorIOMetricGroup; + } + } + + /** mock time service for KafkaWriter. */ + protected static class TriggerTimeService implements ProcessingTimeService { + + private final PriorityQueue> registeredCallbacks = + new PriorityQueue<>(Comparator.comparingLong(o -> o.f0)); + + @Override + public long getCurrentProcessingTime() { + return 0; + } + + @Override + public ScheduledFuture registerTimer( + long time, ProcessingTimeCallback processingTimerCallback) { + registeredCallbacks.add(new Tuple2<>(time, processingTimerCallback)); + return null; + } + + public void trigger() throws Exception { + final Tuple2 registered = registeredCallbacks.poll(); + if (registered == null) { + LOG.warn("Triggered time service but no callback was registered."); + return; + } + registered.f1.onProcessingTime(registered.f0); + } + } +} From 86f796a01cba0d7b3adeb95f413e412c30e466f1 Mon Sep 17 00:00:00 2001 From: klam-shop <78921373+klam-shop@users.noreply.github.com> Date: Thu, 18 Jul 2024 09:45:48 -0400 Subject: [PATCH 274/322] [FLINK-35808] Let `ConsumerConfig.(KEY|VALUE)_DESERIALIZER_CLASS_CONFIG` be overridable by user in `KafkaSourceBuilder` (#108) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What is the purpose of the change Let `ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG` and `ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG` be overridable by user in `KafkaSourceBuilder`, in order to enable the Large Message use-case discussed in this [mailing list discussion](https://lists.apache.org/thread/spl88o63sjm2dv4l5no0ym632d2yt2o6). This allows users to easily implement the [`claim check` large message pattern](https://developer.confluent.io/patterns/event-processing/claim-check/) without bringing any concerns into the Flink codebase otherwise, by specifying a `value.deserializer` that handles it, but otherwise passes through the bytes. Note: [overriding `value.serializer` is already supported on the Producer side. ](https://github.com/apache/flink-connector-kafka/blob/15d3fbd4e65dae6c334e2386dd337d2bf423c216/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java#L82-L83) Other Reading: https://www.kai-waehner.de/blog/2020/08/07/apache-kafka-handling-large-messages-and-files-for-image-video-audio-processing/ https://www.conduktor.io/kafka/how-to-send-large-messages-in-apache-kafka/#Option-1:-using-an-external-store-(GB-size-messages)-0 ## Brief change log - Updates key and value deserializers to be overridable by users in `KafkaSourceBuilder` ## Verifying this change - [x] Test that both key and value deserializers can be overridden - [x] Tests to ensure that the user-supplied deserializer(s) returns bytes (byte[]) ## Does this pull request potentially affect one of the following parts: - Dependencies (does it add or upgrade a dependency): no - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no - The serializers: yes - The runtime per-record code paths (performance sensitive): no - Anything that affects deployment or recovery: no - The S3 file system connector: no ## Documentation - Does this pull request introduce a new feature? no - If yes, how is the feature documented? not applicable --- .../docs/connectors/datastream/kafka.md | 2 - .../docs/connectors/table/upsert-kafka.md | 2 +- .../docs/connectors/datastream/kafka.md | 2 - docs/content/docs/connectors/table/kafka.md | 2 +- .../docs/connectors/table/upsert-kafka.md | 2 +- .../kafka/source/KafkaSourceBuilder.java | 52 +++++++++++++-- .../kafka/source/KafkaSourceBuilderTest.java | 65 +++++++++++++++++++ .../pyflink/datastream/connectors/kafka.py | 4 -- 8 files changed, 114 insertions(+), 17 deletions(-) diff --git a/docs/content.zh/docs/connectors/datastream/kafka.md b/docs/content.zh/docs/connectors/datastream/kafka.md index 9c54aec7d..ad531d72b 100644 --- a/docs/content.zh/docs/connectors/datastream/kafka.md +++ b/docs/content.zh/docs/connectors/datastream/kafka.md @@ -222,8 +222,6 @@ Kafka Source 支持流式和批式两种运行模式。默认情况下,KafkaSo Kafka consumer 的配置可以参考 [Apache Kafka 文档](http://kafka.apache.org/documentation/#consumerconfigs)。 请注意,即使指定了以下配置项,构建器也会将其覆盖: -- ```key.deserializer``` 始终设置为 ByteArrayDeserializer -- ```value.deserializer``` 始终设置为 ByteArrayDeserializer - ```auto.offset.reset.strategy``` 被 OffsetsInitializer#getAutoOffsetResetStrategy() 覆盖 - ```partition.discovery.interval.ms``` 会在批模式下被覆盖为 -1 diff --git a/docs/content.zh/docs/connectors/table/upsert-kafka.md b/docs/content.zh/docs/connectors/table/upsert-kafka.md index 9612de0b8..df2c13878 100644 --- a/docs/content.zh/docs/connectors/table/upsert-kafka.md +++ b/docs/content.zh/docs/connectors/table/upsert-kafka.md @@ -136,7 +136,7 @@ of all available metadata fields. 该选项可以传递任意的 Kafka 参数。选项的后缀名必须匹配定义在 Kafka 参数文档中的参数名。 Flink 会自动移除 选项名中的 "properties." 前缀,并将转换后的键名以及值传入 KafkaClient。 例如,你可以通过 'properties.allow.auto.create.topics' = 'false' - 来禁止自动创建 topic。 但是,某些选项,例如'key.deserializer''value.deserializer' 是不允许通过该方式传递参数,因为 Flink 会重写这些参数的值。 + 来禁止自动创建 topic。 但是,某些选项,例如'auto.offset.reset' 是不允许通过该方式传递参数,因为 Flink 会重写这些参数的值。 diff --git a/docs/content/docs/connectors/datastream/kafka.md b/docs/content/docs/connectors/datastream/kafka.md index 0ab35af6a..1de700bc1 100644 --- a/docs/content/docs/connectors/datastream/kafka.md +++ b/docs/content/docs/connectors/datastream/kafka.md @@ -235,8 +235,6 @@ for more details. Please note that the following keys will be overridden by the builder even if it is configured: -- ```key.deserializer``` is always set to ```ByteArrayDeserializer``` -- ```value.deserializer``` is always set to ```ByteArrayDeserializer``` - ```auto.offset.reset.strategy``` is overridden by ```OffsetsInitializer#getAutoOffsetResetStrategy()``` for the starting offsets - ```partition.discovery.interval.ms``` is overridden to -1 when diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md index 0256301ba..34081219d 100644 --- a/docs/content/docs/connectors/table/kafka.md +++ b/docs/content/docs/connectors/table/kafka.md @@ -233,7 +233,7 @@ Connector Options (none) String - This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in Kafka Configuration documentation. Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'key.deserializer' and 'value.deserializer'. + This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in Kafka Configuration documentation. Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'auto.offset.reset'. diff --git a/docs/content/docs/connectors/table/upsert-kafka.md b/docs/content/docs/connectors/table/upsert-kafka.md index 814ff9bcb..eb662349c 100644 --- a/docs/content/docs/connectors/table/upsert-kafka.md +++ b/docs/content/docs/connectors/table/upsert-kafka.md @@ -144,7 +144,7 @@ Connector Options (none) String - This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in Kafka Configuration documentation. Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'key.deserializer' and 'value.deserializer'. + This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in Kafka Configuration documentation. Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'auto.offset.reset'. diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java index dcad476ba..78a4b0b60 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java @@ -31,9 +31,12 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.Deserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -379,8 +382,6 @@ public KafkaSourceBuilder setRackIdSupplier(SerializableSupplier ra * created. * *

    - *
  • key.deserializer is always set to {@link ByteArrayDeserializer}. - *
  • value.deserializer is always set to {@link ByteArrayDeserializer}. *
  • auto.offset.reset.strategy is overridden by {@link * OffsetsInitializer#getAutoOffsetResetStrategy()} for the starting offsets, which is by * default {@link OffsetsInitializer#earliest()}. @@ -405,8 +406,6 @@ public KafkaSourceBuilder setProperty(String key, String value) { * created. * *
      - *
    • key.deserializer is always set to {@link ByteArrayDeserializer}. - *
    • value.deserializer is always set to {@link ByteArrayDeserializer}. *
    • auto.offset.reset.strategy is overridden by {@link * OffsetsInitializer#getAutoOffsetResetStrategy()} for the starting offsets, which is by * default {@link OffsetsInitializer#earliest()}. @@ -457,11 +456,11 @@ private void parseAndSetRequiredProperties() { maybeOverride( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - true); + false); maybeOverride( ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - true); + false); if (!props.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { LOG.warn( "Offset commit on checkpoint is disabled because {} is not specified", @@ -534,6 +533,47 @@ private void sanityCheck() { if (stoppingOffsetsInitializer instanceof OffsetsInitializerValidator) { ((OffsetsInitializerValidator) stoppingOffsetsInitializer).validate(props); } + if (props.containsKey(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) { + checkDeserializer(props.getProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)); + } + if (props.containsKey(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)) { + checkDeserializer(props.getProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)); + } + } + + private void checkDeserializer(String deserializer) { + try { + Class deserClass = Class.forName(deserializer); + if (!Deserializer.class.isAssignableFrom(deserClass)) { + throw new IllegalArgumentException( + String.format( + "Deserializer class %s is not a subclass of %s", + deserializer, Deserializer.class.getName())); + } + + // Get the generic type information + Type[] interfaces = deserClass.getGenericInterfaces(); + for (Type iface : interfaces) { + if (iface instanceof ParameterizedType) { + ParameterizedType parameterizedType = (ParameterizedType) iface; + Type rawType = parameterizedType.getRawType(); + + // Check if it's Deserializer + if (rawType == Deserializer.class) { + Type[] typeArguments = parameterizedType.getActualTypeArguments(); + if (typeArguments.length != 1 || typeArguments[0] != byte[].class) { + throw new IllegalArgumentException( + String.format( + "Deserializer class %s does not deserialize byte[]", + deserializer)); + } + } + } + } + } catch (ClassNotFoundException e) { + throw new IllegalArgumentException( + String.format("Deserializer class %s not found", deserializer), e); + } } private boolean offsetCommitEnabledManually() { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java index 236e9618f..2829f01e0 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java @@ -27,15 +27,20 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Set; import java.util.regex.Pattern; +import java.util.stream.Stream; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -191,6 +196,27 @@ public void testSettingCustomKafkaSubscriber() { "Cannot use partitions for consumption because a ExampleCustomSubscriber is already set for consumption."); } + @ParameterizedTest + @MethodSource("provideSettingCustomDeserializerTestParameters") + public void testSettingCustomDeserializer(String propertyKey, String propertyValue) { + final KafkaSource kafkaSource = + getBasicBuilder().setProperty(propertyKey, propertyValue).build(); + assertThat( + kafkaSource + .getConfiguration() + .get(ConfigOptions.key(propertyKey).stringType().noDefaultValue())) + .isEqualTo(propertyValue); + } + + @ParameterizedTest + @MethodSource("provideInvalidCustomDeserializersTestParameters") + public void testSettingInvalidCustomDeserializers( + String propertyKey, String propertyValue, String expectedError) { + assertThatThrownBy(() -> getBasicBuilder().setProperty(propertyKey, propertyValue).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining(expectedError); + } + private KafkaSourceBuilder getBasicBuilder() { return new KafkaSourceBuilder() .setBootstrapServers("testServer") @@ -206,4 +232,43 @@ public Set getSubscribedTopicPartitions(AdminClient adminClient) return Collections.singleton(new TopicPartition("topic", 0)); } } + + private static Stream provideSettingCustomDeserializerTestParameters() { + return Stream.of( + Arguments.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + TestByteArrayDeserializer.class.getName()), + Arguments.of( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + TestByteArrayDeserializer.class.getName())); + } + + private static Stream provideInvalidCustomDeserializersTestParameters() { + String deserOne = String.class.getName(); + String deserTwo = "NoneExistentClass"; + String deserThree = StringDeserializer.class.getName(); + return Stream.of( + Arguments.of( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + deserOne, + String.format( + "Deserializer class %s is not a subclass of org.apache.kafka.common.serialization.Deserializer", + deserOne)), + Arguments.of( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + deserTwo, + String.format("Deserializer class %s not found", deserTwo)), + Arguments.of( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + deserThree, + String.format( + "Deserializer class %s does not deserialize byte[]", deserThree)), + Arguments.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + deserThree, + String.format( + "Deserializer class %s does not deserialize byte[]", deserThree))); + } + + private class TestByteArrayDeserializer extends ByteArrayDeserializer {} } diff --git a/flink-python/pyflink/datastream/connectors/kafka.py b/flink-python/pyflink/datastream/connectors/kafka.py index 0e0a12893..062c5b2e8 100644 --- a/flink-python/pyflink/datastream/connectors/kafka.py +++ b/flink-python/pyflink/datastream/connectors/kafka.py @@ -629,8 +629,6 @@ def set_property(self, key: str, value: str) -> 'KafkaSourceBuilder': Note that the following keys will be overridden by the builder when the KafkaSource is created. - * ``key.deserializer`` is always set to ByteArrayDeserializer. - * ``value.deserializer`` is always set to ByteArrayDeserializer. * ``auto.offset.reset.strategy`` is overridden by AutoOffsetResetStrategy returned by :class:`KafkaOffsetsInitializer` for the starting offsets, which is by default :meth:`KafkaOffsetsInitializer.earliest`. @@ -652,8 +650,6 @@ def set_properties(self, props: Dict) -> 'KafkaSourceBuilder': Note that the following keys will be overridden by the builder when the KafkaSource is created. - * ``key.deserializer`` is always set to ByteArrayDeserializer. - * ``value.deserializer`` is always set to ByteArrayDeserializer. * ``auto.offset.reset.strategy`` is overridden by AutoOffsetResetStrategy returned by :class:`KafkaOffsetsInitializer` for the starting offsets, which is by default :meth:`KafkaOffsetsInitializer.earliest`. From b5b8076d2802481980dc5d9c0e9762cdf03848ad Mon Sep 17 00:00:00 2001 From: gongzhongqiang Date: Wed, 21 Aug 2024 13:19:03 +0800 Subject: [PATCH 275/322] [FLINK-36083][docs] Fix kafka table api doc's Connector Options table broken. (#112) --- docs/content.zh/docs/connectors/datastream/dynamic-kafka.md | 2 ++ docs/content/docs/connectors/datastream/dynamic-kafka.md | 2 ++ docs/content/docs/connectors/table/kafka.md | 1 + 3 files changed, 5 insertions(+) diff --git a/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md b/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md index 39adb936c..abe7a7532 100644 --- a/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md +++ b/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md @@ -246,3 +246,5 @@ Kafka cluster. To understand more behind the scenes, please read [FLIP-246](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320) for more details and discussion. + +{{< top >}} diff --git a/docs/content/docs/connectors/datastream/dynamic-kafka.md b/docs/content/docs/connectors/datastream/dynamic-kafka.md index 08fa2401e..e64b93e67 100644 --- a/docs/content/docs/connectors/datastream/dynamic-kafka.md +++ b/docs/content/docs/connectors/datastream/dynamic-kafka.md @@ -246,3 +246,5 @@ Kafka cluster. To understand more behind the scenes, please read [FLIP-246](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320) for more details and discussion. + +{{< top >}} diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md index 34081219d..55a5dbf27 100644 --- a/docs/content/docs/connectors/table/kafka.md +++ b/docs/content/docs/connectors/table/kafka.md @@ -337,6 +337,7 @@ Connector Options
      scan.bounded.mode
      optional + no unbounded Enum Bounded mode for Kafka consumer, valid values are 'latest-offset', 'group-offsets', 'timestamp' and 'specific-offsets'. From ea3a6417e290d02c1b87c7d0afbb6db910b8faa8 Mon Sep 17 00:00:00 2001 From: Yop Lee Date: Thu, 21 Sep 2023 11:05:56 +0900 Subject: [PATCH 276/322] [FLINK-33201][Connectors/Kafka] Fix memory leak in CachingTopicSelector --- .../kafka/sink/KafkaRecordSerializationSchemaBuilder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java index 34cf6ef00..92eb625b2 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java @@ -284,7 +284,7 @@ private static class CachingTopicSelector implements Function, S public String apply(IN in) { final String topic = cache.getOrDefault(in, topicSelector.apply(in)); cache.put(in, topic); - if (cache.size() == CACHE_RESET_SIZE) { + if (cache.size() >= CACHE_RESET_SIZE) { cache.clear(); } return topic; From 020bc33c4e327437bc9f352f4b5d9b479a42568b Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 29 Aug 2024 10:32:25 +0200 Subject: [PATCH 277/322] [FLINK-35886] Leave a note for future Flink 2.0 upgrade Fixing FLIP-471 in the old source would require some extra work, that we hopefully can avoid by removing the old source before doing the upgrade. --- .../connectors/kafka/internals/AbstractFetcher.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java index 841d45288..41b5ad24c 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java @@ -435,6 +435,14 @@ private List> createPartitionStateHolders( kafkaHandle, deserializedWatermarkStrategy.createTimestampAssigner( () -> consumerMetricGroup), + // When upgrading to Flink 2.0, context has to provide also + // the input activity clock. This is not trivial for the old + // sources. Ideally we should drop this old source before + // this connector is upgraded to Flink 2.0. Otherwise, we + // can avoid the compilation error without fixing the bug + // addressed by the FLIP-471, by returning SystemClock, + // which would reproduce the pre-FLIP-471 behavior (without + // fixing the underlying bug). deserializedWatermarkStrategy.createWatermarkGenerator( () -> consumerMetricGroup), immediateOutput, From 268f6b5c209f119611242400e6d7d2fd1e65a0a5 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Fri, 30 Aug 2024 12:08:02 +0200 Subject: [PATCH 278/322] [FLINK-36176] Remove support for Kafka 0.1 (#115) Warp up FLINK-19152 by also removing all related producer code. Move FlinkKafkaProducerBase.getPropertiesFromBrokerList to a new test utility class as this method is used by some tests. --- .../kafka/FlinkKafkaProducer011.java | 72 --- .../kafka/FlinkKafkaProducerBase.java | 442 ---------------- .../kafka/FlinkKafkaProducerBaseTest.java | 482 ------------------ ...inkKafkaProducerMigrationOperatorTest.java | 60 --- .../kafka/KafkaConsumerTestBase.java | 11 +- .../kafka/KafkaProducerTestBase.java | 4 +- .../kafka/testutils/DataGenerators.java | 7 +- .../kafka/testutils/KafkaUtils.java | 41 ++ ...gration-kafka-producer-flink-1.10-snapshot | Bin 2032 -> 0 bytes ...gration-kafka-producer-flink-1.11-snapshot | Bin 2040 -> 0 bytes ...igration-kafka-producer-flink-1.8-snapshot | Bin 2032 -> 0 bytes ...igration-kafka-producer-flink-1.9-snapshot | Bin 2032 -> 0 bytes 12 files changed, 51 insertions(+), 1068 deletions(-) delete mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java delete mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java delete mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java delete mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/KafkaUtils.java delete mode 100644 flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.10-snapshot delete mode 100644 flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.11-snapshot delete mode 100644 flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.8-snapshot delete mode 100644 flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.9-snapshot diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java deleted file mode 100644 index c5b008c1e..000000000 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.connectors.kafka; - -import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; - -/** - * Compatibility class to make migration possible from the 0.11 connector to the universal one. - * - *

      Problem is that FlinkKafkaProducer (universal) and FlinkKafkaProducer011 have different names - * and they both defined static classes NextTransactionalIdHint, KafkaTransactionState and - * KafkaTransactionContext inside the parent classes. This is causing incompatibility problems since - * for example FlinkKafkaProducer011.KafkaTransactionState and - * FlinkKafkaProducer.KafkaTransactionState are treated as completely incompatible classes, despite - * being identical. - * - *

      This issue is solved by using custom serialization logic: keeping a fake/dummy - * FlinkKafkaProducer011.*Serializer classes in the universal connector (this class), as entry - * points for the deserialization and converting them to FlinkKafkaProducer.*Serializer counter - * parts. After all serialized binary data are exactly the same in all of those cases. - * - *

      For more details check FLINK-11249 and the discussion in the pull requests. - */ -// CHECKSTYLE:OFF: JavadocType -public class FlinkKafkaProducer011 { - public static class NextTransactionalIdHintSerializer { - public static final class NextTransactionalIdHintSerializerSnapshot - extends SimpleTypeSerializerSnapshot { - public NextTransactionalIdHintSerializerSnapshot() { - super(FlinkKafkaProducer.NextTransactionalIdHintSerializer::new); - } - } - } - - public static class ContextStateSerializer { - public static final class ContextStateSerializerSnapshot - extends SimpleTypeSerializerSnapshot { - public ContextStateSerializerSnapshot() { - super(FlinkKafkaProducer.ContextStateSerializer::new); - } - } - } - - public static class TransactionStateSerializer { - public static final class TransactionStateSerializerSnapshot - extends SimpleTypeSerializerSnapshot { - public TransactionStateSerializerSnapshot() { - super(FlinkKafkaProducer.TransactionStateSerializer::new); - } - } - } - - public static class NextTransactionalIdHint - extends FlinkKafkaProducer.NextTransactionalIdHint {} -} -// CHECKSTYLE:ON: JavadocType diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java deleted file mode 100644 index 234fd9afa..000000000 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java +++ /dev/null @@ -1,442 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.connectors.kafka; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.serialization.RuntimeContextInitializationContextAdapters; -import org.apache.flink.api.java.ClosureCleaner; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.connectors.kafka.internals.KeyedSerializationSchemaWrapper; -import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; -import org.apache.flink.util.NetUtils; -import org.apache.flink.util.SerializableObject; - -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.Metric; -import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -import static java.util.Objects.requireNonNull; - -/** - * Flink Sink to produce data into a Kafka topic. - * - *

      Please note that this producer provides at-least-once reliability guarantees when checkpoints - * are enabled and setFlushOnCheckpoint(true) is set. Otherwise, the producer doesn't provide any - * reliability guarantees. - * - * @param Type of the messages to write into Kafka. - */ -@Internal -public abstract class FlinkKafkaProducerBase extends RichSinkFunction - implements CheckpointedFunction { - - private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class); - - private static final long serialVersionUID = 1L; - - /** Configuration key for disabling the metrics reporting. */ - public static final String KEY_DISABLE_METRICS = "flink.disable-metrics"; - - /** User defined properties for the Producer. */ - protected final Properties producerConfig; - - /** The name of the default topic this producer is writing data to. */ - protected final String defaultTopicId; - - /** - * (Serializable) SerializationSchema for turning objects used with Flink into. byte[] for - * Kafka. - */ - protected final KeyedSerializationSchema schema; - - /** User-provided partitioner for assigning an object to a Kafka partition for each topic. */ - protected final FlinkKafkaPartitioner flinkKafkaPartitioner; - - /** Partitions of each topic. */ - protected final Map topicPartitionsMap; - - /** Flag indicating whether to accept failures (and log them), or to fail on failures. */ - protected boolean logFailuresOnly; - - /** - * If true, the producer will wait until all outstanding records have been send to the broker. - */ - protected boolean flushOnCheckpoint = true; - - // -------------------------------- Runtime fields ------------------------------------------ - - /** KafkaProducer instance. */ - protected transient KafkaProducer producer; - - /** The callback than handles error propagation or logging callbacks. */ - protected transient Callback callback; - - /** Errors encountered in the async producer are stored here. */ - protected transient volatile Exception asyncException; - - /** Lock for accessing the pending records. */ - protected final SerializableObject pendingRecordsLock = new SerializableObject(); - - /** Number of unacknowledged records. */ - protected long pendingRecords; - - /** - * The main constructor for creating a FlinkKafkaProducer. - * - * @param defaultTopicId The default topic to write data to - * @param serializationSchema A serializable serialization schema for turning user objects into - * a kafka-consumable byte[] supporting key/value messages - * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is - * the only required argument. - * @param customPartitioner A serializable partitioner for assigning messages to Kafka - * partitions. Passing null will use Kafka's partitioner. - */ - public FlinkKafkaProducerBase( - String defaultTopicId, - KeyedSerializationSchema serializationSchema, - Properties producerConfig, - FlinkKafkaPartitioner customPartitioner) { - requireNonNull(defaultTopicId, "TopicID not set"); - requireNonNull(serializationSchema, "serializationSchema not set"); - requireNonNull(producerConfig, "producerConfig not set"); - ClosureCleaner.clean( - customPartitioner, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true); - ClosureCleaner.ensureSerializable(serializationSchema); - - this.defaultTopicId = defaultTopicId; - this.schema = serializationSchema; - this.producerConfig = producerConfig; - this.flinkKafkaPartitioner = customPartitioner; - - // set the producer configuration properties for kafka record key value serializers. - if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) { - this.producerConfig.put( - ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - ByteArraySerializer.class.getName()); - } else { - LOG.warn( - "Overwriting the '{}' is not recommended", - ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); - } - - if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) { - this.producerConfig.put( - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - ByteArraySerializer.class.getName()); - } else { - LOG.warn( - "Overwriting the '{}' is not recommended", - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); - } - - // eagerly ensure that bootstrap servers are set. - if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) { - throw new IllegalArgumentException( - ProducerConfig.BOOTSTRAP_SERVERS_CONFIG - + " must be supplied in the producer config properties."); - } - - this.topicPartitionsMap = new HashMap<>(); - } - - // ---------------------------------- Properties -------------------------- - - /** - * Defines whether the producer should fail on errors, or only log them. If this is set to true, - * then exceptions will be only logged, if set to false, exceptions will be eventually thrown - * and cause the streaming program to fail (and enter recovery). - * - * @param logFailuresOnly The flag to indicate logging-only on exceptions. - */ - public void setLogFailuresOnly(boolean logFailuresOnly) { - this.logFailuresOnly = logFailuresOnly; - } - - /** - * If set to true, the Flink producer will wait for all outstanding messages in the Kafka - * buffers to be acknowledged by the Kafka producer on a checkpoint. This way, the producer can - * guarantee that messages in the Kafka buffers are part of the checkpoint. - * - * @param flush Flag indicating the flushing mode (true = flush on checkpoint) - */ - public void setFlushOnCheckpoint(boolean flush) { - this.flushOnCheckpoint = flush; - } - - /** Used for testing only. */ - @VisibleForTesting - protected KafkaProducer getKafkaProducer(Properties props) { - return new KafkaProducer<>(props); - } - - // ----------------------------------- Utilities -------------------------- - - /** Initializes the connection to Kafka. */ - @Override - public void open(Configuration configuration) throws Exception { - if (schema instanceof KeyedSerializationSchemaWrapper) { - ((KeyedSerializationSchemaWrapper) schema) - .getSerializationSchema() - .open( - RuntimeContextInitializationContextAdapters.serializationAdapter( - getRuntimeContext(), - metricGroup -> metricGroup.addGroup("user"))); - } - producer = getKafkaProducer(this.producerConfig); - - RuntimeContext ctx = getRuntimeContext(); - - if (null != flinkKafkaPartitioner) { - flinkKafkaPartitioner.open( - ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks()); - } - - LOG.info( - "Starting FlinkKafkaProducer ({}/{}) to produce into default topic {}", - ctx.getIndexOfThisSubtask() + 1, - ctx.getNumberOfParallelSubtasks(), - defaultTopicId); - - // register Kafka metrics to Flink accumulators - if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) { - Map metrics = this.producer.metrics(); - - if (metrics == null) { - // MapR's Kafka implementation returns null here. - LOG.info("Producer implementation does not support metrics"); - } else { - final MetricGroup kafkaMetricGroup = - getRuntimeContext().getMetricGroup().addGroup("KafkaProducer"); - for (Map.Entry metric : metrics.entrySet()) { - kafkaMetricGroup.gauge( - metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); - } - } - } - - if (flushOnCheckpoint - && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { - LOG.warn( - "Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing."); - flushOnCheckpoint = false; - } - - if (logFailuresOnly) { - callback = - new Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception e) { - if (e != null) { - LOG.error( - "Error while sending record to Kafka: " + e.getMessage(), - e); - } - acknowledgeMessage(); - } - }; - } else { - callback = - new Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception exception) { - if (exception != null && asyncException == null) { - asyncException = exception; - } - acknowledgeMessage(); - } - }; - } - } - - /** - * Called when new data arrives to the sink, and forwards it to Kafka. - * - * @param next The incoming data - */ - @Override - public void invoke(IN next, Context context) throws Exception { - // propagate asynchronous errors - checkErroneous(); - - byte[] serializedKey = schema.serializeKey(next); - byte[] serializedValue = schema.serializeValue(next); - String targetTopic = schema.getTargetTopic(next); - if (targetTopic == null) { - targetTopic = defaultTopicId; - } - - int[] partitions = this.topicPartitionsMap.get(targetTopic); - if (null == partitions) { - partitions = getPartitionsByTopic(targetTopic, producer); - this.topicPartitionsMap.put(targetTopic, partitions); - } - - ProducerRecord record; - if (flinkKafkaPartitioner == null) { - record = new ProducerRecord<>(targetTopic, serializedKey, serializedValue); - } else { - record = - new ProducerRecord<>( - targetTopic, - flinkKafkaPartitioner.partition( - next, serializedKey, serializedValue, targetTopic, partitions), - serializedKey, - serializedValue); - } - if (flushOnCheckpoint) { - synchronized (pendingRecordsLock) { - pendingRecords++; - } - } - producer.send(record, callback); - } - - @Override - public void close() throws Exception { - if (producer != null) { - producer.close(); - } - - // make sure we propagate pending errors - checkErroneous(); - } - - // ------------------- Logic for handling checkpoint flushing -------------------------- // - - private void acknowledgeMessage() { - if (flushOnCheckpoint) { - synchronized (pendingRecordsLock) { - pendingRecords--; - if (pendingRecords == 0) { - pendingRecordsLock.notifyAll(); - } - } - } - } - - /** Flush pending records. */ - protected abstract void flush(); - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - // nothing to do - } - - @Override - public void snapshotState(FunctionSnapshotContext ctx) throws Exception { - // check for asynchronous errors and fail the checkpoint if necessary - checkErroneous(); - - if (flushOnCheckpoint) { - // flushing is activated: We need to wait until pendingRecords is 0 - flush(); - synchronized (pendingRecordsLock) { - if (pendingRecords != 0) { - throw new IllegalStateException( - "Pending record count must be zero at this point: " + pendingRecords); - } - - // if the flushed requests has errors, we should propagate it also and fail the - // checkpoint - checkErroneous(); - } - } - } - - // ----------------------------------- Utilities -------------------------- - - protected void checkErroneous() throws Exception { - Exception e = asyncException; - if (e != null) { - // prevent double throwing - asyncException = null; - throw new Exception("Failed to send data to Kafka: " + e.getMessage(), e); - } - } - - public static Properties getPropertiesFromBrokerList(String brokerList) { - String[] elements = brokerList.split(","); - - // validate the broker addresses - for (String broker : elements) { - NetUtils.getCorrectHostnamePort(broker); - } - - Properties props = new Properties(); - props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); - return props; - } - - protected static int[] getPartitionsByTopic( - String topic, KafkaProducer producer) { - // the fetched list is immutable, so we're creating a mutable copy in order to sort it - List partitionsList = new ArrayList<>(producer.partitionsFor(topic)); - - // sort the partitions by partition id to make sure the fetched partition list is the same - // across subtasks - Collections.sort( - partitionsList, - new Comparator() { - @Override - public int compare(PartitionInfo o1, PartitionInfo o2) { - return Integer.compare(o1.partition(), o2.partition()); - } - }); - - int[] partitions = new int[partitionsList.size()]; - for (int i = 0; i < partitions.length; i++) { - partitions[i] = partitionsList.get(i).partition(); - } - - return partitions; - } - - @VisibleForTesting - protected long numPendingRecords() { - synchronized (pendingRecordsLock) { - return pendingRecords; - } - } -} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java deleted file mode 100644 index 4274fcff2..000000000 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java +++ /dev/null @@ -1,482 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.connectors.kafka; - -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.serialization.SimpleStringSchema; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.testutils.CheckedThread; -import org.apache.flink.core.testutils.MultiShotLatch; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.functions.sink.SinkContextUtil; -import org.apache.flink.streaming.api.operators.StreamSink; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.connectors.kafka.internals.KeyedSerializationSchemaWrapper; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; - -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -/** Tests for the {@link FlinkKafkaProducerBase}. */ -public class FlinkKafkaProducerBaseTest { - - /** Tests that the constructor eagerly checks bootstrap servers are set in config. */ - @Test(expected = IllegalArgumentException.class) - public void testInstantiationFailsWhenBootstrapServersMissing() throws Exception { - // no bootstrap servers set in props - Properties props = new Properties(); - // should throw IllegalArgumentException - new DummyFlinkKafkaProducer<>( - props, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), null); - } - - /** - * Tests that constructor defaults to key value serializers in config to byte array - * deserializers if not set. - */ - @Test - public void testKeyValueDeserializersSetIfMissing() throws Exception { - Properties props = new Properties(); - props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:12345"); - // should set missing key value deserializers - new DummyFlinkKafkaProducer<>( - props, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), null); - - assertThat(props) - .containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG) - .containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); - assertThat(props.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) - .isEqualTo(ByteArraySerializer.class.getName()); - assertThat(props.getProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) - .isEqualTo(ByteArraySerializer.class.getName()); - } - - /** Tests that partitions list is determinate and correctly provided to custom partitioner. */ - @SuppressWarnings("unchecked") - @Test - public void testPartitionerInvokedWithDeterminatePartitionList() throws Exception { - FlinkKafkaPartitioner mockPartitioner = mock(FlinkKafkaPartitioner.class); - - RuntimeContext mockRuntimeContext = mock(StreamingRuntimeContext.class); - when(mockRuntimeContext.getIndexOfThisSubtask()).thenReturn(0); - when(mockRuntimeContext.getNumberOfParallelSubtasks()).thenReturn(1); - - // out-of-order list of 4 partitions - List mockPartitionsList = new ArrayList<>(4); - mockPartitionsList.add( - new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 3, null, null, null)); - mockPartitionsList.add( - new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 1, null, null, null)); - mockPartitionsList.add( - new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 0, null, null, null)); - mockPartitionsList.add( - new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 2, null, null, null)); - - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - mockPartitioner); - producer.setRuntimeContext(mockRuntimeContext); - - final KafkaProducer mockProducer = producer.getMockKafkaProducer(); - when(mockProducer.partitionsFor(anyString())).thenReturn(mockPartitionsList); - when(mockProducer.metrics()).thenReturn(null); - - producer.open(new Configuration()); - verify(mockPartitioner, times(1)).open(0, 1); - - producer.invoke("foobar", SinkContextUtil.forTimestamp(0)); - verify(mockPartitioner, times(1)) - .partition( - "foobar", - null, - "foobar".getBytes(), - DummyFlinkKafkaProducer.DUMMY_TOPIC, - new int[] {0, 1, 2, 3}); - } - - /** - * Test ensuring that if an invoke call happens right after an async exception is caught, it - * should be rethrown. - */ - @Test - public void testAsyncErrorRethrownOnInvoke() throws Throwable { - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - null); - - OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer)); - - testHarness.open(); - - testHarness.processElement(new StreamRecord<>("msg-1")); - - // let the message request return an async exception - producer.getPendingCallbacks() - .get(0) - .onCompletion(null, new Exception("artificial async exception")); - - try { - testHarness.processElement(new StreamRecord<>("msg-2")); - } catch (Exception e) { - // the next invoke should rethrow the async exception - assertThat(e.getCause().getMessage()).contains("artificial async exception"); - - // test succeeded - return; - } - - fail("unknown failure"); - } - - /** - * Test ensuring that if a snapshot call happens right after an async exception is caught, it - * should be rethrown. - */ - @Test - public void testAsyncErrorRethrownOnCheckpoint() throws Throwable { - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - null); - - OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer)); - - testHarness.open(); - - testHarness.processElement(new StreamRecord<>("msg-1")); - - // let the message request return an async exception - producer.getPendingCallbacks() - .get(0) - .onCompletion(null, new Exception("artificial async exception")); - - try { - testHarness.snapshot(123L, 123L); - } catch (Exception e) { - // the next invoke should rethrow the async exception - assertThat(e.getCause().getMessage()).contains("artificial async exception"); - - // test succeeded - return; - } - - fail("unknown failure"); - } - - /** - * Test ensuring that if an async exception is caught for one of the flushed requests on - * checkpoint, it should be rethrown; we set a timeout because the test will not finish if the - * logic is broken. - * - *

      Note that this test does not test the snapshot method is blocked correctly when there are - * pending records. The test for that is covered in testAtLeastOnceProducer. - */ - @SuppressWarnings("unchecked") - @Test(timeout = 5000) - public void testAsyncErrorRethrownOnCheckpointAfterFlush() throws Throwable { - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - null); - producer.setFlushOnCheckpoint(true); - - final KafkaProducer mockProducer = producer.getMockKafkaProducer(); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer)); - - testHarness.open(); - - testHarness.processElement(new StreamRecord<>("msg-1")); - testHarness.processElement(new StreamRecord<>("msg-2")); - testHarness.processElement(new StreamRecord<>("msg-3")); - - verify(mockProducer, times(3)).send(any(ProducerRecord.class), any(Callback.class)); - - // only let the first callback succeed for now - producer.getPendingCallbacks().get(0).onCompletion(null, null); - - CheckedThread snapshotThread = - new CheckedThread() { - @Override - public void go() throws Exception { - // this should block at first, since there are still two pending records - // that needs to be flushed - testHarness.snapshot(123L, 123L); - } - }; - snapshotThread.start(); - - // let the 2nd message fail with an async exception - producer.getPendingCallbacks() - .get(1) - .onCompletion(null, new Exception("artificial async failure for 2nd message")); - producer.getPendingCallbacks().get(2).onCompletion(null, null); - - try { - snapshotThread.sync(); - } catch (Exception e) { - // the snapshot should have failed with the async exception - assertThat(e.getCause().getMessage()) - .contains("artificial async failure for 2nd message"); - - // test succeeded - return; - } - - fail("unknown failure"); - } - - /** - * Test ensuring that the producer is not dropping buffered records; we set a timeout because - * the test will not finish if the logic is broken. - */ - @SuppressWarnings("unchecked") - @Test(timeout = 10000) - public void testAtLeastOnceProducer() throws Throwable { - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - null); - producer.setFlushOnCheckpoint(true); - - final KafkaProducer mockProducer = producer.getMockKafkaProducer(); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer)); - - testHarness.open(); - - testHarness.processElement(new StreamRecord<>("msg-1")); - testHarness.processElement(new StreamRecord<>("msg-2")); - testHarness.processElement(new StreamRecord<>("msg-3")); - - verify(mockProducer, times(3)).send(any(ProducerRecord.class), any(Callback.class)); - assertThat(producer.getPendingSize()).isEqualTo(3); - - // start a thread to perform checkpointing - CheckedThread snapshotThread = - new CheckedThread() { - @Override - public void go() throws Exception { - // this should block until all records are flushed; - // if the snapshot implementation returns before pending records are - // flushed, - testHarness.snapshot(123L, 123L); - } - }; - snapshotThread.start(); - - // before proceeding, make sure that flushing has started and that the snapshot is still - // blocked; - // this would block forever if the snapshot didn't perform a flush - producer.waitUntilFlushStarted(); - assertThat(snapshotThread.isAlive()) - .as("Snapshot returned before all records were flushed") - .isTrue(); - - // now, complete the callbacks - producer.getPendingCallbacks().get(0).onCompletion(null, null); - assertThat(snapshotThread.isAlive()) - .as("Snapshot returned before all records were flushed") - .isTrue(); - assertThat(producer.getPendingSize()).isEqualTo(2); - - producer.getPendingCallbacks().get(1).onCompletion(null, null); - assertThat(snapshotThread.isAlive()) - .as("Snapshot returned before all records were flushed") - .isTrue(); - assertThat(producer.getPendingSize()).isEqualTo(1); - - producer.getPendingCallbacks().get(2).onCompletion(null, null); - assertThat(producer.getPendingSize()).isEqualTo(0); - - // this would fail with an exception if flushing wasn't completed before the snapshot method - // returned - snapshotThread.sync(); - - testHarness.close(); - } - - /** - * This test is meant to assure that testAtLeastOnceProducer is valid by testing that if - * flushing is disabled, the snapshot method does indeed finishes without waiting for pending - * records; we set a timeout because the test will not finish if the logic is broken. - */ - @SuppressWarnings("unchecked") - @Test(timeout = 5000) - public void testDoesNotWaitForPendingRecordsIfFlushingDisabled() throws Throwable { - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - null); - producer.setFlushOnCheckpoint(false); - - final KafkaProducer mockProducer = producer.getMockKafkaProducer(); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer)); - - testHarness.open(); - - testHarness.processElement(new StreamRecord<>("msg")); - - // make sure that all callbacks have not been completed - verify(mockProducer, times(1)).send(any(ProducerRecord.class), any(Callback.class)); - - // should return even if there are pending records - testHarness.snapshot(123L, 123L); - - testHarness.close(); - } - - // ------------------------------------------------------------------------ - - private static class DummyFlinkKafkaProducer extends FlinkKafkaProducerBase { - private static final long serialVersionUID = 1L; - - private static final String DUMMY_TOPIC = "dummy-topic"; - - private transient KafkaProducer mockProducer; - private transient List pendingCallbacks; - private transient MultiShotLatch flushLatch; - private boolean isFlushed; - - @SuppressWarnings("unchecked") - DummyFlinkKafkaProducer( - Properties producerConfig, - KeyedSerializationSchema schema, - FlinkKafkaPartitioner partitioner) { - - super(DUMMY_TOPIC, schema, producerConfig, partitioner); - - this.mockProducer = mock(KafkaProducer.class); - when(mockProducer.send(any(ProducerRecord.class), any(Callback.class))) - .thenAnswer( - new Answer() { - @Override - public Object answer(InvocationOnMock invocationOnMock) - throws Throwable { - pendingCallbacks.add(invocationOnMock.getArgument(1)); - return null; - } - }); - - this.pendingCallbacks = new ArrayList<>(); - this.flushLatch = new MultiShotLatch(); - } - - long getPendingSize() { - if (flushOnCheckpoint) { - return numPendingRecords(); - } else { - // when flushing is disabled, the implementation does not - // maintain the current number of pending records to reduce - // the extra locking overhead required to do so - throw new UnsupportedOperationException( - "getPendingSize not supported when flushing is disabled"); - } - } - - List getPendingCallbacks() { - return pendingCallbacks; - } - - KafkaProducer getMockKafkaProducer() { - return mockProducer; - } - - @Override - public void snapshotState(FunctionSnapshotContext ctx) throws Exception { - isFlushed = false; - - super.snapshotState(ctx); - - // if the snapshot implementation doesn't wait until all pending records are flushed, we - // should fail the test - if (flushOnCheckpoint && !isFlushed) { - throw new RuntimeException( - "Flushing is enabled; snapshots should be blocked until all pending records are flushed"); - } - } - - public void waitUntilFlushStarted() throws Exception { - flushLatch.await(); - } - - @SuppressWarnings("unchecked") - @Override - protected KafkaProducer getKafkaProducer(Properties props) { - return (KafkaProducer) mockProducer; - } - - @Override - protected void flush() { - flushLatch.trigger(); - - // simply wait until the producer's pending records become zero. - // This relies on the fact that the producer's Callback implementation - // and pending records tracking logic is implemented correctly, otherwise - // we will loop forever. - while (numPendingRecords() > 0) { - try { - Thread.sleep(10); - } catch (InterruptedException e) { - throw new RuntimeException("Unable to flush producer, task was interrupted"); - } - } - - isFlushed = true; - } - } -} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java deleted file mode 100644 index 5e87f04b8..000000000 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.connectors.kafka; - -import org.apache.flink.FlinkVersion; - -import org.junit.Ignore; -import org.junit.runners.Parameterized; - -import java.util.Arrays; -import java.util.Collection; - -/** - * Migration test from FlinkKafkaProducer011 operator. This test depends on the resource generated - * by {@link FlinkKafkaProducer011MigrationTest#writeSnapshot()}. - * - *

      Warning: We need to rename the generated resource based on the file naming pattern specified - * by the {@link #getOperatorSnapshotPath(FlinkVersion)} method then copy the resource to the path - * also specified by the {@link #getOperatorSnapshotPath(FlinkVersion)} method. - */ -public class FlinkKafkaProducerMigrationOperatorTest extends FlinkKafkaProducerMigrationTest { - @Parameterized.Parameters(name = "Migration Savepoint: {0}") - public static Collection parameters() { - return Arrays.asList( - FlinkVersion.v1_8, FlinkVersion.v1_9, FlinkVersion.v1_10, FlinkVersion.v1_11); - } - - public FlinkKafkaProducerMigrationOperatorTest(FlinkVersion testMigrateVersion) { - super(testMigrateVersion); - } - - @Override - public String getOperatorSnapshotPath(FlinkVersion version) { - return "src/test/resources/kafka-0.11-migration-kafka-producer-flink-" - + version - + "-snapshot"; - } - - @Ignore - @Override - public void writeSnapshot() throws Exception { - throw new UnsupportedOperationException(); - } -} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index 88f6ac607..026d49bdf 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -69,6 +69,7 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.testutils.DataGenerators; import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper; +import org.apache.flink.streaming.connectors.kafka.testutils.KafkaUtils; import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidatingMapper; import org.apache.flink.streaming.connectors.kafka.testutils.ThrottledMapper; import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2FlinkPartitioner; @@ -869,7 +870,7 @@ public void cancel() { } }); Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); producerProperties.putAll(secureProps); kafkaServer.produceIntoKafka(stream, topic, sinkSchema, producerProperties, null); @@ -1550,7 +1551,7 @@ public void cancel() {} new TypeInformationKeyValueSerializationSchema<>( Long.class, PojoValue.class, env.getConfig()); Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null); env.execute("Write KV to Kafka"); @@ -1646,7 +1647,7 @@ public void cancel() {} byte[].class, PojoValue.class, env.getConfig()); Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); producerProperties.putAll(secureProps); kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null); @@ -2288,7 +2289,7 @@ public void cancel() { // the producer must not produce duplicates Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "0"); producerProperties.putAll(secureProps); @@ -2392,7 +2393,7 @@ public void cancel() { // the producer must not produce duplicates Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "0"); producerProperties.putAll(secureProps); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index 624381068..cf3bf463f 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -36,6 +36,7 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper; import org.apache.flink.streaming.connectors.kafka.testutils.IntegerSource; +import org.apache.flink.streaming.connectors.kafka.testutils.KafkaUtils; import org.apache.flink.test.util.SuccessException; import org.apache.flink.test.util.TestUtils; import org.apache.flink.util.Preconditions; @@ -148,8 +149,7 @@ public void cancel() { .setParallelism(1); Properties props = new Properties(); - props.putAll( - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings)); + props.putAll(KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings)); props.putAll(secureProps); // sink partitions into diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java index d660bd2f1..be3651e58 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java @@ -26,7 +26,6 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.streaming.api.operators.StreamSink; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironment; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; @@ -98,8 +97,7 @@ public void cancel() { Properties props = new Properties(); props.putAll( - FlinkKafkaProducerBase.getPropertiesFromBrokerList( - testServer.getBrokerConnectionString())); + KafkaUtils.getPropertiesFromBrokerList(testServer.getBrokerConnectionString())); Properties secureProps = testServer.getSecureProperties(); if (secureProps != null) { props.putAll(testServer.getSecureProperties()); @@ -156,8 +154,7 @@ public void run() { OneInputStreamOperatorTestHarness testHarness = null; try { Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList( - server.getBrokerConnectionString()); + KafkaUtils.getPropertiesFromBrokerList(server.getBrokerConnectionString()); producerProperties.setProperty("retries", "3"); StreamSink sink = diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/KafkaUtils.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/KafkaUtils.java new file mode 100644 index 000000000..eeb8ce047 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/KafkaUtils.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.testutils; + +import org.apache.flink.util.NetUtils; + +import org.apache.kafka.clients.producer.ProducerConfig; + +import java.util.Properties; + +/** Some utilities methods around Kafka. */ +public class KafkaUtils { + public static Properties getPropertiesFromBrokerList(String brokerList) { + String[] elements = brokerList.split(","); + + // validate the broker addresses + for (String broker : elements) { + NetUtils.getCorrectHostnamePort(broker); + } + + Properties props = new Properties(); + props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); + return props; + } +} diff --git a/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.10-snapshot b/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.10-snapshot deleted file mode 100644 index f3e6c74e3d25f891e85e523d4b10f550789ab2f3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2032 zcmcIlKX21O6hG&WIw3kBAt50#WJ5mL##GI~khYF6q=^+fMX-dcb7`(|?jql%g)V#o zMiyo^z6W0bcDk{%BXQ?6I8M^E7VVR4-JO5;-uu0O1|Su!B-cgokm7ej7crt7g@Q1O zBAxW~0ii-au{4QMkCgHc@ZfUn6jbH{39QhwOXnJ>JVjPmHEmn(^{R$$<3mf|wzsRg zX>awn46A3`$O2&Fg)(xbM;E#^@TlcE?OM=kcm1FiINd$h58CcdCvaQM?tW{}0p(6p zu9FL!2en4W>H1FFtu@>?PFtQ9z(*DzK{Q6;0KBZ42!@PO910e5I7Iy+ zf^`LWsg_>MdXo^xhG|xsGJ3&$@Z8>W89E=w1V!Ww$5$hKipG4vgmkI6F2pyj(**Hnm-3W2vp%_IG zjtC#A>WXgD_$q?|XfCl;Q485<#OPey4iU%D%@jdqHy6PoxUYJu9%H;vMT(-hO^WLb z9{?{0)dr(SX?;|(+YH$>r;^>j{K~eU%?-u+I<39wdkr^8d)`Uzt?u%-OH|StRr%{r zXfpUX>VpM}E!@u6;H#msIEQb71}yNcZn5E7q& z16Mu}pMVd*jUy6wBOudIIhF{rW%Z&XVF+RG-dR><>1-9=Q4CI4hag#M;u;_a4DMbArsQ2?6wf!B=h|> z=4e9TDLO+?e3;;=AOVNrlnOG&&|#-6CbZ;k2Ar(vxv2;Oko)uf+p|fOsCxn8!a*Sl z0vr%NR@D{UrtwV%Ji~P#AU(5}~`Z}$><2nu7i+kRR?rrY!w`)}58ddr0 zPiZvxIOJ*sc<0O+$_yUJv=ZUUqxu|BaMg5+nI@>+{iWt<2Hb!3H}XZvpSM`P=6qSI ziWTdH%ReO?JO6(t4j2`(N>?4&e4H>YLlLdKd}|umJ!1aILwuwg$z>+DhN%~47qh?B sUx~&)JvsTBT8BB)0dfl+uS=@H@@f1q(}0UyL0Gxe#_Z!`%bgbIcgr-pwMo} zeJrqZT&=h5j%T-=YTbEbx8!XOd}h%Aq6rFy81_R#N05sMqcNca7%)n4AXvoV2=zw@ z))e5S+PV?zO#>Vmrde#r==qDmb9&EZ=sX+|6p{}(x*Fk8G~q)gq)YitA-+lG`zy@R zguoMYilBHu!Bas(4x=d*WQ?K9PFO-{!QC8qy`tx)A_&0N@2_8;P2xn|4G|YEiclEh zknpjpuJAUEuQTX_#tK^%wSbMsj4s5ji#Uc(st8i2g$U-sebrO72;-$HQWV8)Qru+t z7`Pdz))^fn^-)Y8Q)H8zN_PM9E8BiPHx%pZwDykY)*V0Tc{`R{-{o&ts3bM2^4Fiz zc<^z^)d=vYuG7@mcao{T-1m}pEqa5Fn$S1RRT4Q-dWN@2q;_2YumWq>KWGh}CI)r-Ht zKj07YXky|I@aEmSi8EVpS)jBV+esiZ!!z&uKF|9B0LjueJ3Z7x)g1sfUnnP+Ml`jpfrl-}YS+A0yX$&2&*~o7uGhBrI-cEXb`M(z7ASO@ zavwj~I;u4~R@b%KcCBH*wc7GF2R^fC0MQuvLk#-?p(Dsegi%Q70Q!tl>_)QqstL4MPoi>LWY!IC-HSM-&vh8PcL$SU}Yj3$u!}gM%x8irJyZqe}m83>h{`wOd z4?YgL8UfxJbA~d5`!cOWxbmt#1r%L1-4doLYPWu=d71-vUj2z(FNH3+Akv^6T-JHJ6tG^O0 py#M;yx6C$-a~qrAXEt`BDzHM9IE*Y{GYj}}7H~NWxHQ!c`~_;@VF>^L From 37300057d2b59fc88c6483d69980e8e08b6cfde4 Mon Sep 17 00:00:00 2001 From: klam-shop <78921373+klam-shop@users.noreply.github.com> Date: Fri, 6 Sep 2024 10:38:20 -0400 Subject: [PATCH 279/322] [FLINK-22748] Allow dynamic target topic selection in SQL Kafka sinks (#109) [FLINK-22748][connector-kafka] Allow dynamic target topic selection in SQL Kafka sinks Allows writing to different Kafka topics based on the topic metadata column value in SQL, and updates the Table API's KafkaDynamicSink to accept a List topics instead of String topic as well as support topic-pattern. The list acts as an allow-list of acceptable values for the topic metadata column. topic-pattern for sinks is a pattern that must match topic metadata column values, or else an error is thrown. If a single topic is provided, it is used by default for the target topic to produce to If a list is provided, only that list of topics can be produced to If a topic pattern is provided, it must match topic metadata column values --- .../content.zh/docs/connectors/table/kafka.md | 8 +- .../docs/connectors/table/upsert-kafka.md | 2 +- docs/content/docs/connectors/table/kafka.md | 8 +- .../docs/connectors/table/upsert-kafka.md | 2 +- ...DynamicKafkaRecordSerializationSchema.java | 70 +++++++- .../kafka/table/KafkaConnectorOptions.java | 6 +- .../table/KafkaConnectorOptionsUtil.java | 60 +++---- .../kafka/table/KafkaDynamicSink.java | 49 ++++-- .../kafka/table/KafkaDynamicTableFactory.java | 17 +- .../table/UpsertKafkaDynamicTableFactory.java | 26 +-- ...micKafkaRecordSerializationSchemaTest.java | 148 ++++++++++++++++ .../table/KafkaDynamicTableFactoryTest.java | 124 +++++++++---- .../kafka/table/KafkaTableITCase.java | 164 ++++++++++++++++++ .../UpsertKafkaDynamicTableFactoryTest.java | 96 +++++++++- 14 files changed, 648 insertions(+), 132 deletions(-) create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchemaTest.java diff --git a/docs/content.zh/docs/connectors/table/kafka.md b/docs/content.zh/docs/connectors/table/kafka.md index 27c7c9b19..286a922ef 100644 --- a/docs/content.zh/docs/connectors/table/kafka.md +++ b/docs/content.zh/docs/connectors/table/kafka.md @@ -81,7 +81,7 @@ CREATE TABLE KafkaTable ( topic STRING NOT NULL Kafka 记录的 Topic 名。 - R + R/W partition @@ -191,17 +191,17 @@ CREATE TABLE KafkaTable (

      topic
      - required for sink + 可选 (无) String - 当表用作 source 时读取数据的 topic 名。亦支持用分号间隔的 topic 列表,如 'topic-1;topic-2'。注意,对 source 表而言,'topic' 和 'topic-pattern' 两个选项只能使用其中一个。当表被用作 sink 时,该配置表示写入的 topic 名。注意 sink 表不支持 topic 列表。 + 当表用作 source 时读取数据的 topic 名,或当表用作 sink 时写入的 topic 名。它还支持通过分号分隔的 topic 列表,如 'topic-1;topic-2' 来作为 source 的 topic 列表。注意,“topic-pattern”和“topic”只能指定其中一个。对于 sink 来说,topic 名是写入数据的 topic。它还支持 sink 的 topic 列表。提供的 topic 列表被视为 `topic` 元数据列的有效值的允许列表。如果提供了列表,对于 sink 表,“topic”元数据列是可写的并且必须指定。
      topic-pattern
      可选 (无) String - 匹配读取 topic 名称的正则表达式。在作业开始运行时,所有匹配该正则表达式的 topic 都将被 Kafka consumer 订阅。注意,对 source 表而言,'topic' 和 'topic-pattern' 两个选项只能使用其中一个。 + 用于读取或写入的 topic 名称模式的正则表达式。所有匹配指定正则表达式的 topic 名称将在作业开始运行时被消费者订阅。对于 sink 来说,`topic` 元数据列是可写的,必须提供并且与 `topic-pattern` 正则表达式匹配。注意,“topic-pattern”和“topic”只能指定其中一个。
      properties.bootstrap.servers
      diff --git a/docs/content.zh/docs/connectors/table/upsert-kafka.md b/docs/content.zh/docs/connectors/table/upsert-kafka.md index df2c13878..3d28ae56b 100644 --- a/docs/content.zh/docs/connectors/table/upsert-kafka.md +++ b/docs/content.zh/docs/connectors/table/upsert-kafka.md @@ -119,7 +119,7 @@ of all available metadata fields. 必选 (none) String - 用于读取和写入的 Kafka topic 名称。 + 当表用作 source 时读取数据的 topic 名,或当表用作 sink 时写入的 topic 名。它还支持通过分号分隔的 topic 列表,如 'topic-1;topic-2' 来作为 source 的 topic 列表。注意,“topic-pattern”和“topic”只能指定其中一个。对于 sink 来说,topic 名是写入数据的 topic。它还支持 sink 的 topic 列表。提供的 topic 列表被视为 `topic` 元数据列的有效值的允许列表。如果提供了列表,对于 sink 表,“topic”元数据列是可写的并且必须指定。
      properties.bootstrap.servers
      diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md index 55a5dbf27..5756315bc 100644 --- a/docs/content/docs/connectors/table/kafka.md +++ b/docs/content/docs/connectors/table/kafka.md @@ -83,7 +83,7 @@ Read-only columns must be declared `VIRTUAL` to exclude them during an `INSERT I topic STRING NOT NULL Topic name of the Kafka record. - R + R/W partition @@ -196,11 +196,11 @@ Connector Options
      topic
      - required for sink + optional yes (none) String - Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of "topic-pattern" and "topic" can be specified for sources. When the table is used as sink, the topic name is the topic to write data to. Note topic list is not supported for sinks. + Topic name(s) to read data from when the table is used as source, or topics for writing when the table is used as sink. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of "topic-pattern" and "topic" can be specified. For sinks, the topic name is the topic to write data. It also supports topic list for sinks. The provided topic-list is treated as a allow list of valid values for the `topic` metadata column. If a list is provided, for sink table, 'topic' metadata column is writable and must be specified.
      topic-pattern
      @@ -208,7 +208,7 @@ Connector Options yes (none) String - The regular expression for a pattern of topic names to read from. All topics with names that match the specified regular expression will be subscribed by the consumer when the job starts running. Note, only one of "topic-pattern" and "topic" can be specified for sources. + The regular expression for a pattern of topic names to read from or write to. All topics with names that match the specified regular expression will be subscribed by the consumer when the job starts running. For sinks, the `topic` metadata column is writable, must be provided and match the `topic-pattern` regex. Note, only one of "topic-pattern" and "topic" can be specified.
      properties.bootstrap.servers
      diff --git a/docs/content/docs/connectors/table/upsert-kafka.md b/docs/content/docs/connectors/table/upsert-kafka.md index eb662349c..e8e38aeda 100644 --- a/docs/content/docs/connectors/table/upsert-kafka.md +++ b/docs/content/docs/connectors/table/upsert-kafka.md @@ -129,7 +129,7 @@ Connector Options required (none) String - The Kafka topic name to read from and write to. + Topic name(s) to read data from when the table is used as source, or topics for writing when the table is used as sink. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of "topic-pattern" and "topic" can be specified. For sinks, the topic name is the topic to write data. It also supports topic list for sinks. The provided topic-list is treated as a allow list of valid values for the `topic` metadata column. If a list is provided, for sink table, 'topic' metadata column is writable and must be specified.
      properties.bootstrap.servers
      diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java index 7908aded6..71ca41474 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java @@ -30,12 +30,20 @@ import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + import static org.apache.flink.util.Preconditions.checkNotNull; /** SerializationSchema used by {@link KafkaDynamicSink} to configure a {@link KafkaSink}. */ class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationSchema { - private final String topic; + private final Set topics; + private final Pattern topicPattern; private final FlinkKafkaPartitioner partitioner; @Nullable private final SerializationSchema keySerialization; private final SerializationSchema valueSerialization; @@ -44,9 +52,11 @@ class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationS private final boolean hasMetadata; private final int[] metadataPositions; private final boolean upsertMode; + private final Map topicPatternMatches; DynamicKafkaRecordSerializationSchema( - String topic, + @Nullable List topics, + @Nullable Pattern topicPattern, @Nullable FlinkKafkaPartitioner partitioner, @Nullable SerializationSchema keySerialization, SerializationSchema valueSerialization, @@ -60,7 +70,16 @@ class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationS keySerialization != null && keyFieldGetters.length > 0, "Key must be set in upsert mode for serialization schema."); } - this.topic = checkNotNull(topic); + Preconditions.checkArgument( + (topics != null && topicPattern == null && topics.size() > 0) + || (topics == null && topicPattern != null), + "Either Topic or Topic Pattern must be set."); + if (topics != null) { + this.topics = new HashSet<>(topics); + } else { + this.topics = null; + } + this.topicPattern = topicPattern; this.partitioner = partitioner; this.keySerialization = keySerialization; this.valueSerialization = checkNotNull(valueSerialization); @@ -69,6 +88,8 @@ class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationS this.hasMetadata = hasMetadata; this.metadataPositions = metadataPositions; this.upsertMode = upsertMode; + // Cache results of topic pattern matches to avoid re-evaluating the pattern for each record + this.topicPatternMatches = new HashMap<>(); } @Override @@ -77,13 +98,15 @@ public ProducerRecord serialize( // shortcut in case no input projection is required if (keySerialization == null && !hasMetadata) { final byte[] valueSerialized = valueSerialization.serialize(consumedRow); + final String targetTopic = getTargetTopic(consumedRow); return new ProducerRecord<>( - topic, + targetTopic, extractPartition( consumedRow, + targetTopic, null, valueSerialized, - context.getPartitionsForTopic(topic)), + context.getPartitionsForTopic(targetTopic)), null, valueSerialized); } @@ -115,14 +138,15 @@ public ProducerRecord serialize( consumedRow, kind, valueFieldGetters); valueSerialized = valueSerialization.serialize(valueRow); } - + final String targetTopic = getTargetTopic(consumedRow); return new ProducerRecord<>( - topic, + targetTopic, extractPartition( consumedRow, + targetTopic, keySerialized, valueSerialized, - context.getPartitionsForTopic(topic)), + context.getPartitionsForTopic(targetTopic)), readMetadata(consumedRow, KafkaDynamicSink.WritableMetadata.TIMESTAMP), keySerialized, valueSerialized, @@ -144,14 +168,42 @@ public void open( valueSerialization.open(context); } + private String getTargetTopic(RowData element) { + if (topics != null && topics.size() == 1) { + // If topics is a singleton list, we only return the provided topic. + return topics.stream().findFirst().get(); + } + final String targetTopic = readMetadata(element, KafkaDynamicSink.WritableMetadata.TOPIC); + if (targetTopic == null) { + throw new IllegalArgumentException( + "The topic of the sink record is not valid. Expected a single topic but no topic is set."); + } else if (topics != null && !topics.contains(targetTopic)) { + throw new IllegalArgumentException( + String.format( + "The topic of the sink record is not valid. Expected topic to be in: %s but was: %s", + topics, targetTopic)); + } else if (topicPattern != null && !cachedTopicPatternMatch(targetTopic)) { + throw new IllegalArgumentException( + String.format( + "The topic of the sink record is not valid. Expected topic to match: %s but was: %s", + topicPattern, targetTopic)); + } + return targetTopic; + } + + private boolean cachedTopicPatternMatch(String topic) { + return topicPatternMatches.computeIfAbsent(topic, t -> topicPattern.matcher(t).matches()); + } + private Integer extractPartition( RowData consumedRow, + String targetTopic, @Nullable byte[] keySerialized, byte[] valueSerialized, int[] partitions) { if (partitioner != null) { return partitioner.partition( - consumedRow, keySerialized, valueSerialized, topic, partitions); + consumedRow, keySerialized, valueSerialized, targetTopic, partitions); } return null; } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java index 81ff13c3c..11d3c659f 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java @@ -118,15 +118,15 @@ public class KafkaConnectorOptions { .asList() .noDefaultValue() .withDescription( - "Topic names from which the table is read. Either 'topic' or 'topic-pattern' must be set for source. " - + "Option 'topic' is required for sink."); + "Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of 'topic-pattern' and 'topic' can be specified for sources. " + + "When the table is used as sink, the topic name is the topic to write data. It also supports topic list for sinks. The provided topic-list is treated as a allow list of valid values for the `topic` metadata column. If a list is provided, for sink table, 'topic' metadata column is writable and must be specified."); public static final ConfigOption TOPIC_PATTERN = ConfigOptions.key("topic-pattern") .stringType() .noDefaultValue() .withDescription( - "Optional topic pattern from which the table is read for source. Either 'topic' or 'topic-pattern' must be set."); + "Optional topic pattern from which the table is read for source, or topic pattern that must match the provided `topic` metadata column for sink. Either 'topic' or 'topic-pattern' must be set."); public static final ConfigOption PROPS_BOOTSTRAP_SERVERS = ConfigOptions.key("properties.bootstrap.servers") diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java index d6390e27a..f752276a3 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java @@ -98,23 +98,22 @@ class KafkaConnectorOptionsUtil { protected static final String DEBEZIUM_AVRO_CONFLUENT = "debezium-avro-confluent"; private static final List SCHEMA_REGISTRY_FORMATS = Arrays.asList(AVRO_CONFLUENT, DEBEZIUM_AVRO_CONFLUENT); - // -------------------------------------------------------------------------------------------- // Validation // -------------------------------------------------------------------------------------------- public static void validateTableSourceOptions(ReadableConfig tableOptions) { - validateSourceTopic(tableOptions); + validateTopic(tableOptions); validateScanStartupMode(tableOptions); validateScanBoundedMode(tableOptions); } public static void validateTableSinkOptions(ReadableConfig tableOptions) { - validateSinkTopic(tableOptions); + validateTopic(tableOptions); validateSinkPartitioner(tableOptions); } - public static void validateSourceTopic(ReadableConfig tableOptions) { + public static void validateTopic(ReadableConfig tableOptions) { Optional> topic = tableOptions.getOptional(TOPIC); Optional pattern = tableOptions.getOptional(TOPIC_PATTERN); @@ -128,23 +127,6 @@ public static void validateSourceTopic(ReadableConfig tableOptions) { } } - public static void validateSinkTopic(ReadableConfig tableOptions) { - String errorMessageTemp = - "Flink Kafka sink currently only supports single topic, but got %s: %s."; - if (!isSingleTopic(tableOptions)) { - if (tableOptions.getOptional(TOPIC_PATTERN).isPresent()) { - throw new ValidationException( - String.format( - errorMessageTemp, - "'topic-pattern'", - tableOptions.get(TOPIC_PATTERN))); - } else { - throw new ValidationException( - String.format(errorMessageTemp, "'topic'", tableOptions.get(TOPIC))); - } - } - } - private static void validateScanStartupMode(ReadableConfig tableOptions) { tableOptions .getOptional(SCAN_STARTUP_MODE) @@ -254,11 +236,11 @@ private static void validateSinkPartitioner(ReadableConfig tableOptions) { // Utilities // -------------------------------------------------------------------------------------------- - public static List getSourceTopics(ReadableConfig tableOptions) { + public static List getTopics(ReadableConfig tableOptions) { return tableOptions.getOptional(TOPIC).orElse(null); } - public static Pattern getSourceTopicPattern(ReadableConfig tableOptions) { + public static Pattern getTopicPattern(ReadableConfig tableOptions) { return tableOptions.getOptional(TOPIC_PATTERN).map(Pattern::compile).orElse(null); } @@ -636,21 +618,25 @@ public static DynamicTableFactory.Context autoCompleteSchemaRegistrySubject( private static Map autoCompleteSchemaRegistrySubject( Map options) { Configuration configuration = Configuration.fromMap(options); - // the subject autoComplete should only be used in sink, check the topic first - validateSinkTopic(configuration); - final Optional valueFormat = configuration.getOptional(VALUE_FORMAT); - final Optional keyFormat = configuration.getOptional(KEY_FORMAT); - final Optional format = configuration.getOptional(FORMAT); - final String topic = configuration.get(TOPIC).get(0); - - if (format.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(format.get())) { - autoCompleteSubject(configuration, format.get(), topic + "-value"); - } else if (valueFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(valueFormat.get())) { - autoCompleteSubject(configuration, "value." + valueFormat.get(), topic + "-value"); - } + // the subject autoComplete should only be used in sink with a single topic, check the topic + // option first + validateTopic(configuration); + if (configuration.contains(TOPIC) && isSingleTopic(configuration)) { + final Optional valueFormat = configuration.getOptional(VALUE_FORMAT); + final Optional keyFormat = configuration.getOptional(KEY_FORMAT); + final Optional format = configuration.getOptional(FORMAT); + final String topic = configuration.get(TOPIC).get(0); + + if (format.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(format.get())) { + autoCompleteSubject(configuration, format.get(), topic + "-value"); + } else if (valueFormat.isPresent() + && SCHEMA_REGISTRY_FORMATS.contains(valueFormat.get())) { + autoCompleteSubject(configuration, "value." + valueFormat.get(), topic + "-value"); + } - if (keyFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(keyFormat.get())) { - autoCompleteSubject(configuration, "key." + keyFormat.get(), topic + "-key"); + if (keyFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(keyFormat.get())) { + autoCompleteSubject(configuration, "key." + keyFormat.get(), topic + "-key"); + } } return configuration.toMap(); } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java index 3f6bc5a27..8ab0f10c6 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java @@ -59,6 +59,7 @@ import java.util.Map; import java.util.Objects; import java.util.Properties; +import java.util.regex.Pattern; import java.util.stream.Stream; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -114,8 +115,11 @@ public class KafkaDynamicSink implements DynamicTableSink, SupportsWritingMetada */ @Nullable private final String transactionalIdPrefix; - /** The Kafka topic to write to. */ - protected final String topic; + /** The Kafka topics to allow for producing. */ + protected final List topics; + + /** The Kafka topic pattern of topics allowed to produce to. */ + protected final Pattern topicPattern; /** Properties for the Kafka producer. */ protected final Properties properties; @@ -143,7 +147,8 @@ public KafkaDynamicSink( int[] keyProjection, int[] valueProjection, @Nullable String keyPrefix, - String topic, + @Nullable List topics, + @Nullable Pattern topicPattern, Properties properties, @Nullable FlinkKafkaPartitioner partitioner, DeliveryGuarantee deliveryGuarantee, @@ -166,7 +171,8 @@ public KafkaDynamicSink( // Mutable attributes this.metadataKeys = Collections.emptyList(); // Kafka-specific attributes - this.topic = checkNotNull(topic, "Topic must not be null."); + this.topics = topics; + this.topicPattern = topicPattern; this.properties = checkNotNull(properties, "Properties must not be null."); this.partitioner = partitioner; this.deliveryGuarantee = @@ -206,7 +212,8 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { .setKafkaProducerConfig(properties) .setRecordSerializer( new DynamicKafkaRecordSerializationSchema( - topic, + topics, + topicPattern, partitioner, keySerialization, valueSerialization, @@ -250,8 +257,13 @@ public DataStreamSink consumeDataStream( @Override public Map listWritableMetadata() { final Map metadataMap = new LinkedHashMap<>(); - Stream.of(WritableMetadata.values()) - .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + for (WritableMetadata m : WritableMetadata.values()) { + if (topics != null && topics.size() == 1 && WritableMetadata.TOPIC.key.equals(m.key)) { + // When `topic` is a singleton list, TOPIC metadata is not writable + continue; + } + metadataMap.put(m.key, m.dataType); + } return metadataMap; } @@ -272,7 +284,8 @@ public DynamicTableSink copy() { keyProjection, valueProjection, keyPrefix, - topic, + topics, + topicPattern, properties, partitioner, deliveryGuarantee, @@ -306,7 +319,8 @@ public boolean equals(Object o) { && Arrays.equals(keyProjection, that.keyProjection) && Arrays.equals(valueProjection, that.valueProjection) && Objects.equals(keyPrefix, that.keyPrefix) - && Objects.equals(topic, that.topic) + && Objects.equals(topics, that.topics) + && Objects.equals(String.valueOf(topicPattern), String.valueOf(that.topicPattern)) && Objects.equals(properties, that.properties) && Objects.equals(partitioner, that.partitioner) && Objects.equals(deliveryGuarantee, that.deliveryGuarantee) @@ -327,7 +341,8 @@ public int hashCode() { keyProjection, valueProjection, keyPrefix, - topic, + topics, + topicPattern, properties, partitioner, deliveryGuarantee, @@ -393,6 +408,20 @@ private RowData.FieldGetter[] getFieldGetters( // -------------------------------------------------------------------------------------------- enum WritableMetadata { + TOPIC( + "topic", + DataTypes.STRING().notNull(), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object read(RowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + return row.getString(pos).toString(); + } + }), HEADERS( "headers", // key and value of the map are nullable to make handling easier in queries diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java index 89dda61a1..7c23923b5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java @@ -93,9 +93,9 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getBoundedOptions; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getFlinkKafkaPartitioner; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopicPattern; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopics; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getStartupOptions; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getTopicPattern; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getTopics; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.validateTableSinkOptions; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.validateTableSourceOptions; @@ -222,8 +222,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { keyProjection, valueProjection, keyPrefix, - getSourceTopics(tableOptions), - getSourceTopicPattern(tableOptions), + getTopics(tableOptions), + getTopicPattern(tableOptions), properties, startupOptions.startupMode, startupOptions.specificOffsets, @@ -278,7 +278,8 @@ public DynamicTableSink createDynamicTableSink(Context context) { keyProjection, valueProjection, keyPrefix, - tableOptions.get(TOPIC).get(0), + getTopics(tableOptions), + getTopicPattern(tableOptions), getKafkaProperties(context.getCatalogTable().getOptions()), getFlinkKafkaPartitioner(tableOptions, context.getClassLoader()).orElse(null), deliveryGuarantee, @@ -423,7 +424,8 @@ protected KafkaDynamicSink createKafkaTableSink( int[] keyProjection, int[] valueProjection, @Nullable String keyPrefix, - String topic, + @Nullable List topics, + @Nullable Pattern topicPattern, Properties properties, FlinkKafkaPartitioner partitioner, DeliveryGuarantee deliveryGuarantee, @@ -437,7 +439,8 @@ protected KafkaDynamicSink createKafkaTableSink( keyProjection, valueProjection, keyPrefix, - topic, + topics, + topicPattern, properties, partitioner, deliveryGuarantee, diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java index cebe27f2e..78debc175 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java @@ -66,6 +66,7 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_MAX_ROWS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARALLELISM; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC_PATTERN; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FIELDS_INCLUDE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; @@ -75,8 +76,8 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createValueFormatProjection; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getBoundedOptions; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopicPattern; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopics; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getTopicPattern; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getTopics; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.validateScanBoundedMode; /** Upsert-Kafka factory. */ @@ -94,7 +95,6 @@ public String factoryIdentifier() { public Set> requiredOptions() { final Set> options = new HashSet<>(); options.add(PROPS_BOOTSTRAP_SERVERS); - options.add(TOPIC); options.add(KEY_FORMAT); options.add(VALUE_FORMAT); return options; @@ -103,6 +103,8 @@ public Set> requiredOptions() { @Override public Set> optionalOptions() { final Set> options = new HashSet<>(); + options.add(TOPIC); + options.add(TOPIC_PATTERN); options.add(KEY_FIELDS_PREFIX); options.add(VALUE_FIELDS_INCLUDE); options.add(SINK_PARALLELISM); @@ -155,8 +157,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { keyValueProjections.f0, keyValueProjections.f1, keyPrefix, - getSourceTopics(tableOptions), - getSourceTopicPattern(tableOptions), + getTopics(tableOptions), + getTopicPattern(tableOptions), properties, earliest, Collections.emptyMap(), @@ -212,7 +214,8 @@ public DynamicTableSink createDynamicTableSink(Context context) { keyValueProjections.f0, keyValueProjections.f1, keyPrefix, - tableOptions.get(TOPIC).get(0), + getTopics(tableOptions), + getTopicPattern(tableOptions), properties, null, tableOptions.get(DELIVERY_GUARANTEE), @@ -247,7 +250,6 @@ private static void validateSource( Format keyFormat, Format valueFormat, int[] primaryKeyIndexes) { - validateTopic(tableOptions); validateScanBoundedMode(tableOptions); validateFormat(keyFormat, valueFormat, tableOptions); validatePKConstraints(primaryKeyIndexes); @@ -258,21 +260,11 @@ private static void validateSink( Format keyFormat, Format valueFormat, int[] primaryKeyIndexes) { - validateTopic(tableOptions); validateFormat(keyFormat, valueFormat, tableOptions); validatePKConstraints(primaryKeyIndexes); validateSinkBufferFlush(tableOptions); } - private static void validateTopic(ReadableConfig tableOptions) { - List topic = tableOptions.get(TOPIC); - if (topic.size() > 1) { - throw new ValidationException( - "The 'upsert-kafka' connector doesn't support topic list now. " - + "Please use single topic as the value of the parameter 'topic'."); - } - } - private static void validateFormat( Format keyFormat, Format valueFormat, ReadableConfig tableOptions) { if (!keyFormat.getChangelogMode().containsOnly(RowKind.INSERT)) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchemaTest.java new file mode 100644 index 000000000..6371ae5ef --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchemaTest.java @@ -0,0 +1,148 @@ +package org.apache.flink.streaming.connectors.kafka.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.util.TestLogger; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.regex.Pattern; +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link DynamicKafkaRecordSerializationSchema}. */ +public class DynamicKafkaRecordSerializationSchemaTest extends TestLogger { + private static final List MULTIPLE_TOPICS = Arrays.asList("topic1", "topic2"); + private static final String SINGLE_TOPIC = "topic"; + private static final Pattern TOPIC_PATTERN = Pattern.compile("topic*"); + + @ParameterizedTest + @MethodSource("provideTopicMetadataTestParameters") + public void testTopicMetadata( + List topics, Pattern topicPattern, String rowTopic, String expectedTopic) { + GenericRowData rowData = createRowData(rowTopic); + DynamicKafkaRecordSerializationSchema schema = createSchema(topics, topicPattern); + KafkaRecordSerializationSchema.KafkaSinkContext context = createContext(); + + // Call serialize method + ProducerRecord record = schema.serialize(rowData, context, null); + + // Assert the returned ProducerRecord is routed to the correct topic + assertThat(record.topic()).isEqualTo(expectedTopic); + } + + @ParameterizedTest + @MethodSource("provideInvalidTopicMetadataTestParameters") + public void testInvalidTopicMetadata( + List topics, Pattern topicPattern, String rowTopic, String expectedError) { + GenericRowData rowData = createRowData(rowTopic); + DynamicKafkaRecordSerializationSchema schema = createSchema(topics, topicPattern); + KafkaRecordSerializationSchema.KafkaSinkContext context = createContext(); + + // Call serialize method + assertThatThrownBy(() -> schema.serialize(rowData, context, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining(expectedError); + } + + private static Stream provideTopicMetadataTestParameters() { + String topic1 = "topic1"; + return Stream.of( + Arguments.of( + Collections.singletonList(SINGLE_TOPIC), null, SINGLE_TOPIC, SINGLE_TOPIC), + Arguments.of(Collections.singletonList(SINGLE_TOPIC), null, topic1, SINGLE_TOPIC), + Arguments.of(Collections.singletonList(SINGLE_TOPIC), null, null, SINGLE_TOPIC), + Arguments.of(MULTIPLE_TOPICS, null, topic1, topic1), + Arguments.of(null, TOPIC_PATTERN, SINGLE_TOPIC, SINGLE_TOPIC)); + } + + private static Stream provideInvalidTopicMetadataTestParameters() { + String other = "other"; + return Stream.of( + Arguments.of( + MULTIPLE_TOPICS, + null, + other, + String.format( + "The topic of the sink record is not valid. Expected topic to be in: %s but was: %s", + MULTIPLE_TOPICS, other)), + Arguments.of( + null, + TOPIC_PATTERN, + other, + String.format( + "The topic of the sink record is not valid. Expected topic to match: %s but was: %s", + "topic*", other))); + } + + private DynamicKafkaRecordSerializationSchema createSchema( + List topics, Pattern topicPattern) { + // Create a SerializationSchema for RowData + SerializationSchema serializationSchema = + new SerializationSchema() { + @Override + public byte[] serialize(RowData element) { + return ((StringData) element.getString(0)).toBytes(); + } + + @Override + public void open(InitializationContext context) throws Exception {} + }; + + int[] metadataPositions = new int[3]; + metadataPositions[KafkaDynamicSink.WritableMetadata.TOPIC.ordinal()] = 1; + metadataPositions[KafkaDynamicSink.WritableMetadata.HEADERS.ordinal()] = 2; + metadataPositions[KafkaDynamicSink.WritableMetadata.TIMESTAMP.ordinal()] = 3; + + return new DynamicKafkaRecordSerializationSchema( + topics, + topicPattern, + null, + null, + serializationSchema, + new RowData.FieldGetter[] {r -> r.getString(0)}, + new RowData.FieldGetter[] {r -> r.getString(0)}, + true, + metadataPositions, + false); + } + + private GenericRowData createRowData(String topic) { + GenericRowData rowData = new GenericRowData(4); + rowData.setField(0, StringData.fromString("test")); + rowData.setField(1, StringData.fromString(topic)); + rowData.setField(2, null); + rowData.setField(3, null); + return rowData; + } + + private KafkaRecordSerializationSchema.KafkaSinkContext createContext() { + return new KafkaRecordSerializationSchema.KafkaSinkContext() { + @Override + public int getParallelInstanceId() { + return 0; + } + + @Override + public int getNumberOfParallelInstances() { + return 1; + } + + @Override + public int[] getPartitionsForTopic(String topic) { + return new int[] {0}; + } + }; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java index 1246d53a3..c1d796d08 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java @@ -600,7 +600,8 @@ public void testTableSink() { new int[0], new int[] {0, 1, 2}, null, - TOPIC, + Collections.singletonList(TOPIC), + null, KAFKA_SINK_PROPERTIES, new FlinkFixedPartitioner<>(), DeliveryGuarantee.EXACTLY_ONCE, @@ -616,6 +617,10 @@ public void testTableSink() { final SinkV2Provider sinkProvider = (SinkV2Provider) provider; final Sink sinkFunction = sinkProvider.createSink(); assertThat(sinkFunction).isInstanceOf(KafkaSink.class); + assertThat(actualKafkaSink.listWritableMetadata()) + .containsOnlyKeys( + KafkaDynamicSink.WritableMetadata.HEADERS.key, + KafkaDynamicSink.WritableMetadata.TIMESTAMP.key); } @Test @@ -640,7 +645,8 @@ public void testTableSinkSemanticTranslation() { new int[0], new int[] {0, 1, 2}, null, - TOPIC, + Collections.singletonList(TOPIC), + null, KAFKA_SINK_PROPERTIES, new FlinkFixedPartitioner<>(), DeliveryGuarantee.valueOf(semantic.toUpperCase().replace("-", "_")), @@ -683,7 +689,8 @@ public void testTableSinkWithKeyValue() { new int[] {0}, new int[] {1, 2}, null, - TOPIC, + Collections.singletonList(TOPIC), + null, KAFKA_FINAL_SINK_PROPERTIES, new FlinkFixedPartitioner<>(), DeliveryGuarantee.EXACTLY_ONCE, @@ -711,7 +718,8 @@ public void testTableSinkWithParallelism() { new int[0], new int[] {0, 1, 2}, null, - TOPIC, + Collections.singletonList(TOPIC), + null, KAFKA_SINK_PROPERTIES, new FlinkFixedPartitioner<>(), DeliveryGuarantee.EXACTLY_ONCE, @@ -805,6 +813,77 @@ public void testTableSinkAutoCompleteSchemaRegistrySubject() { "sub2"); } + @Test + public void testTableSinkWithTopicList() { + final Map modifiedOptions = + getModifiedOptions(getBasicSinkOptions(), options -> options.put("topic", TOPICS)); + KafkaDynamicSink actualSink = (KafkaDynamicSink) createTableSink(SCHEMA, modifiedOptions); + + final EncodingFormat> valueEncodingFormat = + new EncodingFormatMock(","); + + final DynamicTableSink expectedSink = + createExpectedSink( + SCHEMA_DATA_TYPE, + null, + valueEncodingFormat, + new int[0], + new int[] {0, 1, 2}, + null, + Arrays.asList(TOPICS.split(";")), + null, + KAFKA_SINK_PROPERTIES, + new FlinkFixedPartitioner<>(), + DeliveryGuarantee.EXACTLY_ONCE, + null, + "kafka-sink"); + assertThat(actualSink).isEqualTo(expectedSink); + final KafkaDynamicSink actualKafkaSink = (KafkaDynamicSink) actualSink; + assertThat(actualKafkaSink.listWritableMetadata()) + .containsOnlyKeys( + KafkaDynamicSink.WritableMetadata.TOPIC.key, + KafkaDynamicSink.WritableMetadata.HEADERS.key, + KafkaDynamicSink.WritableMetadata.TIMESTAMP.key); + } + + @Test + public void testTableSinkWithTopicPattern() { + final Map modifiedOptions = + getModifiedOptions( + getBasicSinkOptions(), + options -> { + options.remove("topic"); + options.put("topic-pattern", TOPIC_REGEX); + }); + KafkaDynamicSink actualSink = (KafkaDynamicSink) createTableSink(SCHEMA, modifiedOptions); + + final EncodingFormat> valueEncodingFormat = + new EncodingFormatMock(","); + + final DynamicTableSink expectedSink = + createExpectedSink( + SCHEMA_DATA_TYPE, + null, + valueEncodingFormat, + new int[0], + new int[] {0, 1, 2}, + null, + null, + Pattern.compile(TOPIC_REGEX), + KAFKA_SINK_PROPERTIES, + new FlinkFixedPartitioner<>(), + DeliveryGuarantee.EXACTLY_ONCE, + null, + "kafka-sink"); + assertThat(actualSink).isEqualTo(expectedSink); + final KafkaDynamicSink actualKafkaSink = (KafkaDynamicSink) actualSink; + assertThat(actualKafkaSink.listWritableMetadata()) + .containsOnlyKeys( + KafkaDynamicSink.WritableMetadata.TOPIC.key, + KafkaDynamicSink.WritableMetadata.HEADERS.key, + KafkaDynamicSink.WritableMetadata.TIMESTAMP.key); + } + private void verifyEncoderSubject( Consumer> optionModifier, String expectedValueSubject, @@ -1002,7 +1081,7 @@ public void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { } @Test - public void testSinkWithTopicListOrTopicPattern() { + public void testSinkWithTopicListAndTopicPattern() { Map modifiedOptions = getModifiedOptions( getBasicSinkOptions(), @@ -1010,32 +1089,13 @@ public void testSinkWithTopicListOrTopicPattern() { options.put("topic", TOPICS); options.put("scan.startup.mode", "earliest-offset"); options.remove("specific-offsets"); + options.put("topic-pattern", TOPIC_REGEX); }); final String errorMessageTemp = - "Flink Kafka sink currently only supports single topic, but got %s: %s."; - - try { - createTableSink(SCHEMA, modifiedOptions); - } catch (Throwable t) { - assertThat(t.getCause().getMessage()) - .isEqualTo( - String.format( - errorMessageTemp, - "'topic'", - String.format("[%s]", String.join(", ", TOPIC_LIST)))); - } - - modifiedOptions = - getModifiedOptions( - getBasicSinkOptions(), - options -> options.put("topic-pattern", TOPIC_REGEX)); - - try { - createTableSink(SCHEMA, modifiedOptions); - } catch (Throwable t) { - assertThat(t.getCause().getMessage()) - .isEqualTo(String.format(errorMessageTemp, "'topic-pattern'", TOPIC_REGEX)); - } + "Option 'topic' and 'topic-pattern' shouldn't be set together."; + assertThatThrownBy(() -> createTableSink(SCHEMA, modifiedOptions)) + .isInstanceOf(ValidationException.class) + .satisfies(anyCauseMatches(ValidationException.class, errorMessageTemp)); } @Test @@ -1217,7 +1277,8 @@ private static KafkaDynamicSink createExpectedSink( int[] keyProjection, int[] valueProjection, @Nullable String keyPrefix, - String topic, + @Nullable List topics, + @Nullable Pattern topicPattern, Properties properties, @Nullable FlinkKafkaPartitioner partitioner, DeliveryGuarantee deliveryGuarantee, @@ -1231,7 +1292,8 @@ private static KafkaDynamicSink createExpectedSink( keyProjection, valueProjection, keyPrefix, - topic, + topics, + topicPattern, properties, partitioner, deliveryGuarantee, diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java index 409acd977..15aa722f1 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java @@ -188,6 +188,170 @@ public void testKafkaSourceSink() throws Exception { deleteTestTopic(topic); } + @Test + public void testKafkaSourceSinkWithTopicList() throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic1 = "topics1_" + format + "_" + UUID.randomUUID(); + final String topic2 = "topics2_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic2, 1, 1); + createTestTopic(topic1, 1, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + String groupId = getStandardProps().getProperty("group.id"); + String bootstraps = getBootstrapServers(); + final String createTableTemplate = + "CREATE TABLE %s (\n" + + " `topic` STRING METADATA,\n" + + " `user_id` INT,\n" + + " `item_id` INT,\n" + + " `behavior` STRING\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'properties.group.id' = '%s',\n" + + " 'scan.startup.mode' = 'earliest-offset',\n" + + " 'scan.bounded.mode' = 'latest-offset',\n" + + " %s\n" + + ")\n"; + final String createTopicListTable = + String.format( + createTableTemplate, + "kafka", + KafkaDynamicTableFactory.IDENTIFIER, + String.join(";", Arrays.asList(topic1, topic2)), + bootstraps, + groupId, + formatOptions()); + final String createTopic1Table = + String.format( + createTableTemplate, + "topic1", + KafkaDynamicTableFactory.IDENTIFIER, + topic1, + bootstraps, + groupId, + formatOptions()); + final String createTopic2Table = + String.format( + createTableTemplate, + "topic2", + KafkaDynamicTableFactory.IDENTIFIER, + topic2, + bootstraps, + groupId, + formatOptions()); + + tEnv.executeSql(createTopicListTable); + tEnv.executeSql(createTopic1Table); + tEnv.executeSql(createTopic2Table); + + List values = + Arrays.asList( + Row.of(topic1, 1, 1102, "behavior 1"), + Row.of(topic2, 2, 1103, "behavior 2")); + tEnv.fromValues(values).insertInto("kafka").execute().await(); + + // ---------- Consume stream from Kafka ------------------- + List results = collectAllRows(tEnv.sqlQuery("SELECT * from kafka")); + List topic1Results = collectAllRows(tEnv.sqlQuery("SELECT * from topic1")); + List topic2Results = collectAllRows(tEnv.sqlQuery("SELECT * from topic2")); + assertThat(results) + .containsExactlyInAnyOrder( + Row.of(topic1, 1, 1102, "behavior 1"), + Row.of(topic2, 2, 1103, "behavior 2")); + assertThat(topic1Results).containsExactly(Row.of(topic1, 1, 1102, "behavior 1")); + assertThat(topic2Results).containsExactly(Row.of(topic2, 2, 1103, "behavior 2")); + + // ------------- cleanup ------------------- + deleteTestTopic(topic1); + deleteTestTopic(topic2); + } + + @Test + public void testKafkaSourceSinkWithTopicPattern() throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic1 = "topics1_" + format + "_" + UUID.randomUUID(); + final String topic2 = "topics2_" + format + "_" + UUID.randomUUID(); + final String topicPattern = "topics.*"; + createTestTopic(topic2, 1, 1); + createTestTopic(topic1, 1, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + String groupId = getStandardProps().getProperty("group.id"); + String bootstraps = getBootstrapServers(); + final String createTableTemplate = + "CREATE TABLE %s (\n" + + " `topic` STRING METADATA,\n" + + " `user_id` INT,\n" + + " `item_id` INT,\n" + + " `behavior` STRING\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topic-pattern' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'properties.group.id' = '%s',\n" + + " 'scan.startup.mode' = 'earliest-offset',\n" + + " 'scan.bounded.mode' = 'latest-offset',\n" + + " %s\n" + + ")\n"; + final String createTopicPatternTable = + String.format( + createTableTemplate, + "kafka", + KafkaDynamicTableFactory.IDENTIFIER, + topicPattern, + bootstraps, + groupId, + formatOptions()); + final String createTopic1Table = + String.format( + createTableTemplate, + "topic1", + KafkaDynamicTableFactory.IDENTIFIER, + topic1, + bootstraps, + groupId, + formatOptions()); + final String createTopic2Table = + String.format( + createTableTemplate, + "topic2", + KafkaDynamicTableFactory.IDENTIFIER, + topic2, + bootstraps, + groupId, + formatOptions()); + + tEnv.executeSql(createTopicPatternTable); + tEnv.executeSql(createTopic1Table); + tEnv.executeSql(createTopic2Table); + + List values = + Arrays.asList( + Row.of(topic1, 1, 1102, "behavior 1"), + Row.of(topic2, 2, 1103, "behavior 2")); + tEnv.fromValues(values).insertInto("kafka").execute().await(); + + // ---------- Consume stream from Kafka ------------------- + List results = collectAllRows(tEnv.sqlQuery("SELECT * from kafka")); + List topic1Results = collectAllRows(tEnv.sqlQuery("SELECT * from topic1")); + List topic2Results = collectAllRows(tEnv.sqlQuery("SELECT * from topic2")); + assertThat(results) + .containsExactlyInAnyOrder( + Row.of(topic1, 1, 1102, "behavior 1"), + Row.of(topic2, 2, 1103, "behavior 2")); + assertThat(topic1Results).containsExactly(Row.of(topic1, 1, 1102, "behavior 1")); + assertThat(topic2Results).containsExactly(Row.of(topic2, 2, 1103, "behavior 2")); + + // ------------- cleanup ------------------- + + deleteTestTopic(topic1); + deleteTestTopic(topic2); + } + @Test public void testKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { // we always use a different topic name for each parameterized topic, diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java index 15c740d21..1bcd775a1 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java @@ -77,10 +77,12 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.function.Consumer; +import java.util.regex.Pattern; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode; @@ -165,7 +167,38 @@ public void testTableSource() { SOURCE_KEY_FIELDS, SOURCE_VALUE_FIELDS, null, - SOURCE_TOPIC, + Collections.singletonList(SOURCE_TOPIC), + UPSERT_KAFKA_SOURCE_PROPERTIES); + assertThat(actualSource).isEqualTo(expectedSource); + + final KafkaDynamicSource actualUpsertKafkaSource = (KafkaDynamicSource) actualSource; + ScanTableSource.ScanRuntimeProvider provider = + actualUpsertKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertKafkaSource(provider); + } + + @Test + public void testTableSourceWithTopicList() { + final Map modifiedOptions = + getModifiedOptions( + getFullSourceOptions(), + options -> { + options.put( + "topic", String.format("%s;%s", SOURCE_TOPIC, SOURCE_TOPIC)); + }); + final DataType producedDataType = SOURCE_SCHEMA.toPhysicalRowDataType(); + // Construct table source using options and table source factory + final DynamicTableSource actualSource = createTableSource(SOURCE_SCHEMA, modifiedOptions); + + final KafkaDynamicSource expectedSource = + createExpectedScanSource( + producedDataType, + keyDecodingFormat, + valueDecodingFormat, + SOURCE_KEY_FIELDS, + SOURCE_VALUE_FIELDS, + null, + Arrays.asList(SOURCE_TOPIC, SOURCE_TOPIC), UPSERT_KAFKA_SOURCE_PROPERTIES); assertThat(actualSource).isEqualTo(expectedSource); @@ -195,7 +228,50 @@ public void testTableSink() { SINK_KEY_FIELDS, SINK_VALUE_FIELDS, null, - SINK_TOPIC, + Collections.singletonList(SINK_TOPIC), + null, + UPSERT_KAFKA_SINK_PROPERTIES, + DeliveryGuarantee.EXACTLY_ONCE, + SinkBufferFlushMode.DISABLED, + null, + "kafka-sink"); + + // Test sink format. + final KafkaDynamicSink actualUpsertKafkaSink = (KafkaDynamicSink) actualSink; + assertThat(actualSink).isEqualTo(expectedSink); + + // Test kafka producer. + DynamicTableSink.SinkRuntimeProvider provider = + actualUpsertKafkaSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false)); + assertThat(provider).isInstanceOf(SinkV2Provider.class); + final SinkV2Provider sinkFunctionProvider = (SinkV2Provider) provider; + final Sink sink = sinkFunctionProvider.createSink(); + assertThat(sink).isInstanceOf(KafkaSink.class); + } + + @Test + public void testTableSinkWithTopicList() { + // Construct table sink using options and table sink factory. + final Map modifiedOptions = + getModifiedOptions( + getFullSinkOptions(), + options -> { + options.put("sink.delivery-guarantee", "exactly-once"); + options.put("sink.transactional-id-prefix", "kafka-sink"); + options.put("topic", String.format("%s;%s", SINK_TOPIC, SINK_TOPIC)); + }); + final DynamicTableSink actualSink = createTableSink(SINK_SCHEMA, modifiedOptions); + + final DynamicTableSink expectedSink = + createExpectedSink( + SINK_SCHEMA.toPhysicalRowDataType(), + keyEncodingFormat, + valueEncodingFormat, + SINK_KEY_FIELDS, + SINK_VALUE_FIELDS, + null, + Arrays.asList(SINK_TOPIC, SINK_TOPIC), + null, UPSERT_KAFKA_SINK_PROPERTIES, DeliveryGuarantee.EXACTLY_ONCE, SinkBufferFlushMode.DISABLED, @@ -239,7 +315,8 @@ public void testBufferedTableSink() { SINK_KEY_FIELDS, SINK_VALUE_FIELDS, null, - SINK_TOPIC, + Collections.singletonList(SINK_TOPIC), + null, UPSERT_KAFKA_SINK_PROPERTIES, DeliveryGuarantee.EXACTLY_ONCE, new SinkBufferFlushMode(100, 1000L), @@ -293,7 +370,8 @@ public void testTableSinkWithParallelism() { SINK_KEY_FIELDS, SINK_VALUE_FIELDS, null, - SINK_TOPIC, + Collections.singletonList(SINK_TOPIC), + null, UPSERT_KAFKA_SINK_PROPERTIES, DeliveryGuarantee.EXACTLY_ONCE, SinkBufferFlushMode.DISABLED, @@ -772,7 +850,7 @@ private KafkaDynamicSource createExpectedScanSource( int[] keyFields, int[] valueFields, String keyPrefix, - String topic, + List topic, Properties properties) { return new KafkaDynamicSource( producedDataType, @@ -781,7 +859,7 @@ private KafkaDynamicSource createExpectedScanSource( keyFields, valueFields, keyPrefix, - Collections.singletonList(topic), + topic, null, properties, StartupMode.EARLIEST, @@ -801,7 +879,8 @@ private static KafkaDynamicSink createExpectedSink( int[] keyProjection, int[] valueProjection, String keyPrefix, - String topic, + List topics, + Pattern topicPattern, Properties properties, DeliveryGuarantee deliveryGuarantee, SinkBufferFlushMode flushMode, @@ -815,7 +894,8 @@ private static KafkaDynamicSink createExpectedSink( keyProjection, valueProjection, keyPrefix, - topic, + topics, + topicPattern, properties, null, deliveryGuarantee, From 7929b16dcfe648da30b6cc9755f63de2ed3d5319 Mon Sep 17 00:00:00 2001 From: xiaochen <598457447@qq.com> Date: Thu, 12 Sep 2024 14:10:47 +0800 Subject: [PATCH 280/322] [FLINK-36210] Optimize the logic for fetching topic metadata in the TopicPatternSubscriber mode (#117) In TopicPatternSubscriber mode, our current logic for fetch topic metadata for all topics and then filtering it. We can optimize this by first filtering the topic names and then fetch metadata only for the filtered topics. Co-authored-by: ClownXC --- .../subscriber/KafkaSubscriberUtils.java | 18 ++++++++++++++++++ .../subscriber/TopicPatternSubscriber.java | 18 ++++++++---------- 2 files changed, 26 insertions(+), 10 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java index 404ffaef8..72e7f64d0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java @@ -23,6 +23,8 @@ import java.util.Map; import java.util.Set; +import java.util.regex.Pattern; +import java.util.stream.Collectors; /** The base implementations of {@link KafkaSubscriber}. */ class KafkaSubscriberUtils { @@ -38,6 +40,22 @@ static Map getAllTopicMetadata(AdminClient adminClient } } + static Map getTopicMetadata( + AdminClient adminClient, Pattern topicPattern) { + try { + Set allTopicNames = adminClient.listTopics().names().get(); + Set matchedTopicNames = + allTopicNames.stream() + .filter(name -> topicPattern.matcher(name).matches()) + .collect(Collectors.toSet()); + return getTopicMetadata(adminClient, matchedTopicNames); + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to get metadata for %s topics.", topicPattern.pattern()), + e); + } + } + static Map getTopicMetadata( AdminClient adminClient, Set topicNames) { try { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java index 2a9a75331..985ca7137 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java @@ -30,7 +30,7 @@ import java.util.Set; import java.util.regex.Pattern; -import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getAllTopicMetadata; +import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getTopicMetadata; /** A subscriber to a topic pattern. */ class TopicPatternSubscriber implements KafkaSubscriber { @@ -44,19 +44,17 @@ class TopicPatternSubscriber implements KafkaSubscriber { @Override public Set getSubscribedTopicPartitions(AdminClient adminClient) { - LOG.debug("Fetching descriptions for all topics on Kafka cluster"); - final Map allTopicMetadata = getAllTopicMetadata(adminClient); + LOG.debug("Fetching descriptions for {} topics on Kafka cluster", topicPattern.pattern()); + final Map matchedTopicMetadata = + getTopicMetadata(adminClient, topicPattern); Set subscribedTopicPartitions = new HashSet<>(); - allTopicMetadata.forEach( + matchedTopicMetadata.forEach( (topicName, topicDescription) -> { - if (topicPattern.matcher(topicName).matches()) { - for (TopicPartitionInfo partition : topicDescription.partitions()) { - subscribedTopicPartitions.add( - new TopicPartition( - topicDescription.name(), partition.partition())); - } + for (TopicPartitionInfo partition : topicDescription.partitions()) { + subscribedTopicPartitions.add( + new TopicPartition(topicDescription.name(), partition.partition())); } }); From 122a74394629701ce1096b6ea49a03a0e0744b2b Mon Sep 17 00:00:00 2001 From: dongwoo kim Date: Tue, 17 Sep 2024 21:34:10 +0800 Subject: [PATCH 281/322] [FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition on split reader (#100) Problem: In batch mode, flink kafka connector could hang when consuming transactional messages or reading from deleted records. Solution: Use consumer.position() instead of lastRecord's offset to skip control and deleted messages, preventing the hang. --- .../reader/KafkaPartitionSplitReader.java | 44 +++++++------- .../kafka/source/KafkaSourceITCase.java | 35 ++++++++++++ .../source/reader/KafkaSourceReaderTest.java | 2 +- .../kafka/testutils/KafkaSourceTestEnv.java | 8 ++- .../connectors/kafka/KafkaTestBase.java | 15 ++++- .../kafka/KafkaTestEnvironment.java | 7 +++ .../kafka/table/KafkaTableITCase.java | 57 +++++++++++++++++++ .../kafka/table/KafkaTableTestBase.java | 19 +++++++ 8 files changed, 162 insertions(+), 25 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java index 94940b8e7..23956f5d5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java @@ -122,32 +122,32 @@ public RecordsWithSplitIds> fetch() throws IOExce KafkaPartitionSplitRecords recordsBySplits = new KafkaPartitionSplitRecords(consumerRecords, kafkaSourceReaderMetrics); List finishedPartitions = new ArrayList<>(); - for (TopicPartition tp : consumerRecords.partitions()) { + for (TopicPartition tp : consumer.assignment()) { long stoppingOffset = getStoppingOffset(tp); - final List> recordsFromPartition = - consumerRecords.records(tp); - - if (recordsFromPartition.size() > 0) { - final ConsumerRecord lastRecord = - recordsFromPartition.get(recordsFromPartition.size() - 1); - - // After processing a record with offset of "stoppingOffset - 1", the split reader - // should not continue fetching because the record with stoppingOffset may not - // exist. Keep polling will just block forever. - if (lastRecord.offset() >= stoppingOffset - 1) { - recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); - finishSplitAtRecord( - tp, - stoppingOffset, - lastRecord.offset(), - finishedPartitions, - recordsBySplits); - } + long consumerPosition = consumer.position(tp); + // Stop fetching when the consumer's position reaches the stoppingOffset. + // Control messages may follow the last record; therefore, using the last record's + // offset as a stopping condition could result in indefinite blocking. + if (consumerPosition >= stoppingOffset) { + LOG.debug( + "Position of {}: {}, has reached stopping offset: {}", + tp, + consumerPosition, + stoppingOffset); + recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); + finishSplitAtRecord( + tp, stoppingOffset, consumerPosition, finishedPartitions, recordsBySplits); } - // Track this partition's record lag if it never appears before - kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, tp); } + // Only track non-empty partition's record lag if it never appears before + consumerRecords + .partitions() + .forEach( + trackTp -> { + kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, trackTp); + }); + markEmptySplitsAsFinished(recordsBySplits); // Unassign the partitions that has finished. diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java index 38ef80d51..6c0bd7e50 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java @@ -80,6 +80,7 @@ import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.PARTITION; import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.TOPIC; +import static org.apache.flink.streaming.connectors.kafka.KafkaTestBase.kafkaServer; import static org.assertj.core.api.Assertions.assertThat; /** Unite test class for {@link KafkaSource}. */ @@ -369,6 +370,40 @@ public void testConsumingTopicWithEmptyPartitions() throws Throwable { WatermarkStrategy.noWatermarks(), "testConsumingTopicWithEmptyPartitions")); } + + @Test + public void testConsumingTransactionalMessage() throws Throwable { + String transactionalTopic = "transactionalTopic-" + UUID.randomUUID(); + KafkaSourceTestEnv.createTestTopic( + transactionalTopic, KafkaSourceTestEnv.NUM_PARTITIONS, 1); + List> records = + KafkaSourceTestEnv.getRecordsForTopic(transactionalTopic); + KafkaSourceTestEnv.produceToKafka( + records, kafkaServer.getTransactionalProducerConfig()); + // After running KafkaSourceTestEnv.setupEarliestOffsets(transactionalTopic): + // - For each partition, records with offsets before partition number P are deleted. + // - Partition 0: offset 0 is earliest + // - Partition 5: offset 5 is earliest, 0-4 are deleted. + // - Partition 9: offset 9 is earliest, 0-8 are deleted. + KafkaSourceTestEnv.setupEarliestOffsets(transactionalTopic); + KafkaSource source = + KafkaSource.builder() + .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings) + .setTopics(transactionalTopic) + .setGroupId("topic-with-transactional-message-test") + .setDeserializer(new TestingKafkaRecordDeserializationSchema(false)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setBounded(OffsetsInitializer.latest()) + .build(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + executeAndVerify( + env, + env.fromSource( + source, + WatermarkStrategy.noWatermarks(), + "testConsumingTransactionalMessage")); + } } /** Integration test based on connector testing framework. */ diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java index f5aa7f5fd..5ad87ffc2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java @@ -116,7 +116,7 @@ public static void setup() throws Throwable { "Waiting for offsets topic creation failed."); } KafkaSourceTestEnv.produceToKafka( - getRecords(), StringSerializer.class, IntegerSerializer.class); + getRecords(), StringSerializer.class, IntegerSerializer.class, null); } @AfterAll diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java index 5173f9dc0..d82425f57 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java @@ -249,7 +249,13 @@ public static void setupCommittedOffsets(String topic) public static void produceToKafka(Collection> records) throws Throwable { - produceToKafka(records, StringSerializer.class, IntegerSerializer.class); + produceToKafka(records, StringSerializer.class, IntegerSerializer.class, null); + } + + public static void produceToKafka( + Collection> records, Properties extraProps) + throws Throwable { + produceToKafka(records, StringSerializer.class, IntegerSerializer.class, extraProps); } public static void setupTopic( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index deafb7d6d..4b9acbf1e 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -46,6 +46,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; @@ -242,11 +244,15 @@ public static void produceToKafka( Collection> records, Class> keySerializerClass, Class> - valueSerializerClass) + valueSerializerClass, + @Nullable Properties extraProps) throws Throwable { Properties props = new Properties(); props.putAll(standardProps); props.putAll(kafkaServer.getIdempotentProducerConfig()); + if (extraProps != null) { + props.putAll(extraProps); + } props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass.getName()); props.setProperty( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName()); @@ -261,9 +267,16 @@ public static void produceToKafka( } }; try (KafkaProducer producer = new KafkaProducer<>(props)) { + if (props.containsKey(ProducerConfig.TRANSACTIONAL_ID_CONFIG)) { + producer.initTransactions(); + producer.beginTransaction(); + } for (ProducerRecord record : records) { producer.send(record, callback); } + if (props.containsKey(ProducerConfig.TRANSACTIONAL_ID_CONFIG)) { + producer.commitTransaction(); + } } if (sendingError.get() != null) { throw sendingError.get(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java index 6687cd525..1494ff1f4 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java @@ -34,6 +34,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.UUID; /** Abstract class providing a Kafka test environment. */ public abstract class KafkaTestEnvironment { @@ -115,6 +116,12 @@ public Properties getIdempotentProducerConfig() { return props; } + public Properties getTransactionalProducerConfig() { + Properties props = new Properties(); + props.put("transactional.id", UUID.randomUUID().toString()); + return props; + } + // -- consumer / producer instances: public FlinkKafkaConsumerBase getConsumer( List topics, DeserializationSchema deserializationSchema, Properties props) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java index 15aa722f1..8630120b2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java @@ -54,6 +54,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Random; @@ -352,6 +353,62 @@ public void testKafkaSourceSinkWithTopicPattern() throws Exception { deleteTestTopic(topic2); } + @Test + public void testKafkaSourceEmptyResultOnDeletedOffsets() throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "bounded_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic, 1, 1); + // ---------- Produce an event time stream into Kafka ------------------- + String groupId = getStandardProps().getProperty("group.id"); + String bootstraps = getBootstrapServers(); + + final String createTable = + String.format( + "CREATE TABLE kafka (\n" + + " `user_id` INT,\n" + + " `item_id` INT,\n" + + " `behavior` STRING\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'properties.group.id' = '%s',\n" + + " 'scan.startup.mode' = 'specific-offsets',\n" + + " 'scan.bounded.mode' = 'specific-offsets',\n" + + " 'scan.startup.specific-offsets' = 'partition:0,offset:1',\n" + + " 'scan.bounded.specific-offsets' = 'partition:0,offset:3',\n" + + " %s\n" + + ")\n", + KafkaDynamicTableFactory.IDENTIFIER, + topic, + bootstraps, + groupId, + formatOptions()); + tEnv.executeSql(createTable); + List values = + Arrays.asList( + Row.of(1, 1102, "behavior 1"), + Row.of(2, 1103, "behavior 2"), + Row.of(3, 1104, "behavior 3")); + tEnv.fromValues(values).insertInto("kafka").execute().await(); + // ---------- Delete events from Kafka ------------------- + Map partitionOffsetsToDelete = new HashMap<>(); + partitionOffsetsToDelete.put(0, 3L); + deleteRecords(topic, partitionOffsetsToDelete); + // ---------- Consume stream from Kafka ------------------- + List results = new ArrayList<>(); + env = StreamExecutionEnvironment.getExecutionEnvironment(); + tEnv = StreamTableEnvironment.create(env); + tEnv.executeSql(createTable); + results.addAll(collectAllRows(tEnv.sqlQuery("SELECT * FROM kafka"))); + assertThat(results).isEmpty(); + + // ------------- cleanup ------------------- + + deleteTestTopic(topic); + } + @Test public void testKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { // we always use a different topic name for each parameterized topic, diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java index cffe2d6c0..e9d8bb76b 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java @@ -28,6 +28,7 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult; import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.RecordsToDelete; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.admin.TopicListing; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -170,6 +171,24 @@ public void deleteTestTopic(String topic) { } } + public void deleteRecords(String topic, Map partitionOffsetsToDelete) { + Map properties = new HashMap<>(); + properties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, getBootstrapServers()); + + try (AdminClient admin = AdminClient.create(properties)) { + Map recordsToDelete = new HashMap<>(); + for (Map.Entry entry : partitionOffsetsToDelete.entrySet()) { + TopicPartition partition = new TopicPartition(topic, entry.getKey()); + RecordsToDelete records = RecordsToDelete.beforeOffset(entry.getValue()); + recordsToDelete.put(partition, records); + } + admin.deleteRecords(recordsToDelete).all().get(); + } catch (Exception e) { + throw new IllegalStateException( + String.format("Fail to delete records on topic [%s].", topic), e); + } + } + // ------------------------ For Debug Logging Purpose ---------------------------------- private void scheduleTimeoutLogger(Duration period, Runnable loggingAction) { From 52e7e580cd3b453a92a87afff4daf19ec53212dd Mon Sep 17 00:00:00 2001 From: xiaochen <598457447@qq.com> Date: Thu, 19 Sep 2024 14:24:19 +0800 Subject: [PATCH 282/322] [FLINK-36209] Remove redundant operations in the initialization of KafkaSourceEnumState (#116) In certain methods, such as the DynamicKafkaSourceEnumerator#onHandleSubscribedStreamsFetch() method, partitions are divided into assignedPartitions and unassignedInitialPartitions before being passed as parameters to the KafkaSourceEnumState constructor. However, within the constructor, these assignedPartitions and unassignedInitialPartitions are recombined into partitions, leading to unnecessary operations and reduced performance. By optimizing the code to pass partitions directly as a parameter when initializing KafkaSourceEnumState, we can eliminate redundant operations and enhance performance. --------- Co-authored-by: ClownXC --- .../DynamicKafkaSourceEnumerator.java | 19 ++++++------------- .../enumerator/KafkaSourceEnumState.java | 3 ++- .../KafkaSourceEnumStateSerializer.java | 17 ++++++++--------- .../enumerator/KafkaSourceEnumerator.java | 2 +- 4 files changed, 17 insertions(+), 24 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java index 20e8b9238..b61cee403 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java @@ -35,6 +35,7 @@ import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator; +import org.apache.flink.connector.kafka.source.enumerator.TopicPartitionAndAssignmentStatus; import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; @@ -298,24 +299,16 @@ private void onHandleSubscribedStreamsFetch(Set fetchedKafkaStreams final Set activeTopics = activeClusterTopics.getValue(); // filter out removed topics - Set activeAssignedPartitions = - kafkaSourceEnumState.assignedPartitions().stream() - .filter(tp -> activeTopics.contains(tp.topic())) - .collect(Collectors.toSet()); - Set activeUnassignedInitialPartitions = - kafkaSourceEnumState.unassignedInitialPartitions().stream() - .filter(tp -> activeTopics.contains(tp.topic())) + Set partitions = + kafkaSourceEnumState.partitions().stream() + .filter(tp -> activeTopics.contains(tp.topicPartition().topic())) .collect(Collectors.toSet()); newKafkaSourceEnumState = new KafkaSourceEnumState( - activeAssignedPartitions, - activeUnassignedInitialPartitions, - kafkaSourceEnumState.initialDiscoveryFinished()); + partitions, kafkaSourceEnumState.initialDiscoveryFinished()); } else { - newKafkaSourceEnumState = - new KafkaSourceEnumState( - Collections.emptySet(), Collections.emptySet(), false); + newKafkaSourceEnumState = new KafkaSourceEnumState(Collections.emptySet(), false); } // restarts enumerator from state using only the active topic partitions, to avoid diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java index 70c435ee3..66ceeeb8a 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java @@ -32,7 +32,8 @@ public class KafkaSourceEnumState { /** Partitions with status: ASSIGNED or UNASSIGNED_INITIAL. */ private final Set partitions; /** - * this flag will be marked as true if inital partitions are discovered after enumerator starts. + * this flag will be marked as true if initial partitions are discovered after enumerator + * starts. */ private final boolean initialDiscoveryFinished; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java index 0ea4d9f65..f8dc17deb 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java @@ -89,9 +89,7 @@ public KafkaSourceEnumState deserialize(int version, byte[] serialized) throws I case CURRENT_VERSION: return deserializeTopicPartitionAndAssignmentStatus(serialized); case VERSION_1: - final Set assignedPartitions = - deserializeTopicPartitions(serialized); - return new KafkaSourceEnumState(assignedPartitions, new HashSet<>(), true); + return deserializeAssignedTopicPartitions(serialized); case VERSION_0: Map> currentPartitionAssignment = SerdeUtils.deserializeSplitAssignments( @@ -113,23 +111,24 @@ public KafkaSourceEnumState deserialize(int version, byte[] serialized) throws I } } - private static Set deserializeTopicPartitions(byte[] serializedTopicPartitions) - throws IOException { + private static KafkaSourceEnumState deserializeAssignedTopicPartitions( + byte[] serializedTopicPartitions) throws IOException { try (ByteArrayInputStream bais = new ByteArrayInputStream(serializedTopicPartitions); DataInputStream in = new DataInputStream(bais)) { final int numPartitions = in.readInt(); - Set topicPartitions = new HashSet<>(numPartitions); + Set partitions = new HashSet<>(numPartitions); for (int i = 0; i < numPartitions; i++) { final String topic = in.readUTF(); final int partition = in.readInt(); - topicPartitions.add(new TopicPartition(topic, partition)); + partitions.add( + new TopicPartitionAndAssignmentStatus( + new TopicPartition(topic, partition), AssignmentStatus.ASSIGNED)); } if (in.available() > 0) { throw new IOException("Unexpected trailing bytes in serialized topic partitions"); } - - return topicPartitions; + return new KafkaSourceEnumState(partitions, true); } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java index 02323a74f..10025fa2a 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java @@ -112,7 +112,7 @@ public KafkaSourceEnumerator( properties, context, boundedness, - new KafkaSourceEnumState(Collections.emptySet(), Collections.emptySet(), false)); + new KafkaSourceEnumState(Collections.emptySet(), false)); } public KafkaSourceEnumerator( From c0be7d5ca65b37110bba2b3281de952ad36124ad Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Fri, 13 Sep 2024 15:03:10 +0200 Subject: [PATCH 283/322] [FLINK-36278] Reduce log size by avoiding container logs by default Currently, container logs appear under an o.a.f logger and thus are visible on CI. This results in compressed log size >40MB for a run and often leads to download errors. This PR reroutes container logs to a special container logger. It also uses a custom format to significantly reduce the size of each log line. The logs for containers are disabled by default. --- .../kafka/SQLClientSchemaRegistryITCase.java | 18 ++--- .../tests/util/kafka/SmokeKafkaITCase.java | 11 ++- .../src/test/resources/log4j2-test.properties | 24 ++++++ .../FlinkKafkaInternalProducerITCase.java | 8 +- .../connector/kafka/sink/KafkaSinkITCase.java | 3 +- .../kafka/sink/KafkaTransactionLogITCase.java | 3 +- .../kafka/sink/KafkaWriterTestBase.java | 3 +- .../connector/kafka/testutils/KafkaUtil.java | 73 ++++++++++--------- .../FlinkKafkaProducerMigrationTest.java | 1 - .../kafka/KafkaTestEnvironmentImpl.java | 2 +- .../KafkaMetricMutableWrapperTest.java | 6 +- .../kafka/table/KafkaTableTestBase.java | 15 +--- .../src/test/resources/log4j2-test.properties | 19 ++++- tools/ci/log4j.properties | 24 ++++++ 14 files changed, 125 insertions(+), 85 deletions(-) diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java index 721cf59d7..e3b18194a 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.connector.kafka.testutils.DockerImageVersions; +import org.apache.flink.connector.kafka.testutils.KafkaUtil; import org.apache.flink.connector.testframe.container.FlinkContainers; import org.apache.flink.connector.testframe.container.TestcontainersSettings; import org.apache.flink.test.resources.ResourceTestUtils; @@ -40,11 +41,8 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.Timeout; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.containers.Network; -import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.utility.DockerImageName; import java.nio.file.Path; @@ -60,9 +58,6 @@ /** End-to-end test for SQL client using Avro Confluent Registry format. */ public class SQLClientSchemaRegistryITCase { - private static final Logger LOG = LoggerFactory.getLogger(SQLClientSchemaRegistryITCase.class); - private static final Slf4jLogConsumer LOG_CONSUMER = new Slf4jLogConsumer(LOG); - public static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; public static final String INTER_CONTAINER_REGISTRY_ALIAS = "registry"; private static final Path sqlAvroJar = ResourceTestUtils.getResource(".*avro.jar"); @@ -78,10 +73,9 @@ public class SQLClientSchemaRegistryITCase { @ClassRule public static final KafkaContainer KAFKA = - new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA)) + KafkaUtil.createKafkaContainer(SQLClientSchemaRegistryITCase.class) .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS) - .withLogConsumer(LOG_CONSUMER); + .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); @ClassRule public static final SchemaRegistryContainer REGISTRY = @@ -92,7 +86,11 @@ public class SQLClientSchemaRegistryITCase { .dependsOn(KAFKA); public final TestcontainersSettings testcontainersSettings = - TestcontainersSettings.builder().network(NETWORK).logger(LOG).dependsOn(KAFKA).build(); + TestcontainersSettings.builder() + .network(NETWORK) + .logger(KafkaUtil.getLogger("flink", SQLClientSchemaRegistryITCase.class)) + .dependsOn(KAFKA) + .build(); public final FlinkContainers flink = FlinkContainers.builder().withTestcontainersSettings(testcontainersSettings).build(); diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java index b6102ef92..eef63e57c 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java @@ -45,8 +45,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.RegisterExtension; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.containers.Network; import org.testcontainers.junit.jupiter.Container; @@ -62,7 +60,6 @@ import java.util.UUID; import java.util.stream.Collectors; -import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.assertj.core.api.Assertions.assertThat; @@ -71,20 +68,22 @@ @Testcontainers class SmokeKafkaITCase { - private static final Logger LOG = LoggerFactory.getLogger(SmokeKafkaITCase.class); private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; private static final Network NETWORK = Network.newNetwork(); private static final String EXAMPLE_JAR_MATCHER = "flink-streaming-kafka-test.*"; @Container public static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) + createKafkaContainer(SmokeKafkaITCase.class) .withEmbeddedZookeeper() .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); public static final TestcontainersSettings TESTCONTAINERS_SETTINGS = - TestcontainersSettings.builder().logger(LOG).dependsOn(KAFKA_CONTAINER).build(); + TestcontainersSettings.builder() + .logger(KafkaUtil.getLogger("flink", SmokeKafkaITCase.class)) + .dependsOn(KAFKA_CONTAINER) + .build(); @RegisterExtension public static final FlinkContainers FLINK = diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties index 358fd81ef..9c49ae58a 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties @@ -32,3 +32,27 @@ appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n #logger.yarn.name = org.testcontainers.shaded.com.github.dockerjava.core #logger.yarn.level = WARN #logger.yarn.appenderRef.console.ref = TestLogger + +# Logger configuration for containers, by default this is off +# If you want to investigate test failures, overwrite the level as above +logger.container.name = container +logger.container.level = OFF +logger.container.additivity = false # This prevents messages from being logged by the root logger +logger.container.appenderRef.containerappender.ref = ContainerLogger + +logger.kafkacontainer.name = container.kafka +logger.kafkacontainer.level = OFF + +logger.flinkcontainer.name = container.flink +logger.flinkcontainer.level = OFF + +logger.flinkenv.name = org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment +logger.flinkenv.level = OFF +logger.flinkenv.additivity = false # This prevents messages from being logged by the root logger +logger.flinkenv.appenderRef.containerappender.ref = ContainerLogger + +appender.containerappender.name = ContainerLogger +appender.containerappender.type = CONSOLE +appender.containerappender.target = SYSTEM_ERR +appender.containerappender.layout.type = PatternLayout +appender.containerappender.layout.pattern = [%c{1}] %m%n diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java index 22795e7fe..a4685be07 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java @@ -34,8 +34,6 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; @@ -47,7 +45,6 @@ import java.util.function.Consumer; import java.util.stream.Collectors; -import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -56,12 +53,9 @@ @ExtendWith(TestLoggerExtension.class) class FlinkKafkaInternalProducerITCase { - private static final Logger LOG = - LoggerFactory.getLogger(FlinkKafkaInternalProducerITCase.class); - @Container private static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG).withEmbeddedZookeeper(); + createKafkaContainer(FlinkKafkaInternalProducerITCase.class).withEmbeddedZookeeper(); @Test void testInitTransactionId() { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java index 9cc80518a..a257a44d4 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java @@ -101,7 +101,6 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; -import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; @@ -124,7 +123,7 @@ public class KafkaSinkITCase extends TestLogger { @ClassRule public static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) + createKafkaContainer(KafkaSinkITCase.class) .withEmbeddedZookeeper() .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java index 1497c9bb9..121955acb 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java @@ -44,7 +44,6 @@ import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.Ongoing; import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.PrepareAbort; import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.PrepareCommit; -import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.assertj.core.api.Assertions.assertThat; @@ -57,7 +56,7 @@ public class KafkaTransactionLogITCase extends TestLogger { @ClassRule public static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG).withEmbeddedZookeeper(); + createKafkaContainer(KafkaTransactionLogITCase.class).withEmbeddedZookeeper(); private final List> openProducers = new ArrayList<>(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java index c9dce3670..c5e6e9199 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java @@ -55,7 +55,6 @@ import java.util.concurrent.ScheduledFuture; import java.util.function.Consumer; -import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; /** Test base for KafkaWriter. */ @@ -73,7 +72,7 @@ public abstract class KafkaWriterTestBase { protected TriggerTimeService timeService; protected static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) + createKafkaContainer(KafkaWriterTestBase.class) .withEmbeddedZookeeper() .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java index 267f7c7c8..3dbe9bbbf 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java @@ -18,8 +18,6 @@ package org.apache.flink.connector.kafka.testutils; -import org.apache.flink.util.StringUtils; - import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -49,24 +47,34 @@ public class KafkaUtil { private KafkaUtil() {} - /** - * This method helps to set commonly used Kafka configurations and aligns the internal Kafka log - * levels with the ones used by the capturing logger. - * - * @param dockerImageVersion describing the Kafka image - * @param logger to derive the log level from - * @return configured Kafka container - */ - public static KafkaContainer createKafkaContainer(String dockerImageVersion, Logger logger) { - return createKafkaContainer(dockerImageVersion, logger, null); + /** This method helps to set commonly used Kafka configurations and sets up the logger. */ + public static KafkaContainer createKafkaContainer(Class testCase) { + return createKafkaContainer(getContainerName("kafka", testCase)); } - /** - * This method helps to set commonly used Kafka configurations and aligns the internal Kafka log - * levels with the ones used by the capturing logger, and set the prefix of logger. - */ - public static KafkaContainer createKafkaContainer( - String dockerImageVersion, Logger logger, String loggerPrefix) { + /** This method helps to set commonly used Kafka configurations and sets up the logger. */ + public static KafkaContainer createKafkaContainer(String containerName) { + Logger logger = getLogger(containerName); + + String logLevel = inferLogLevel(logger); + + Slf4jLogConsumer logConsumer = new Slf4jLogConsumer(logger, true); + return new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA)) + .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1") + .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1") + .withEnv("KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR", "1") + .withEnv("KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE", "false") + .withEnv("KAFKA_LOG4J_ROOT_LOGLEVEL", logLevel) + .withEnv("KAFKA_LOG4J_LOGGERS", "state.change.logger=" + logLevel) + .withEnv("KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE", "false") + .withEnv( + "KAFKA_TRANSACTION_MAX_TIMEOUT_MS", + String.valueOf(Duration.ofHours(2).toMillis())) + .withEnv("KAFKA_LOG4J_TOOLS_ROOT_LOGLEVEL", logLevel) + .withLogConsumer(logConsumer); + } + + private static String inferLogLevel(Logger logger) { String logLevel; if (logger.isTraceEnabled()) { logLevel = "TRACE"; @@ -81,24 +89,19 @@ public static KafkaContainer createKafkaContainer( } else { logLevel = "OFF"; } + return logLevel; + } - Slf4jLogConsumer logConsumer = new Slf4jLogConsumer(logger); - if (!StringUtils.isNullOrWhitespaceOnly(loggerPrefix)) { - logConsumer.withPrefix(loggerPrefix); - } - return new KafkaContainer(DockerImageName.parse(dockerImageVersion)) - .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1") - .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1") - .withEnv("KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR", "1") - .withEnv("KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE", "false") - .withEnv("KAFKA_LOG4J_ROOT_LOGLEVEL", logLevel) - .withEnv("KAFKA_LOG4J_LOGGERS", "state.change.logger=" + logLevel) - .withEnv("KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE", "false") - .withEnv( - "KAFKA_TRANSACTION_MAX_TIMEOUT_MS", - String.valueOf(Duration.ofHours(2).toMillis())) - .withEnv("KAFKA_LOG4J_TOOLS_ROOT_LOGLEVEL", logLevel) - .withLogConsumer(logConsumer); + public static Logger getLogger(String containerName) { + return LoggerFactory.getLogger("container." + containerName); + } + + public static Logger getLogger(String type, Class testClass) { + return getLogger(getContainerName(type, testClass)); + } + + private static String getContainerName(String type, Class testClass) { + return type + "." + testClass.getSimpleName(); } /** diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java index 98ab88296..8a413f423 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java @@ -54,7 +54,6 @@ protected Properties createProperties() { Properties properties = new Properties(); properties.putAll(standardProps); properties.putAll(secureProps); - properties.put(ProducerConfig.CLIENT_ID_CONFIG, "producer-client-id"); properties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "producer-transaction-id"); properties.put(FlinkKafkaProducer.KEY_DISABLE_METRICS, "true"); return properties; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 431e2924b..31ddbbedf 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -434,7 +434,7 @@ private KafkaContainer createKafkaContainer( int brokerID, @Nullable GenericContainer zookeeper) { String brokerName = String.format("Kafka-%d", brokerID); KafkaContainer broker = - KafkaUtil.createKafkaContainer(DockerImageVersions.KAFKA, LOG, brokerName) + KafkaUtil.createKafkaContainer(brokerName) .withNetworkAliases(brokerName) .withEnv("KAFKA_BROKER_ID", String.valueOf(brokerID)) .withEnv("KAFKA_MESSAGE_MAX_BYTES", String.valueOf(50 * 1024 * 1024)) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java index c9948e00c..295db6155 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java @@ -28,8 +28,6 @@ import org.apache.kafka.common.serialization.ByteArraySerializer; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.containers.Network; import org.testcontainers.junit.jupiter.Container; @@ -42,20 +40,18 @@ import java.util.function.Function; import java.util.stream.Stream; -import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; @Testcontainers @ExtendWith(TestLoggerExtension.class) class KafkaMetricMutableWrapperTest { - private static final Logger LOG = LoggerFactory.getLogger(KafkaMetricMutableWrapperTest.class); private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; private static final Network NETWORK = Network.newNetwork(); @Container public static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) + createKafkaContainer(KafkaMetricMutableWrapperTest.class) .withEmbeddedZookeeper() .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java index e9d8bb76b..cc7c73271 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.kafka.table; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.connector.kafka.testutils.DockerImageVersions; +import org.apache.flink.connector.kafka.testutils.KafkaUtil; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.test.util.AbstractTestBase; @@ -42,8 +42,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.utility.DockerImageName; import java.time.Duration; import java.util.ArrayList; @@ -66,15 +64,8 @@ public abstract class KafkaTableTestBase extends AbstractTestBase { @ClassRule public static final KafkaContainer KAFKA_CONTAINER = - new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA)) { - @Override - protected void doStart() { - super.doStart(); - if (LOG.isInfoEnabled()) { - this.followOutput(new Slf4jLogConsumer(LOG)); - } - } - }.withEmbeddedZookeeper() + KafkaUtil.createKafkaContainer(KafkaTableTestBase.class) + .withEmbeddedZookeeper() .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS) .withEnv( "KAFKA_TRANSACTION_MAX_TIMEOUT_MS", diff --git a/flink-connector-kafka/src/test/resources/log4j2-test.properties b/flink-connector-kafka/src/test/resources/log4j2-test.properties index 3d901331b..88c8d66df 100644 --- a/flink-connector-kafka/src/test/resources/log4j2-test.properties +++ b/flink-connector-kafka/src/test/resources/log4j2-test.properties @@ -37,5 +37,20 @@ logger.zookeeper.level = OFF logger.I0Itec.name = org.I0Itec logger.I0Itec.level = OFF -logger.splitreader.name = org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader -logger.splitreader.level = DEBUG +# Logger configuration for containers, by default this is off +# If you want to investigate test failures, overwrite the level as above +logger.container.name = container +logger.container.level = OFF +logger.container.additivity = false # This prevents messages from being logged by the root logger +logger.container.appenderRef.containerappender.ref = ContainerLogger + +logger.flinkenv.name = org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment +logger.flinkenv.level = OFF +logger.flinkenv.additivity = false # This prevents messages from being logged by the root logger +logger.flinkenv.appenderRef.containerappender.ref = ContainerLogger + +appender.containerappender.name = ContainerLogger +appender.containerappender.type = CONSOLE +appender.containerappender.target = SYSTEM_ERR +appender.containerappender.layout.type = PatternLayout +appender.containerappender.layout.pattern = [%c{1}] %m%n diff --git a/tools/ci/log4j.properties b/tools/ci/log4j.properties index 7daf1c336..1199c24b0 100644 --- a/tools/ci/log4j.properties +++ b/tools/ci/log4j.properties @@ -41,3 +41,27 @@ appender.file.createOnDemand = true # suppress the irrelevant (wrong) warnings from the netty channel handler logger.netty.name = org.jboss.netty.channel.DefaultChannelPipeline logger.netty.level = ERROR + +# Logger configuration for containers, by default this is off +# If you want to investigate test failures, overwrite the level as above +logger.container.name = container +logger.container.level = OFF +logger.container.additivity = false # This prevents messages from being logged by the root logger +logger.container.appenderRef.containerappender.ref = ContainerLogger + +logger.kafkacontainer.name = container.kafka +logger.kafkacontainer.level = WARN + +logger.flinkcontainer.name = container.flink +logger.flinkcontainer.level = WARN + +logger.flinkenv.name = org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment +logger.flinkenv.level = WARN +logger.flinkenv.additivity = false # This prevents messages from being logged by the root logger +logger.flinkenv.appenderRef.containerappender.ref = ContainerLogger + +appender.containerappender.name = ContainerLogger +appender.containerappender.type = CONSOLE +appender.containerappender.target = SYSTEM_ERR +appender.containerappender.layout.type = PatternLayout +appender.containerappender.layout.pattern = [%c{1}] %m%n From 2f8648464d71f3799e6398295ad3e983c11d72ad Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 17 Sep 2024 20:30:20 +0200 Subject: [PATCH 284/322] [FLINK-36278] Decrease log levels for hotpath logs We should never use INFO for tracking records on the hotpath. Ideally, we would use trace but for this commit I just decreased to DEBUG to minimize the impact on production settings (is it even possible to leave production on INFO currently?). --- .../kafka/dynamic/source/reader/DynamicKafkaSourceReader.java | 2 +- .../kafka/testutils/DynamicKafkaSourceExternalContext.java | 2 +- .../flink/streaming/connectors/kafka/KafkaConsumerTestBase.java | 2 +- .../apache/flink/streaming/connectors/kafka/KafkaTestBase.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java index 4f307e11a..8220ea14c 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java @@ -138,7 +138,7 @@ public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { } if (restartingReaders.get()) { - logger.info("Poll next invoked while restarting readers"); + logger.debug("Poll next invoked while restarting readers"); return logAndReturnInputStatus(InputStatus.NOTHING_AVAILABLE); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java index e9bc77e83..a165cc790 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java @@ -240,7 +240,7 @@ public void writeRecords(List records) { } } - logger.info("Writing producer records: {}", producerRecords); + logger.debug("Writing producer records: {}", producerRecords); DynamicKafkaSourceTestHelper.produceToKafka( clusterPropertiesMap.get(cluster), diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index 026d49bdf..9a9acdea0 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -2097,7 +2097,7 @@ public void flatMap( count++; - LOG.info("Received message {}, total {} messages", value, count); + LOG.debug("Received message {}, total {} messages", value, count); // verify if we've seen everything if (count == finalCount) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index 4b9acbf1e..75341f956 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -167,7 +167,7 @@ public static void startClusters(KafkaTestEnvironment.Config environmentConfig) throws Exception { kafkaServer = constructKafkaTestEnvironment(); - LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion()); + LOG.info("Starting KafkaTestBase.prepare() for Kafka {}", kafkaServer.getVersion()); kafkaServer.prepare(environmentConfig); From ec8d7c203b9522d0831448168ee610f74051d5b9 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 17 Sep 2024 23:30:52 +0200 Subject: [PATCH 285/322] [FLINK-36278] Reduce Kafka log level Reduce the information that Kafka consumer and producer is logging + Kafka server for old school tests. --- .../src/test/resources/log4j2-test.properties | 10 ++-------- tools/ci/log4j.properties | 4 ++++ 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/flink-connector-kafka/src/test/resources/log4j2-test.properties b/flink-connector-kafka/src/test/resources/log4j2-test.properties index 88c8d66df..0ca74ace0 100644 --- a/flink-connector-kafka/src/test/resources/log4j2-test.properties +++ b/flink-connector-kafka/src/test/resources/log4j2-test.properties @@ -27,15 +27,9 @@ appender.testlogger.target = SYSTEM_ERR appender.testlogger.layout.type = PatternLayout appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n -logger.kafka.name = kafka +# Kafka producer and consumer level +logger.kafka.name = org.apache.kafka logger.kafka.level = OFF -logger.kafka2.name = state.change -logger.kafka2.level = OFF - -logger.zookeeper.name = org.apache.zookeeper -logger.zookeeper.level = OFF -logger.I0Itec.name = org.I0Itec -logger.I0Itec.level = OFF # Logger configuration for containers, by default this is off # If you want to investigate test failures, overwrite the level as above diff --git a/tools/ci/log4j.properties b/tools/ci/log4j.properties index 1199c24b0..260fb17cd 100644 --- a/tools/ci/log4j.properties +++ b/tools/ci/log4j.properties @@ -38,6 +38,10 @@ appender.file.layout.type = PatternLayout appender.file.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n appender.file.createOnDemand = true +# Kafka producer and consumer level +logger.kafka.name = org.apache.kafka +logger.kafka.level = OFF + # suppress the irrelevant (wrong) warnings from the netty channel handler logger.netty.name = org.jboss.netty.channel.DefaultChannelPipeline logger.netty.level = ERROR From 1b02ce8c16943565d0c10277c55aeb0d04620e45 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 19 Sep 2024 09:40:22 +0200 Subject: [PATCH 286/322] [FLINK-36278] Reduce Flink log levels Keep INFO only for connector related logs. This will avoid all the different JM and TM logs that deal with task life-cycles. Exceptional things still bubble up as warnings. --- .../src/test/resources/log4j2-test.properties | 6 ++++++ tools/ci/log4j.properties | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/flink-connector-kafka/src/test/resources/log4j2-test.properties b/flink-connector-kafka/src/test/resources/log4j2-test.properties index 0ca74ace0..920652c94 100644 --- a/flink-connector-kafka/src/test/resources/log4j2-test.properties +++ b/flink-connector-kafka/src/test/resources/log4j2-test.properties @@ -27,6 +27,12 @@ appender.testlogger.target = SYSTEM_ERR appender.testlogger.layout.type = PatternLayout appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n +# Overwrite the level for all Flink related loggers +logger.flink.name = org.apache.flink +logger.flink.level = OFF # WARN for starting debugging +logger.flinkconnector.name = org.apache.flink.connector +logger.flinkconnector.level = OFF # INFO/DEBUG for starting debugging + # Kafka producer and consumer level logger.kafka.name = org.apache.kafka logger.kafka.level = OFF diff --git a/tools/ci/log4j.properties b/tools/ci/log4j.properties index 260fb17cd..25ef1cf96 100644 --- a/tools/ci/log4j.properties +++ b/tools/ci/log4j.properties @@ -38,6 +38,12 @@ appender.file.layout.type = PatternLayout appender.file.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n appender.file.createOnDemand = true +# Reduce most flink logs except for connector specific loggers +logger.flink.name = org.apache.flink +logger.flink.level = WARN +logger.flinkconnector.name = org.apache.flink.connector +logger.flinkconnector.level = INFO + # Kafka producer and consumer level logger.kafka.name = org.apache.kafka logger.kafka.level = OFF From 2ee9b9af1126caf8735aa73d647d4a727a2fef5c Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 19 Sep 2024 20:39:32 +0200 Subject: [PATCH 287/322] [hotfix] Remove unused test dependencies --- flink-connector-kafka/pom.xml | 26 ------------------- .../connectors/kafka/KafkaTestBase.java | 5 ---- pom.xml | 17 ------------ 3 files changed, 48 deletions(-) diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index c16d4d79b..4a10bdc9a 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -133,32 +133,6 @@ under the License. test-jar - - - - org.apache.kafka - kafka_${scala.binary.version} - ${kafka.version} - - - org.slf4j - slf4j-api - - - io.dropwizard.metrics - metrics-core - - - test - - - - org.apache.zookeeper - zookeeper - ${zookeeper.version} - test - - org.testcontainers kafka diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index 75341f956..eb35391f5 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -55,11 +55,8 @@ import java.util.List; import java.util.Properties; import java.util.Set; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import scala.concurrent.duration.FiniteDuration; - import static org.assertj.core.api.Assertions.fail; /** @@ -92,8 +89,6 @@ public abstract class KafkaTestBase extends TestLogger { public static Properties standardProps; - public static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS); - public static KafkaTestEnvironment kafkaServer; public static List kafkaClusters = new ArrayList<>(); diff --git a/pom.xml b/pom.xml index 92e1ebd78..9215540fd 100644 --- a/pom.xml +++ b/pom.xml @@ -52,7 +52,6 @@ under the License. 1.17.0 3.4.0 - 3.7.2 7.4.4 2.15.2 @@ -232,22 +231,6 @@ under the License. ${kafka.version} - - org.apache.zookeeper - zookeeper - ${zookeeper.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.xerial.snappy snappy-java From 9b97c511bdef50ac2d25656b1f1c1ffc497a2ea6 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 19 Sep 2024 22:19:07 +0200 Subject: [PATCH 288/322] [hotfix] Fix FlinkKafkaConsumerBaseTest.testClosePartitionDiscovererWithCancellation Make a copy of AbstractPartitionDiscoverer#getAllTopics before modifying it. --- .../AbstractPartitionDiscoverer.java | 29 +++++++------------ 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java index 05e078f66..18322bf68 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java @@ -19,8 +19,8 @@ import org.apache.flink.annotation.Internal; +import java.util.ArrayList; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Set; @@ -130,21 +130,18 @@ public List discoverPartitions() throws WakeupException, Cl // topics or a topic pattern if (topicsDescriptor.isFixedTopics()) { newDiscoveredPartitions = - getAllPartitionsForTopics(topicsDescriptor.getFixedTopics()); + new ArrayList<>( + getAllPartitionsForTopics(topicsDescriptor.getFixedTopics())); } else { - List matchedTopics = getAllTopics(); + List matchedTopics = new ArrayList<>(getAllTopics()); // retain topics that match the pattern - Iterator iter = matchedTopics.iterator(); - while (iter.hasNext()) { - if (!topicsDescriptor.isMatchingTopic(iter.next())) { - iter.remove(); - } - } + matchedTopics.removeIf(s -> !topicsDescriptor.isMatchingTopic(s)); - if (matchedTopics.size() != 0) { + if (!matchedTopics.isEmpty()) { // get partitions only for matched topics - newDiscoveredPartitions = getAllPartitionsForTopics(matchedTopics); + newDiscoveredPartitions = + new ArrayList<>(getAllPartitionsForTopics(matchedTopics)); } else { newDiscoveredPartitions = null; } @@ -157,14 +154,8 @@ public List discoverPartitions() throws WakeupException, Cl "Unable to retrieve any partitions with KafkaTopicsDescriptor: " + topicsDescriptor); } else { - Iterator iter = newDiscoveredPartitions.iterator(); - KafkaTopicPartition nextPartition; - while (iter.hasNext()) { - nextPartition = iter.next(); - if (!setAndCheckDiscoveredPartition(nextPartition)) { - iter.remove(); - } - } + newDiscoveredPartitions.removeIf( + nextPartition -> !setAndCheckDiscoveredPartition(nextPartition)); } return newDiscoveredPartitions; From 21d07b7d468ad6d3b0b1a3120745e17ceb1ab1c2 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 26 Sep 2024 14:05:42 +0200 Subject: [PATCH 289/322] [FLINK-36177] Deprecate KafkaShuffle and more This commit deprecates all classes that are slated for removal in the kafka-4.0 release compatible with Flink 2.0. I also deprecated internal classes to make later removal easier. Some public classes will cease to be public API but are still internally used. --- .../deserializer/KafkaDeserializationSchemaWrapper.java | 2 ++ .../deserializer/KafkaRecordDeserializationSchema.java | 2 ++ .../connectors/kafka/FlinkKafkaConsumerBase.java | 1 + .../streaming/connectors/kafka/FlinkKafkaErrorCode.java | 8 +++++++- .../streaming/connectors/kafka/FlinkKafkaException.java | 8 +++++++- .../streaming/connectors/kafka/KafkaContextAware.java | 3 +++ .../connectors/kafka/KafkaDeserializationSchema.java | 2 ++ .../connectors/kafka/KafkaSerializationSchema.java | 2 ++ .../connectors/kafka/config/OffsetCommitMode.java | 1 + .../connectors/kafka/config/OffsetCommitModes.java | 1 + .../connectors/kafka/internals/AbstractFetcher.java | 1 + .../kafka/internals/AbstractPartitionDiscoverer.java | 1 + .../connectors/kafka/internals/ClosableBlockingQueue.java | 1 + .../connectors/kafka/internals/ExceptionProxy.java | 1 + .../kafka/internals/FlinkKafkaInternalProducer.java | 1 + .../streaming/connectors/kafka/internals/Handover.java | 1 + .../connectors/kafka/internals/KafkaCommitCallback.java | 1 + .../connectors/kafka/internals/KafkaConsumerThread.java | 1 + .../internals/KafkaDeserializationSchemaWrapper.java | 1 + .../connectors/kafka/internals/KafkaFetcher.java | 1 + .../kafka/internals/KafkaPartitionDiscoverer.java | 1 + .../kafka/internals/KafkaSerializationSchemaWrapper.java | 1 + .../connectors/kafka/internals/KafkaShuffleFetcher.java | 1 + .../connectors/kafka/internals/KafkaTopicPartition.java | 5 +++++ .../kafka/internals/KafkaTopicPartitionAssigner.java | 1 + .../kafka/internals/KafkaTopicPartitionLeader.java | 1 + .../kafka/internals/KafkaTopicPartitionState.java | 1 + .../KafkaTopicPartitionStateWithWatermarkGenerator.java | 1 + .../connectors/kafka/internals/KafkaTopicsDescriptor.java | 1 + .../kafka/internals/KeyedSerializationSchemaWrapper.java | 1 + .../internals/SourceContextWatermarkOutputAdapter.java | 1 + .../kafka/internals/TransactionalIdsGenerator.java | 1 + .../internals/metrics/KafkaConsumerMetricConstants.java | 1 + .../kafka/internals/metrics/KafkaMetricWrapper.java | 1 + .../kafka/partitioner/FlinkFixedPartitioner.java | 4 ++++ .../kafka/partitioner/FlinkKafkaPartitioner.java | 4 ++++ .../connectors/kafka/shuffle/FlinkKafkaShuffle.java | 5 +++++ .../kafka/shuffle/FlinkKafkaShuffleConsumer.java | 1 + .../kafka/shuffle/FlinkKafkaShuffleProducer.java | 1 + .../connectors/kafka/shuffle/StreamKafkaShuffleSink.java | 1 + .../kafka/table/DynamicKafkaDeserializationSchema.java | 2 ++ .../table/DynamicKafkaRecordSerializationSchema.java | 2 ++ .../serialization/JSONKeyValueDeserializationSchema.java | 1 + .../TypeInformationKeyValueSerializationSchema.java | 1 + 44 files changed, 78 insertions(+), 2 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java index 94197e347..1cc7dde79 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java @@ -33,7 +33,9 @@ * ConsumerRecord ConsumerRecords}. * * @param the type of the deserialized records. + * @deprecated Remove with @{@link KafkaDeserializationSchema} */ +@Deprecated class KafkaDeserializationSchemaWrapper implements KafkaRecordDeserializationSchema { private static final long serialVersionUID = 1L; private final KafkaDeserializationSchema kafkaDeserializationSchema; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java index 6ad6607c9..91d1f3439 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java @@ -71,7 +71,9 @@ default void open(DeserializationSchema.InitializationContext context) throws Ex * @param the return type of the deserialized record. * @return A {@link KafkaRecordDeserializationSchema} that uses the given {@link * KafkaDeserializationSchema} to deserialize the {@link ConsumerRecord ConsumerRecords}. + * @deprecated Will be removed with {@link KafkaDeserializationSchema}. */ + @Deprecated static KafkaRecordDeserializationSchema of( KafkaDeserializationSchema kafkaDeserializationSchema) { return new KafkaDeserializationSchemaWrapper<>(kafkaDeserializationSchema); diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index f291b05bc..7a85b434e 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -90,6 +90,7 @@ * @param The type of records produced by this data source */ @Internal +@Deprecated public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFunction implements CheckpointListener, ResultTypeQueryable, CheckpointedFunction { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java index c11f1b176..3c1ae27e3 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java @@ -19,8 +19,14 @@ import org.apache.flink.annotation.PublicEvolving; -/** Error codes used in {@link FlinkKafkaException}. */ +/** + * Error codes used in {@link FlinkKafkaException}. + * + * @deprecated Will be removed with {@link FlinkKafkaProducer} and {@link + * org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffle}. + */ @PublicEvolving +@Deprecated public enum FlinkKafkaErrorCode { PRODUCERS_POOL_EMPTY, EXTERNAL_ERROR diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java index 77d023130..65b654c64 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java @@ -20,8 +20,14 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.util.FlinkException; -/** Exception used by {@link FlinkKafkaProducer} and {@link FlinkKafkaConsumer}. */ +/** + * Exception used by {@link FlinkKafkaProducer} and {@link FlinkKafkaConsumer}. + * + * @deprecated Will be removed with {@link FlinkKafkaProducer} and {@link + * org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffle}. + */ @PublicEvolving +@Deprecated public class FlinkKafkaException extends FlinkException { private static final long serialVersionUID = 920269130311214200L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java index ad977cd95..d40139595 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java @@ -26,8 +26,11 @@ * *

      You only need to override the methods for the information that you need. However, {@link * #getTargetTopic(Object)} is required because it is used to determine the available partitions. + * + * @deprecated Will be turned into internal API when {@link FlinkKafkaProducer} is removed. */ @PublicEvolving +@Deprecated public interface KafkaContextAware { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java index b54b9835f..8f15b921b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java @@ -31,8 +31,10 @@ * (Java/Scala objects) that are processed by Flink. * * @param The type created by the keyed deserialization schema. + * @deprecated Will be turned into internal API when {@link FlinkKafkaConsumer} is removed. */ @PublicEvolving +@Deprecated public interface KafkaDeserializationSchema extends Serializable, ResultTypeQueryable { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java index 89e2b9209..7ed987fce 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java @@ -35,8 +35,10 @@ * which the Kafka Producer is running. * * @param the type of values being serialized + * @deprecated Will be turned into internal API when {@link FlinkKafkaProducer} is removed. */ @PublicEvolving +@Deprecated public interface KafkaSerializationSchema extends Serializable { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java index 32b9d4cdc..6ad4f8337 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java @@ -26,6 +26,7 @@ *

      The exact value of this is determined at runtime in the consumer subtasks. */ @Internal +@Deprecated public enum OffsetCommitMode { /** Completely disable offset committing. */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java index 1394af72f..32ac2f5f0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java @@ -21,6 +21,7 @@ /** Utilities for {@link OffsetCommitMode}. */ @Internal +@Deprecated public class OffsetCommitModes { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java index 41b5ad24c..074363021 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java @@ -61,6 +61,7 @@ * @param The type of topic/partition identifier used by Kafka in the specific version. */ @Internal +@Deprecated public abstract class AbstractFetcher { private static final int NO_TIMESTAMPS_WATERMARKS = 0; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java index 18322bf68..c8dc18360 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java @@ -41,6 +41,7 @@ * allows the discoverer to be interrupted during a {@link #discoverPartitions()} call. */ @Internal +@Deprecated public abstract class AbstractPartitionDiscoverer { /** Describes whether we are discovering partitions for fixed topics or a topic pattern. */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java index 22800611d..3b1751d40 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java @@ -49,6 +49,7 @@ * @param The type of elements in the queue. */ @Internal +@Deprecated public class ClosableBlockingQueue { /** The lock used to make queue accesses and open checks atomic. */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java index 204a01b4c..a9f9c9cae 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java @@ -65,6 +65,7 @@ * } */ @Internal +@Deprecated public class ExceptionProxy { /** The thread that should be interrupted when an exception occurs. */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java index 12dad9fba..6e618cbe0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java @@ -58,6 +58,7 @@ /** Internal flink kafka producer. */ @PublicEvolving +@Deprecated public class FlinkKafkaInternalProducer implements Producer { private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaInternalProducer.class); diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java index 7fc50e625..64132b0b9 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java @@ -47,6 +47,7 @@ */ @ThreadSafe @Internal +@Deprecated public final class Handover implements Closeable { private final Object lock = new Object(); diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java index d7666772b..f1180b8b5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java @@ -25,6 +25,7 @@ * commit request completes, which should normally be triggered from checkpoint complete event. */ @Internal +@Deprecated public interface KafkaCommitCallback { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java index f7f40b80b..5b6fb4d43 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java @@ -61,6 +61,7 @@ * an indirection to the KafkaConsumer calls that change signature. */ @Internal +@Deprecated public class KafkaConsumerThread extends Thread { /** Logger for this consumer. */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java index d53e4ff4d..b754b4d09 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java @@ -32,6 +32,7 @@ * @param The type created by the deserialization schema. */ @Internal +@Deprecated public class KafkaDeserializationSchemaWrapper implements KafkaDeserializationSchema { private static final long serialVersionUID = 2651665280744549932L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java index 9c4d8387c..428e6c7ce 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java @@ -51,6 +51,7 @@ * @param The type of elements produced by the fetcher. */ @Internal +@Deprecated public class KafkaFetcher extends AbstractFetcher { private static final Logger LOG = LoggerFactory.getLogger(KafkaFetcher.class); diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java index ec788991d..ef7162bde 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java @@ -34,6 +34,7 @@ * brokers via the Kafka high-level consumer API. */ @Internal +@Deprecated public class KafkaPartitionDiscoverer extends AbstractPartitionDiscoverer { private final Properties kafkaProperties; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java index 73b1d42a6..147fad9b6 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java @@ -35,6 +35,7 @@ * KafkaSerializationSchema}. */ @Internal +@Deprecated public class KafkaSerializationSchemaWrapper implements KafkaSerializationSchema, KafkaContextAware { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java index fe7ee7f7b..c61db83f0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java @@ -50,6 +50,7 @@ /** Fetch data from Kafka for Kafka Shuffle. */ @Internal +@Deprecated public class KafkaShuffleFetcher extends KafkaFetcher { /** The handler to check and generate watermarks from fetched records. * */ private final WatermarkHandler watermarkHandler; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java index f262a222c..0e91042f6 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java @@ -32,8 +32,13 @@ * *

      Note: This class must not change in its structure, because it would change the serialization * format and make previous savepoints unreadable. + * + * @deprecated Will be turned into internal class when {@link + * org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer} is removed. Replace with + * {@link org.apache.kafka.common.TopicPartition}. */ @PublicEvolving +@Deprecated public final class KafkaTopicPartition implements Serializable { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java index be61e8ad7..83c7483ff 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java @@ -21,6 +21,7 @@ /** Utility for assigning Kafka partitions to consumer subtasks. */ @Internal +@Deprecated public class KafkaTopicPartitionAssigner { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java index a2ef12882..031400d6e 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java @@ -27,6 +27,7 @@ * Serializable Topic Partition info with leader Node information. This class is used at runtime. */ @Internal +@Deprecated public class KafkaTopicPartitionLeader implements Serializable { private static final long serialVersionUID = 9145855900303748582L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java index c09df342c..ee669e7e1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java @@ -29,6 +29,7 @@ * @param The type of the Kafka partition descriptor, which varies across Kafka versions. */ @Internal +@Deprecated public class KafkaTopicPartitionState { // ------------------------------------------------------------------------ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java index 6c843409d..f9c815fcc 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java @@ -34,6 +34,7 @@ * @param The type of the Kafka partition descriptor, which varies across Kafka versions. */ @Internal +@Deprecated public final class KafkaTopicPartitionStateWithWatermarkGenerator extends KafkaTopicPartitionState { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java index 8261a2b31..4bb37b1c2 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java @@ -33,6 +33,7 @@ * list of topics, or a topic pattern. */ @Internal +@Deprecated public class KafkaTopicsDescriptor implements Serializable { private static final long serialVersionUID = -3807227764764900975L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java index c95cd9c40..ae4e922d5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java @@ -27,6 +27,7 @@ * @param The type to serialize */ @Internal +@Deprecated public class KeyedSerializationSchemaWrapper implements KeyedSerializationSchema { private static final long serialVersionUID = 1351665280744549933L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java index 8e5674313..68c4db12a 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java @@ -25,6 +25,7 @@ * A {@link org.apache.flink.api.common.eventtime.WatermarkOutput} that forwards calls to a {@link * org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext}. */ +@Deprecated public class SourceContextWatermarkOutputAdapter implements WatermarkOutput { private final SourceContext sourceContext; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java index e21355e8f..cd6270acc 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java @@ -40,6 +40,7 @@ * subtask. */ @Internal +@Deprecated public class TransactionalIdsGenerator { private final String prefix; private final int subtaskIndex; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java index 30d469714..731089028 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java @@ -26,6 +26,7 @@ * metrics. */ @Internal +@Deprecated public class KafkaConsumerMetricConstants { public static final String KAFKA_CONSUMER_METRICS_GROUP = "KafkaConsumer"; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java index 1ab41ce9c..2893bad33 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java @@ -23,6 +23,7 @@ /** Gauge for getting the current value of a Kafka metric. */ @Internal +@Deprecated public class KafkaMetricWrapper implements Gauge { private final org.apache.kafka.common.Metric kafkaMetric; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java index 16b57f621..e70baea3b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java @@ -54,8 +54,12 @@ *

      Not all Kafka partitions contain data To avoid such an unbalanced partitioning, use a * round-robin kafka partitioner (note that this will cause a lot of network connections between all * the Flink instances and all the Kafka brokers). + * + * @deprecated Will be turned into internal class when {@link + * org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer} is removed. */ @PublicEvolving +@Deprecated public class FlinkFixedPartitioner extends FlinkKafkaPartitioner { private static final long serialVersionUID = -3785320239953858777L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java index 2fb89e205..7318ed697 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java @@ -24,8 +24,12 @@ /** * A {@link FlinkKafkaPartitioner} wraps logic on how to partition records across partitions of * multiple Kafka topics. + * + * @deprecated Will be turned into internal class when {@link + * org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer} is removed. */ @PublicEvolving +@Deprecated public abstract class FlinkKafkaPartitioner implements Serializable { private static final long serialVersionUID = -9086719227828020494L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java index ae9af29f0..bb7c76a67 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java @@ -98,8 +98,13 @@ * | * | ----------> KafkaShuffleConsumerReuse -> ... * + * + * @deprecated This experimental feature never graduated to a stable feature and will be removed in + * future releases. In case of interest to port it to the Source/Sink API, please reach out to + * the Flink community. */ @Experimental +@Deprecated public class FlinkKafkaShuffle { static final String PRODUCER_PARALLELISM = "producer parallelism"; static final String PARTITION_NUMBER = "partition number"; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java index 886343bea..b96e9c0f5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java @@ -39,6 +39,7 @@ /** Flink Kafka Shuffle Consumer Function. */ @Internal +@Deprecated public class FlinkKafkaShuffleConsumer extends FlinkKafkaConsumer { private final TypeSerializer typeSerializer; private final int producerParallelism; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java index e05e8f9a3..46754f270 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java @@ -44,6 +44,7 @@ * handling elements and watermarks */ @Internal +@Deprecated public class FlinkKafkaShuffleProducer extends FlinkKafkaProducer { private final KafkaSerializer kafkaSerializer; private final KeySelector keySelector; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java index 8bd77840f..e24e15650 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java @@ -29,6 +29,7 @@ * this way to avoid public interface change. */ @Internal +@Deprecated class StreamKafkaShuffleSink extends StreamSink { public StreamKafkaShuffleSink(FlinkKafkaShuffleProducer flinkKafkaShuffleProducer) { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java index 91798281d..ef5eca95b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.kafka.table; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; @@ -38,6 +39,7 @@ import java.util.List; /** A specific {@link KafkaSerializationSchema} for {@link KafkaDynamicSource}. */ +@Internal class DynamicKafkaDeserializationSchema implements KafkaDeserializationSchema { private static final long serialVersionUID = 1L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java index 71ca41474..f3a7acb3b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.connectors.kafka.table; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; import org.apache.flink.connector.kafka.sink.KafkaSink; @@ -40,6 +41,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** SerializationSchema used by {@link KafkaDynamicSink} to configure a {@link KafkaSink}. */ +@Internal class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationSchema { private final Set topics; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java index 970bad1c3..cffdc8ea4 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java @@ -42,6 +42,7 @@ * (String) and "partition" (int). */ @PublicEvolving +@Deprecated public class JSONKeyValueDeserializationSchema implements KafkaDeserializationSchema { private static final long serialVersionUID = 1509391548173891955L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java index 1c3eaa620..05e0eaea1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java @@ -41,6 +41,7 @@ * @param The value type to be serialized. */ @PublicEvolving +@Deprecated public class TypeInformationKeyValueSerializationSchema implements KafkaDeserializationSchema>, KeyedSerializationSchema> { From 157813eac012847b4682313b4d1ae65428c5a354 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Fri, 27 Sep 2024 17:14:01 +0200 Subject: [PATCH 290/322] [FLINK-36177] Introduce KafkaPartitioner to replace FlinkKafkaPartitioner Relocate FlinkKafkaPartitioner to KafkaSink package and turn it into a functional interface. --- .../kafka/sink/KafkaPartitioner.java | 50 +++++++++++++++++++ ...KafkaRecordSerializationSchemaBuilder.java | 20 ++++++-- .../partitioner/FlinkKafkaPartitioner.java | 33 ++---------- ...DynamicKafkaRecordSerializationSchema.java | 6 +-- .../table/KafkaConnectorOptionsUtil.java | 16 +++--- .../kafka/table/KafkaDynamicSink.java | 6 +-- .../kafka/table/KafkaDynamicTableFactory.java | 4 +- 7 files changed, 87 insertions(+), 48 deletions(-) create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaPartitioner.java diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaPartitioner.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaPartitioner.java new file mode 100644 index 000000000..19a2d4731 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaPartitioner.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; + +/** + * A {@code KafkaPartitioner} wraps logic on how to partition records across partitions of multiple + * Kafka topics. + */ +@PublicEvolving +public interface KafkaPartitioner extends Serializable { + /** + * Initializer for the partitioner. This is called once on each parallel sink instance of the + * Flink Kafka producer. This method should be overridden if necessary. + * + * @param parallelInstanceId 0-indexed id of the parallel sink instance in Flink + * @param parallelInstances the total number of parallel instances + */ + default void open(int parallelInstanceId, int parallelInstances) {} + + /** + * Determine the id of the partition that the record should be written to. + * + * @param record the record value + * @param key serialized key of the record + * @param value serialized value of the record + * @param targetTopic target topic for the record + * @param partitions found partitions for the target topic + * @return the id of the target partition + */ + int partition(T record, byte[] key, byte[] value, String targetTopic, int[] partitions); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java index 92eb625b2..e9fc413b2 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java @@ -82,7 +82,7 @@ public class KafkaRecordSerializationSchemaBuilder { @Nullable private Function topicSelector; @Nullable private SerializationSchema valueSerializationSchema; - @Nullable private FlinkKafkaPartitioner partitioner; + @Nullable private KafkaPartitioner partitioner; @Nullable private SerializationSchema keySerializationSchema; @Nullable private HeaderProvider headerProvider; @@ -91,6 +91,7 @@ public class KafkaRecordSerializationSchemaBuilder { * * @param partitioner * @return {@code this} + * @deprecated use {@link #setPartitioner(KafkaPartitioner)} */ public KafkaRecordSerializationSchemaBuilder setPartitioner( FlinkKafkaPartitioner partitioner) { @@ -99,6 +100,19 @@ public KafkaRecordSerializationSchemaBuilder setPartitioner( return self; } + /** + * Sets a custom partitioner determining the target partition of the target topic. + * + * @param partitioner + * @return {@code this} + */ + public KafkaRecordSerializationSchemaBuilder setPartitioner( + KafkaPartitioner partitioner) { + KafkaRecordSerializationSchemaBuilder self = self(); + self.partitioner = checkNotNull(partitioner); + return self; + } + /** * Sets a fixed topic which used as destination for all records. * @@ -295,7 +309,7 @@ private static class KafkaRecordSerializationSchemaWrapper implements KafkaRecordSerializationSchema { private final SerializationSchema valueSerializationSchema; private final Function topicSelector; - private final FlinkKafkaPartitioner partitioner; + private final KafkaPartitioner partitioner; private final SerializationSchema keySerializationSchema; private final HeaderProvider headerProvider; @@ -303,7 +317,7 @@ private static class KafkaRecordSerializationSchemaWrapper Function topicSelector, SerializationSchema valueSerializationSchema, @Nullable SerializationSchema keySerializationSchema, - @Nullable FlinkKafkaPartitioner partitioner, + @Nullable KafkaPartitioner partitioner, @Nullable HeaderProvider headerProvider) { this.topicSelector = checkNotNull(topicSelector); this.valueSerializationSchema = checkNotNull(valueSerializationSchema); diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java index 7318ed697..9568349a3 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java @@ -18,43 +18,18 @@ package org.apache.flink.streaming.connectors.kafka.partitioner; import org.apache.flink.annotation.PublicEvolving; - -import java.io.Serializable; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; /** * A {@link FlinkKafkaPartitioner} wraps logic on how to partition records across partitions of * multiple Kafka topics. * - * @deprecated Will be turned into internal class when {@link - * org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer} is removed. + * @deprecated Use {@link KafkaPartitioner} instead for {@link + * org.apache.flink.connector.kafka.sink.KafkaSink}. */ @PublicEvolving @Deprecated -public abstract class FlinkKafkaPartitioner implements Serializable { +public abstract class FlinkKafkaPartitioner implements KafkaPartitioner { private static final long serialVersionUID = -9086719227828020494L; - - /** - * Initializer for the partitioner. This is called once on each parallel sink instance of the - * Flink Kafka producer. This method should be overridden if necessary. - * - * @param parallelInstanceId 0-indexed id of the parallel sink instance in Flink - * @param parallelInstances the total number of parallel instances - */ - public void open(int parallelInstanceId, int parallelInstances) { - // overwrite this method if needed. - } - - /** - * Determine the id of the partition that the record should be written to. - * - * @param record the record value - * @param key serialized key of the record - * @param value serialized value of the record - * @param targetTopic target topic for the record - * @param partitions found partitions for the target topic - * @return the id of the target partition - */ - public abstract int partition( - T record, byte[] key, byte[] value, String targetTopic, int[] partitions); } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java index f3a7acb3b..229b08b5f 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java @@ -19,9 +19,9 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; import org.apache.flink.connector.kafka.sink.KafkaSink; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.types.RowKind; @@ -46,7 +46,7 @@ class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationS private final Set topics; private final Pattern topicPattern; - private final FlinkKafkaPartitioner partitioner; + private final KafkaPartitioner partitioner; @Nullable private final SerializationSchema keySerialization; private final SerializationSchema valueSerialization; private final RowData.FieldGetter[] keyFieldGetters; @@ -59,7 +59,7 @@ class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationS DynamicKafkaRecordSerializationSchema( @Nullable List topics, @Nullable Pattern topicPattern, - @Nullable FlinkKafkaPartitioner partitioner, + @Nullable KafkaPartitioner partitioner, @Nullable SerializationSchema keySerialization, SerializationSchema valueSerialization, RowData.FieldGetter[] keyFieldGetters, diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java index f752276a3..5960a709a 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java @@ -24,11 +24,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ValueFieldsStrategy; @@ -386,7 +386,7 @@ public static Properties getKafkaProperties(Map tableOptions) { * The partitioner can be either "fixed", "round-robin" or a customized partitioner full class * name. */ - public static Optional> getFlinkKafkaPartitioner( + public static Optional> getFlinkKafkaPartitioner( ReadableConfig tableOptions, ClassLoader classLoader) { return tableOptions .getOptional(SINK_PARTITIONER) @@ -465,19 +465,19 @@ private static boolean hasKafkaClientProperties(Map tableOptions } /** Returns a class value with the given class name. */ - private static FlinkKafkaPartitioner initializePartitioner( + private static KafkaPartitioner initializePartitioner( String name, ClassLoader classLoader) { try { Class clazz = Class.forName(name, true, classLoader); - if (!FlinkKafkaPartitioner.class.isAssignableFrom(clazz)) { + if (!KafkaPartitioner.class.isAssignableFrom(clazz)) { throw new ValidationException( String.format( - "Sink partitioner class '%s' should extend from the required class %s", - name, FlinkKafkaPartitioner.class.getName())); + "Sink partitioner class '%s' should implement the required class %s", + name, KafkaPartitioner.class.getName())); } @SuppressWarnings("unchecked") - final FlinkKafkaPartitioner kafkaPartitioner = - InstantiationUtil.instantiate(name, FlinkKafkaPartitioner.class, classLoader); + final KafkaPartitioner kafkaPartitioner = + InstantiationUtil.instantiate(name, KafkaPartitioner.class, classLoader); return kafkaPartitioner; } catch (ClassNotFoundException | FlinkException e) { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java index 8ab0f10c6..2bb52c94d 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java @@ -24,11 +24,11 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; import org.apache.flink.connector.kafka.sink.KafkaSink; import org.apache.flink.connector.kafka.sink.KafkaSinkBuilder; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.Projection; @@ -125,7 +125,7 @@ public class KafkaDynamicSink implements DynamicTableSink, SupportsWritingMetada protected final Properties properties; /** Partitioner to select Kafka partition for each item. */ - protected final @Nullable FlinkKafkaPartitioner partitioner; + protected final @Nullable KafkaPartitioner partitioner; /** * Flag to determine sink mode. In upsert mode sink transforms the delete/update-before message @@ -150,7 +150,7 @@ public KafkaDynamicSink( @Nullable List topics, @Nullable Pattern topicPattern, Properties properties, - @Nullable FlinkKafkaPartitioner partitioner, + @Nullable KafkaPartitioner partitioner, DeliveryGuarantee deliveryGuarantee, boolean upsertMode, SinkBufferFlushMode flushMode, diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java index 7c23923b5..8124691a5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java @@ -26,11 +26,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; import org.apache.flink.connector.kafka.source.KafkaSourceOptions; import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.BoundedOptions; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ObjectIdentifier; @@ -427,7 +427,7 @@ protected KafkaDynamicSink createKafkaTableSink( @Nullable List topics, @Nullable Pattern topicPattern, Properties properties, - FlinkKafkaPartitioner partitioner, + KafkaPartitioner partitioner, DeliveryGuarantee deliveryGuarantee, Integer parallelism, @Nullable String transactionalIdPrefix) { From a87bf7268c9e3d69f56ba69e1e237fd47f91ebeb Mon Sep 17 00:00:00 2001 From: Danny Cranmer Date: Mon, 22 Apr 2024 14:24:33 +0100 Subject: [PATCH 291/322] [FLINK-35138][Connectors/Kafka] Drop support for Flink 1.17 --- .github/workflows/push_pr.yml | 8 +++----- .github/workflows/weekly.yml | 15 +++++++-------- pom.xml | 2 +- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 0f3da2ce6..951579406 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -28,11 +28,9 @@ jobs: compile_and_test: strategy: matrix: - flink: [ 1.17.2 ] - jdk: [ '8, 11' ] + flink: [ 1.18.1 ] + jdk: [ '8, 11, 17' ] include: - - flink: 1.18.1 - jdk: '8, 11, 17' - flink: 1.19.0 jdk: '8, 11, 17, 21' uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils @@ -42,7 +40,7 @@ jobs: python_test: strategy: matrix: - flink: [ 1.17.2, 1.18.1, 1.19.0 ] + flink: [ 1.18.1, 1.19.0 ] uses: apache/flink-connector-shared-utils/.github/workflows/python_ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} \ No newline at end of file diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index 7ee0f419b..1caecd5c9 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -30,11 +30,7 @@ jobs: strategy: matrix: flink_branches: [{ - flink: 1.17-SNAPSHOT, - branch: main - }, { flink: 1.18-SNAPSHOT, - jdk: '8, 11, 17', branch: main }, { flink: 1.19-SNAPSHOT, @@ -45,11 +41,14 @@ jobs: jdk: '8, 11, 17, 21', branch: main }, { - flink: 1.17.2, - branch: v3.1 + flink: 1.18.1, + branch: v3.2 + }, { + flink: 1.19.0, + branch: v3.2, + jdk: '8, 11, 17, 21', }, { flink: 1.18.1, - jdk: '8, 11, 17', branch: v3.1 }, { flink: 1.19.0, @@ -60,5 +59,5 @@ jobs: with: flink_version: ${{ matrix.flink_branches.flink }} connector_branch: ${{ matrix.flink_branches.branch }} - jdk_version: ${{ matrix.flink_branches.jdk || '8, 11' }} + jdk_version: ${{ matrix.flink_branches.jdk || '8, 11, 17' }} run_dependency_convergence: false \ No newline at end of file diff --git a/pom.xml b/pom.xml index 9215540fd..a86da54d7 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,7 @@ under the License. - 1.17.0 + 1.18.0 3.4.0 7.4.4 From ac8fa12a38dd9e34221952764f4d7bf92bf0a3ce Mon Sep 17 00:00:00 2001 From: Aleksandr Pilipenko Date: Thu, 3 Oct 2024 18:33:26 +0100 Subject: [PATCH 292/322] [FLINK-36422][docs] Fix PyFlink jar download link --- docs/content.zh/docs/connectors/datastream/kafka.md | 2 +- docs/content/docs/connectors/datastream/kafka.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/connectors/datastream/kafka.md b/docs/content.zh/docs/connectors/datastream/kafka.md index ad531d72b..4a90ece31 100644 --- a/docs/content.zh/docs/connectors/datastream/kafka.md +++ b/docs/content.zh/docs/connectors/datastream/kafka.md @@ -45,7 +45,7 @@ Apache Flink 集成了通用的 Kafka 连接器,它会尽力与 Kafka client Flink 目前的流连接器还不是二进制发行版的一部分。 [在此处]({{< ref "docs/dev/configuration/overview" >}})可以了解到如何链接它们,从而在集群中运行。 -{{< py_download_link "kafka" >}} +{{< py_connector_download_link "kafka" >}} ## Kafka Source {{< hint info >}} diff --git a/docs/content/docs/connectors/datastream/kafka.md b/docs/content/docs/connectors/datastream/kafka.md index 1de700bc1..bddcefec9 100644 --- a/docs/content/docs/connectors/datastream/kafka.md +++ b/docs/content/docs/connectors/datastream/kafka.md @@ -41,7 +41,7 @@ For details on Kafka compatibility, please refer to the official [Kafka document Flink's streaming connectors are not part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). -{{< py_download_link "kafka" >}} +{{< py_connector_download_link "kafka" >}} ## Kafka Source {{< hint info >}} From 3fbb68d148a458bb2cbfdb92c70e7ef48677af9a Mon Sep 17 00:00:00 2001 From: Danny Cranmer Date: Thu, 3 Oct 2024 12:11:54 +0100 Subject: [PATCH 293/322] [FLINK-36420][Connectors/Kafka] Bump avro from 1.11.3 to 1.11.4 --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index a86da54d7..aeb1a4ec2 100644 --- a/pom.xml +++ b/pom.xml @@ -68,8 +68,8 @@ under the License. 2.12.7 2.12.7 1.1.10.5 - 1.11.3 - 32.1.2-jre + 1.11.4 + 32.1.2-jre false 1.17.0 From 264ce8c52cf9f4e19809d530eff7f9c546b25220 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 8 Oct 2024 15:55:58 +0200 Subject: [PATCH 294/322] [hotfix] Copy Flink .editorconfig We have the same code standard (enforced through spotless) and we should have the same IDE configurations. --- .editorconfig | 275 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 275 insertions(+) create mode 100644 .editorconfig diff --git a/.editorconfig b/.editorconfig new file mode 100644 index 000000000..5aa9a33cc --- /dev/null +++ b/.editorconfig @@ -0,0 +1,275 @@ +root = true + +[*] +charset = utf-8 +end_of_line = lf +insert_final_newline = true +max_line_length = 100 +# ij_formatter_off_tag = @formatter:off +# ij_formatter_on_tag = @formatter:on +# ij_formatter_tags_enabled = false +# ij_smart_tabs = false +# ij_wrap_on_typing = false + +[*.java] +indent_size = 4 +indent_style = space +tab_width = 4 +ij_continuation_indent_size = 8 +# ij_java_align_consecutive_assignments = false +# ij_java_align_consecutive_variable_declarations = false +# ij_java_align_group_field_declarations = false +# ij_java_align_multiline_annotation_parameters = false +# ij_java_align_multiline_array_initializer_expression = false +# ij_java_align_multiline_assignment = false +# ij_java_align_multiline_binary_operation = false +# ij_java_align_multiline_chained_methods = false +# ij_java_align_multiline_extends_list = false +# ij_java_align_multiline_for = true +# ij_java_align_multiline_method_parentheses = false +# ij_java_align_multiline_parameters = true +# ij_java_align_multiline_parameters_in_calls = false +# ij_java_align_multiline_parenthesized_expression = false +# ij_java_align_multiline_records = true +# ij_java_align_multiline_resources = true +# ij_java_align_multiline_ternary_operation = false +# ij_java_align_multiline_text_blocks = false +# ij_java_align_multiline_throws_list = false +# ij_java_align_subsequent_simple_methods = false +# ij_java_align_throws_keyword = false +# ij_java_annotation_parameter_wrap = off +# ij_java_array_initializer_new_line_after_left_brace = false +# ij_java_array_initializer_right_brace_on_new_line = false +# ij_java_array_initializer_wrap = off +# ij_java_assert_statement_colon_on_next_line = false +# ij_java_assert_statement_wrap = off +# ij_java_assignment_wrap = off +ij_java_binary_operation_sign_on_next_line = true +ij_java_binary_operation_wrap = normal +# ij_java_blank_lines_after_anonymous_class_header = 0 +# ij_java_blank_lines_after_class_header = 0 +# ij_java_blank_lines_after_imports = 1 +# ij_java_blank_lines_after_package = 1 +# ij_java_blank_lines_around_class = 1 +# ij_java_blank_lines_around_field = 0 +# ij_java_blank_lines_around_field_in_interface = 0 +# ij_java_blank_lines_around_initializer = 1 +# ij_java_blank_lines_around_method = 1 +# ij_java_blank_lines_around_method_in_interface = 1 +# ij_java_blank_lines_before_class_end = 0 +# ij_java_blank_lines_before_imports = 1 +# ij_java_blank_lines_before_method_body = 0 +# ij_java_blank_lines_before_package = 0 +# ij_java_block_brace_style = end_of_line +# ij_java_block_comment_at_first_column = true +ij_java_call_parameters_new_line_after_left_paren = true +# ij_java_call_parameters_right_paren_on_new_line = false +ij_java_call_parameters_wrap = on_every_item +# ij_java_case_statement_on_separate_line = true +# ij_java_catch_on_new_line = false +# ij_java_class_annotation_wrap = split_into_lines +# ij_java_class_brace_style = end_of_line +ij_java_class_count_to_use_import_on_demand = 9999 +# ij_java_class_names_in_javadoc = 1 +# ij_java_do_not_indent_top_level_class_members = false +# ij_java_do_not_wrap_after_single_annotation = false +# ij_java_do_while_brace_force = never +# ij_java_doc_add_blank_line_after_description = true +ij_java_doc_add_blank_line_after_param_comments = true +ij_java_doc_add_blank_line_after_return = true +# ij_java_doc_add_p_tag_on_empty_lines = true +ij_java_doc_align_exception_comments = false +ij_java_doc_align_param_comments = false +ij_java_doc_do_not_wrap_if_one_line = true +ij_java_doc_enable_formatting = true +# ij_java_doc_enable_leading_asterisks = true +ij_java_doc_indent_on_continuation = true +ij_java_doc_keep_empty_lines = true +# ij_java_doc_keep_empty_parameter_tag = true +# ij_java_doc_keep_empty_return_tag = true +# ij_java_doc_keep_empty_throws_tag = true +# ij_java_doc_keep_invalid_tags = true +# ij_java_doc_param_description_on_new_line = false +ij_java_doc_preserve_line_breaks = false +# ij_java_doc_use_throws_not_exception_tag = true +# ij_java_else_on_new_line = false +# ij_java_entity_dd_suffix = EJB +# ij_java_entity_eb_suffix = Bean +# ij_java_entity_hi_suffix = Home +# ij_java_entity_lhi_prefix = Local +# ij_java_entity_lhi_suffix = Home +# ij_java_entity_li_prefix = Local +# ij_java_entity_pk_class = java.lang.String +# ij_java_entity_vo_suffix = VO +# ij_java_enum_constants_wrap = off +# ij_java_extends_keyword_wrap = off +# ij_java_extends_list_wrap = off +# ij_java_field_annotation_wrap = split_into_lines +# ij_java_finally_on_new_line = false +# ij_java_for_brace_force = never +# ij_java_for_statement_new_line_after_left_paren = false +# ij_java_for_statement_right_paren_on_new_line = false +# ij_java_for_statement_wrap = off +# ij_java_generate_final_locals = false +# ij_java_generate_final_parameters = false +# ij_java_if_brace_force = never +ij_java_imports_layout = org.apache.flink.**,|,org.apache.flink.shaded.**,|,*,|,javax.**,|,java.**,|,scala.**,|,$* +# ij_java_indent_case_from_switch = true +# ij_java_insert_inner_class_imports = false +# ij_java_insert_override_annotation = true +# ij_java_keep_blank_lines_before_right_brace = 2 +# ij_java_keep_blank_lines_between_package_declaration_and_header = 2 +# ij_java_keep_blank_lines_in_code = 2 +# ij_java_keep_blank_lines_in_declarations = 2 +# ij_java_keep_control_statement_in_one_line = true +# ij_java_keep_first_column_comment = true +# ij_java_keep_indents_on_empty_lines = false +# ij_java_keep_line_breaks = true +# ij_java_keep_multiple_expressions_in_one_line = false +# ij_java_keep_simple_blocks_in_one_line = false +# ij_java_keep_simple_classes_in_one_line = false +# ij_java_keep_simple_lambdas_in_one_line = false +# ij_java_keep_simple_methods_in_one_line = false +# ij_java_label_indent_absolute = false +# ij_java_label_indent_size = 0 +# ij_java_lambda_brace_style = end_of_line +ij_java_layout_static_imports_separately = true +# ij_java_line_comment_add_space = false +# ij_java_line_comment_at_first_column = true +# ij_java_message_dd_suffix = EJB +# ij_java_message_eb_suffix = Bean +# ij_java_method_annotation_wrap = split_into_lines +# ij_java_method_brace_style = end_of_line +ij_java_method_call_chain_wrap = on_every_item +ij_java_method_parameters_new_line_after_left_paren = true +# ij_java_method_parameters_right_paren_on_new_line = false +ij_java_method_parameters_wrap = on_every_item +# ij_java_modifier_list_wrap = false +ij_java_names_count_to_use_import_on_demand = 9999 +# ij_java_new_line_after_lparen_in_record_header = false +# ij_java_packages_to_use_import_on_demand = java.awt.*,javax.swing.* +# ij_java_parameter_annotation_wrap = off +# ij_java_parentheses_expression_new_line_after_left_paren = false +# ij_java_parentheses_expression_right_paren_on_new_line = false +# ij_java_place_assignment_sign_on_next_line = false +# ij_java_prefer_longer_names = true +# ij_java_prefer_parameters_wrap = false +# ij_java_record_components_wrap = normal +# ij_java_repeat_synchronized = true +# ij_java_replace_instanceof_and_cast = false +# ij_java_replace_null_check = true +# ij_java_replace_sum_lambda_with_method_ref = true +# ij_java_resource_list_new_line_after_left_paren = false +# ij_java_resource_list_right_paren_on_new_line = false +# ij_java_resource_list_wrap = off +# ij_java_rparen_on_new_line_in_record_header = false +# ij_java_session_dd_suffix = EJB +# ij_java_session_eb_suffix = Bean +# ij_java_session_hi_suffix = Home +# ij_java_session_lhi_prefix = Local +# ij_java_session_lhi_suffix = Home +# ij_java_session_li_prefix = Local +# ij_java_session_si_suffix = Service +# ij_java_space_after_closing_angle_bracket_in_type_argument = false +# ij_java_space_after_colon = true +# ij_java_space_after_comma = true +# ij_java_space_after_comma_in_type_arguments = true +# ij_java_space_after_for_semicolon = true +# ij_java_space_after_quest = true +# ij_java_space_after_type_cast = true +# ij_java_space_before_annotation_array_initializer_left_brace = false +# ij_java_space_before_annotation_parameter_list = false +# ij_java_space_before_array_initializer_left_brace = false +# ij_java_space_before_catch_keyword = true +# ij_java_space_before_catch_left_brace = true +# ij_java_space_before_catch_parentheses = true +# ij_java_space_before_class_left_brace = true +# ij_java_space_before_colon = true +# ij_java_space_before_colon_in_foreach = true +# ij_java_space_before_comma = false +# ij_java_space_before_do_left_brace = true +# ij_java_space_before_else_keyword = true +# ij_java_space_before_else_left_brace = true +# ij_java_space_before_finally_keyword = true +# ij_java_space_before_finally_left_brace = true +# ij_java_space_before_for_left_brace = true +# ij_java_space_before_for_parentheses = true +# ij_java_space_before_for_semicolon = false +# ij_java_space_before_if_left_brace = true +# ij_java_space_before_if_parentheses = true +# ij_java_space_before_method_call_parentheses = false +# ij_java_space_before_method_left_brace = true +# ij_java_space_before_method_parentheses = false +# ij_java_space_before_opening_angle_bracket_in_type_parameter = false +# ij_java_space_before_quest = true +# ij_java_space_before_switch_left_brace = true +# ij_java_space_before_switch_parentheses = true +# ij_java_space_before_synchronized_left_brace = true +# ij_java_space_before_synchronized_parentheses = true +# ij_java_space_before_try_left_brace = true +# ij_java_space_before_try_parentheses = true +# ij_java_space_before_type_parameter_list = false +# ij_java_space_before_while_keyword = true +# ij_java_space_before_while_left_brace = true +# ij_java_space_before_while_parentheses = true +# ij_java_space_inside_one_line_enum_braces = false +# ij_java_space_within_empty_array_initializer_braces = false +# ij_java_space_within_empty_method_call_parentheses = false +# ij_java_space_within_empty_method_parentheses = false +# ij_java_spaces_around_additive_operators = true +# ij_java_spaces_around_assignment_operators = true +# ij_java_spaces_around_bitwise_operators = true +# ij_java_spaces_around_equality_operators = true +# ij_java_spaces_around_lambda_arrow = true +# ij_java_spaces_around_logical_operators = true +# ij_java_spaces_around_method_ref_dbl_colon = false +# ij_java_spaces_around_multiplicative_operators = true +# ij_java_spaces_around_relational_operators = true +# ij_java_spaces_around_shift_operators = true +# ij_java_spaces_around_type_bounds_in_type_parameters = true +# ij_java_spaces_around_unary_operator = false +# ij_java_spaces_within_angle_brackets = false +# ij_java_spaces_within_annotation_parentheses = false +# ij_java_spaces_within_array_initializer_braces = false +# ij_java_spaces_within_braces = false +# ij_java_spaces_within_brackets = false +# ij_java_spaces_within_cast_parentheses = false +# ij_java_spaces_within_catch_parentheses = false +# ij_java_spaces_within_for_parentheses = false +# ij_java_spaces_within_if_parentheses = false +# ij_java_spaces_within_method_call_parentheses = false +# ij_java_spaces_within_method_parentheses = false +# ij_java_spaces_within_parentheses = false +# ij_java_spaces_within_switch_parentheses = false +# ij_java_spaces_within_synchronized_parentheses = false +# ij_java_spaces_within_try_parentheses = false +# ij_java_spaces_within_while_parentheses = false +# ij_java_special_else_if_treatment = true +# ij_java_subclass_name_suffix = Impl +# ij_java_ternary_operation_signs_on_next_line = false +# ij_java_ternary_operation_wrap = off +# ij_java_test_name_suffix = Test +# ij_java_throws_keyword_wrap = off +# ij_java_throws_list_wrap = off +# ij_java_use_external_annotations = false +# ij_java_use_fq_class_names = false +# ij_java_use_relative_indents = false +# ij_java_use_single_class_imports = true +ij_java_variable_annotation_wrap = normal +# ij_java_visibility = public +# ij_java_while_brace_force = never +# ij_java_while_on_new_line = false +# ij_java_wrap_comments = false +ij_java_wrap_first_method_in_call_chain = true +# ij_java_wrap_long_lines = false + +[*.out] +insert_final_newline = false + +[*.xml] +indent_style = tab +indent_size = 4 + +[*.py] +indent_style = space +indent_size = 4 From a73810418ae0c0990a3b03c6700ec21d24c49839 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 2 Oct 2024 13:43:36 +0200 Subject: [PATCH 295/322] [FLINK-36441] Show full stacktraces in tests for failures Make assertj print full stacktraces when encountering unexpected exceptions. --- .../flink/KafkaAssertjConfiguration.java | 30 +++++++++++++++++++ ...g.assertj.core.configuration.Configuration | 19 ++++++++++++ 2 files changed, 49 insertions(+) create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/KafkaAssertjConfiguration.java create mode 100644 flink-connector-kafka/src/test/resources/META-INF/services/org.assertj.core.configuration.Configuration diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/KafkaAssertjConfiguration.java b/flink-connector-kafka/src/test/java/org/apache/flink/KafkaAssertjConfiguration.java new file mode 100644 index 000000000..26d080484 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/KafkaAssertjConfiguration.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink; + +import org.assertj.core.configuration.Configuration; + +/** Configuration for AssertJ that shows full stack traces for unmatched exceptions. */ +public class KafkaAssertjConfiguration extends Configuration { + public KafkaAssertjConfiguration() { + // in case of an assertion error, show the full stack trace + // for green builds, this is not changing anything + setMaxStackTraceElementsDisplayed(10000); + } +} diff --git a/flink-connector-kafka/src/test/resources/META-INF/services/org.assertj.core.configuration.Configuration b/flink-connector-kafka/src/test/resources/META-INF/services/org.assertj.core.configuration.Configuration new file mode 100644 index 000000000..e3112c94e --- /dev/null +++ b/flink-connector-kafka/src/test/resources/META-INF/services/org.assertj.core.configuration.Configuration @@ -0,0 +1,19 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +org.apache.flink.KafkaAssertjConfiguration From ab333a48efd527bdb60a2c04f6477882308f925a Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 2 Oct 2024 23:36:07 +0200 Subject: [PATCH 296/322] [FLINK-36441] Ignore exceptions during test cleanup Job may be at various states including race conditions during shutdown. Ideally, the framework would provide idempotence but we can workaround that by ignoring specific exceptions. --- .../kafka/table/KafkaTableITCase.java | 87 ++++++++++++------- 1 file changed, 58 insertions(+), 29 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java index 8630120b2..acd0550e4 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java @@ -23,8 +23,8 @@ import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.core.testutils.FlinkAssertions; import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; +import org.apache.flink.runtime.messages.FlinkJobTerminatedWithoutCancellationException; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; @@ -36,11 +36,14 @@ import org.apache.flink.table.utils.EncodingUtils; import org.apache.flink.test.util.SuccessException; import org.apache.flink.types.Row; +import org.apache.flink.util.function.RunnableWithException; import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.assertj.core.api.Assertions; +import org.assertj.core.api.ThrowingConsumer; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -65,6 +68,7 @@ import java.util.stream.IntStream; import static org.apache.flink.core.testutils.CommonTestUtils.waitUtil; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectAllRows; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectRows; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.readLines; @@ -186,7 +190,7 @@ public void testKafkaSourceSink() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -266,8 +270,8 @@ public void testKafkaSourceSinkWithTopicList() throws Exception { assertThat(topic2Results).containsExactly(Row.of(topic2, 2, 1103, "behavior 2")); // ------------- cleanup ------------------- - deleteTestTopic(topic1); - deleteTestTopic(topic2); + cleanupTopic(topic1); + cleanupTopic(topic2); } @Test @@ -349,8 +353,8 @@ public void testKafkaSourceSinkWithTopicPattern() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic1); - deleteTestTopic(topic2); + cleanupTopic(topic1); + cleanupTopic(topic2); } @Test @@ -406,7 +410,7 @@ public void testKafkaSourceEmptyResultOnDeletedOffsets() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -460,7 +464,7 @@ public void testKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -517,7 +521,7 @@ public void testKafkaSourceSinkWithBoundedTimestamp() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -702,7 +706,7 @@ public void testKafkaSourceSinkWithMetadata() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -783,7 +787,7 @@ public void testKafkaSourceSinkWithKeyAndPartialValue() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -861,7 +865,7 @@ public void testKafkaSourceSinkWithKeyAndFullValue() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -977,8 +981,8 @@ public void testKafkaTemporalJoinChangelog() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(orderTopic); - deleteTestTopic(productTopic); + cleanupTopic(orderTopic); + cleanupTopic(productTopic); } private void initialProductChangelog(String topic, String bootstraps) throws Exception { @@ -1093,8 +1097,8 @@ public void testPerPartitionWatermarkKafka() throws Exception { // ------------- cleanup ------------------- - tableResult.getJobClient().ifPresent(JobClient::cancel); - deleteTestTopic(topic); + cancelJob(tableResult); + cleanupTopic(topic); } @Test @@ -1168,8 +1172,8 @@ public void testPerPartitionWatermarkWithIdleSource() throws Exception { // ------------- cleanup ------------------- - tableResult.getJobClient().ifPresent(JobClient::cancel); - deleteTestTopic(topic); + cancelJob(tableResult); + cleanupTopic(topic); } @Test @@ -1300,8 +1304,8 @@ public void testLatestOffsetStrategyResume() throws Exception { // ------------- cleanup ------------------- - tableResult.getJobClient().ifPresent(JobClient::cancel); - deleteTestTopic(topic); + cancelJob(tableResult); + cleanupTopic(topic); } @Test @@ -1317,7 +1321,7 @@ public void testStartFromGroupOffsetsEarliest() throws Exception { @Test public void testStartFromGroupOffsetsNone() { Assertions.assertThatThrownBy(() -> testStartFromGroupOffsetsWithNoneResetStrategy()) - .satisfies(FlinkAssertions.anyCauseMatches(NoOffsetForPartitionException.class)); + .satisfies(anyCauseMatches(NoOffsetForPartitionException.class)); } private List appendNewData( @@ -1433,10 +1437,8 @@ private void testStartFromGroupOffsets(String resetStrategy) throws Exception { KafkaTableTestUtils.waitingExpectedResults(sinkName, expected, Duration.ofSeconds(15)); } finally { // ------------- cleanup ------------------- - if (tableResult != null) { - tableResult.getJobClient().ifPresent(JobClient::cancel); - } - deleteTestTopic(topic); + cancelJob(tableResult); + cleanupTopic(topic); } } @@ -1455,10 +1457,8 @@ private void testStartFromGroupOffsetsWithNoneResetStrategy() tableResult.await(); } finally { // ------------- cleanup ------------------- - if (tableResult != null) { - tableResult.getJobClient().ifPresent(JobClient::cancel); - } - deleteTestTopic(topic); + cancelJob(tableResult); + cleanupTopic(topic); } } @@ -1514,4 +1514,33 @@ private static boolean isCausedByJobFinished(Throwable e) { return false; } } + + private void cleanupTopic(String topic) { + ignoreExceptions( + () -> deleteTestTopic(topic), + anyCauseMatches(UnknownTopicOrPartitionException.class)); + } + + private static void cancelJob(TableResult tableResult) { + if (tableResult != null && tableResult.getJobClient().isPresent()) { + ignoreExceptions( + () -> tableResult.getJobClient().get().cancel().get(), + anyCauseMatches(FlinkJobTerminatedWithoutCancellationException.class), + anyCauseMatches( + "MiniCluster is not yet running or has already been shut down.")); + } + } + + @SafeVarargs + private static void ignoreExceptions( + RunnableWithException runnable, ThrowingConsumer... ignoreIf) { + try { + runnable.run(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (Exception ex) { + // check if the exception is one of the ignored ones + assertThat(ex).satisfiesAnyOf(ignoreIf); + } + } } From 8472d0353cbc83bd70aeba59811533d0dd261230 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 8 Oct 2024 13:54:10 +0200 Subject: [PATCH 297/322] [FLINK-36441] Fix leak for non-transactional FlinkKafkaProducer. For non-transactional producers, a notifyCheckpointCompleted after finishOperator will set the transaction inside the 2PCSinkFunction to null, such that on close, the producer is leaked. Since transactional producer stores the transactions in pendingTransactions before that, we just need to fix the cases where we don't preCommit/commit. The easiest solution is to actually close the producer on finishOperator - no new record can arrive. --- .../connectors/kafka/FlinkKafkaProducer.java | 243 ++++++++---------- .../kafka/FlinkKafkaProducerITCase.java | 26 +- 2 files changed, 116 insertions(+), 153 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java index d6cbe2e7d..4b39749d9 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java @@ -29,10 +29,12 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kafka.sink.KafkaSink; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.metrics.MetricGroup; @@ -49,7 +51,6 @@ import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.NetUtils; -import org.apache.flink.util.Preconditions; import org.apache.flink.util.TemporaryClassLoaderContext; import org.apache.commons.lang3.StringUtils; @@ -70,6 +71,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.io.ObjectInputStream; import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -91,10 +93,10 @@ /** * Flink Sink to produce data into a Kafka topic. By default producer will use {@link - * FlinkKafkaProducer.Semantic#AT_LEAST_ONCE} semantic. Before using {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE} please refer to Flink's Kafka connector documentation. + * Semantic#AT_LEAST_ONCE} semantic. Before using {@link Semantic#EXACTLY_ONCE} please refer to + * Flink's Kafka connector documentation. * - * @deprecated Please use {@link org.apache.flink.connector.kafka.sink.KafkaSink}. + * @deprecated Please use {@link KafkaSink}. */ @Deprecated @PublicEvolving @@ -173,10 +175,7 @@ public enum Semantic { */ public static final int SAFE_SCALE_DOWN_FACTOR = 5; - /** - * Default number of KafkaProducers in the pool. See {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}. - */ + /** Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}. */ public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5; /** Default value for kafka transaction timeout. */ @@ -191,27 +190,26 @@ public enum Semantic { * NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR_V2. */ @Deprecated - private static final ListStateDescriptor + private static final ListStateDescriptor NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR = new ListStateDescriptor<>( "next-transactional-id-hint", TypeInformation.of(NextTransactionalIdHint.class)); - private static final ListStateDescriptor + private static final ListStateDescriptor NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR_V2 = new ListStateDescriptor<>( "next-transactional-id-hint-v2", new NextTransactionalIdHintSerializer()); /** State for nextTransactionalIdHint. */ - private transient ListState - nextTransactionalIdHintState; + private transient ListState nextTransactionalIdHintState; /** Generator for Transactional IDs. */ private transient TransactionalIdsGenerator transactionalIdsGenerator; /** Hint for picking next transactional id. */ - private transient FlinkKafkaProducer.NextTransactionalIdHint nextTransactionalIdHint; + private transient NextTransactionalIdHint nextTransactionalIdHint; /** User defined properties for the Producer. */ protected final Properties producerConfig; @@ -256,7 +254,7 @@ public enum Semantic { private boolean logFailuresOnly; /** Semantic chosen for this instance. */ - protected FlinkKafkaProducer.Semantic semantic; + protected Semantic semantic; // -------------------------------- Runtime fields ------------------------------------------ @@ -360,17 +358,16 @@ public FlinkKafkaProducer( * @param customPartitioner A serializable partitioner for assigning messages to Kafka * partitions. If a partitioner is not provided, records will be distributed to Kafka * partitions in a round-robin fashion. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}). + * Semantic#EXACTLY_ONCE}). */ public FlinkKafkaProducer( String topicId, SerializationSchema serializationSchema, Properties producerConfig, @Nullable FlinkKafkaPartitioner customPartitioner, - FlinkKafkaProducer.Semantic semantic, + Semantic semantic, int kafkaProducersPoolSize) { this( topicId, @@ -399,7 +396,7 @@ public FlinkKafkaProducer( * @param topicId ID of the Kafka topic. * @param serializationSchema User defined serialization schema supporting key/value messages * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties, - * FlinkKafkaProducer.Semantic)} + * Semantic)} */ @Deprecated public FlinkKafkaProducer( @@ -425,7 +422,7 @@ public FlinkKafkaProducer( * @param serializationSchema User defined serialization schema supporting key/value messages * @param producerConfig Properties with the producer configuration. * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties, - * FlinkKafkaProducer.Semantic)} + * Semantic)} */ @Deprecated public FlinkKafkaProducer( @@ -449,17 +446,16 @@ public FlinkKafkaProducer( * @param topicId ID of the Kafka topic. * @param serializationSchema User defined serialization schema supporting key/value messages * @param producerConfig Properties with the producer configuration. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties, - * FlinkKafkaProducer.Semantic)} + * Semantic)} */ @Deprecated public FlinkKafkaProducer( String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, - FlinkKafkaProducer.Semantic semantic) { + Semantic semantic) { this( topicId, serializationSchema, @@ -491,7 +487,7 @@ public FlinkKafkaProducer( * keys are {@code null}, then records will be distributed to Kafka partitions in a * round-robin fashion. * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties, - * FlinkKafkaProducer.Semantic)} + * Semantic)} */ @Deprecated public FlinkKafkaProducer( @@ -504,7 +500,7 @@ public FlinkKafkaProducer( serializationSchema, producerConfig, customPartitioner, - FlinkKafkaProducer.Semantic.AT_LEAST_ONCE, + Semantic.AT_LEAST_ONCE, DEFAULT_KAFKA_PRODUCERS_POOL_SIZE); } @@ -529,12 +525,11 @@ public FlinkKafkaProducer( * each record (determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If the * keys are {@code null}, then records will be distributed to Kafka partitions in a * round-robin fashion. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}). + * Semantic#EXACTLY_ONCE}). * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties, - * FlinkKafkaProducer.Semantic)} + * Semantic)} */ @Deprecated public FlinkKafkaProducer( @@ -542,7 +537,7 @@ public FlinkKafkaProducer( KeyedSerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner, - FlinkKafkaProducer.Semantic semantic, + Semantic semantic, int kafkaProducersPoolSize) { this( defaultTopicId, @@ -564,14 +559,13 @@ public FlinkKafkaProducer( * a kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is * the only required argument. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). */ public FlinkKafkaProducer( String defaultTopic, KafkaSerializationSchema serializationSchema, Properties producerConfig, - FlinkKafkaProducer.Semantic semantic) { + Semantic semantic) { this( defaultTopic, serializationSchema, @@ -590,16 +584,15 @@ public FlinkKafkaProducer( * a kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is * the only required argument. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}). + * Semantic#EXACTLY_ONCE}). */ public FlinkKafkaProducer( String defaultTopic, KafkaSerializationSchema serializationSchema, Properties producerConfig, - FlinkKafkaProducer.Semantic semantic, + Semantic semantic, int kafkaProducersPoolSize) { this( defaultTopic, @@ -634,10 +627,9 @@ public FlinkKafkaProducer( * kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is * the only required argument. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}). + * Semantic#EXACTLY_ONCE}). */ private FlinkKafkaProducer( String defaultTopic, @@ -645,11 +637,9 @@ private FlinkKafkaProducer( FlinkKafkaPartitioner customPartitioner, KafkaSerializationSchema kafkaSchema, Properties producerConfig, - FlinkKafkaProducer.Semantic semantic, + Semantic semantic, int kafkaProducersPoolSize) { - super( - new FlinkKafkaProducer.TransactionStateSerializer(), - new FlinkKafkaProducer.ContextStateSerializer()); + super(new TransactionStateSerializer(), new ContextStateSerializer()); this.defaultTopicId = checkNotNull(defaultTopic, "defaultTopic is null"); @@ -729,7 +719,7 @@ private FlinkKafkaProducer( // Enable transactionTimeoutWarnings to avoid silent data loss // See KAFKA-6119 (affects versions 0.11.0.0 and 0.11.0.1): // The KafkaProducer may not throw an exception if the transaction failed to commit - if (semantic == FlinkKafkaProducer.Semantic.EXACTLY_ONCE) { + if (semantic == Semantic.EXACTLY_ONCE) { final long transactionTimeout = getTransactionTimeout(producerConfig); super.setTransactionTimeout(transactionTimeout); super.enableTransactionTimeoutWarnings(0.8); @@ -780,7 +770,7 @@ public void setLogFailuresOnly(boolean logFailuresOnly) { * @throws NullPointerException Thrown, if the transactionalIdPrefix was null. */ public void setTransactionalIdPrefix(String transactionalIdPrefix) { - this.transactionalIdPrefix = Preconditions.checkNotNull(transactionalIdPrefix); + this.transactionalIdPrefix = checkNotNull(transactionalIdPrefix); } /** @@ -853,8 +843,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } @Override - public void invoke( - FlinkKafkaProducer.KafkaTransactionState transaction, IN next, Context context) + public void invoke(KafkaTransactionState transaction, IN next, Context context) throws FlinkKafkaException { checkErroneous(); @@ -929,6 +918,10 @@ public void close() throws FlinkKafkaException { // First close the producer for current transaction. try { final KafkaTransactionState currentTransaction = currentTransaction(); + LOG.error( + "Closing producer for current transaction: {} {}", + currentTransaction, + currentTransaction != null ? currentTransaction.producer : null); if (currentTransaction != null) { // to avoid exceptions on aborting transactions with some pending records flush(currentTransaction); @@ -978,33 +971,27 @@ public void close() throws FlinkKafkaException { // ------------------- Logic for handling checkpoint flushing -------------------------- // @Override - protected FlinkKafkaProducer.KafkaTransactionState beginTransaction() - throws FlinkKafkaException { + protected KafkaTransactionState beginTransaction() throws FlinkKafkaException { switch (semantic) { case EXACTLY_ONCE: FlinkKafkaInternalProducer producer = createTransactionalProducer(); producer.beginTransaction(); - return new FlinkKafkaProducer.KafkaTransactionState( - producer.getTransactionalId(), producer); + return new KafkaTransactionState(producer.getTransactionalId(), producer); case AT_LEAST_ONCE: case NONE: // Do not create new producer on each beginTransaction() if it is not necessary - final FlinkKafkaProducer.KafkaTransactionState currentTransaction = - currentTransaction(); + final KafkaTransactionState currentTransaction = currentTransaction(); if (currentTransaction != null && currentTransaction.producer != null) { - return new FlinkKafkaProducer.KafkaTransactionState( - currentTransaction.producer); + return new KafkaTransactionState(currentTransaction.producer); } - return new FlinkKafkaProducer.KafkaTransactionState( - initNonTransactionalProducer(true)); + return new KafkaTransactionState(initNonTransactionalProducer(true)); default: throw new UnsupportedOperationException("Not implemented semantic"); } } @Override - protected void preCommit(FlinkKafkaProducer.KafkaTransactionState transaction) - throws FlinkKafkaException { + protected void preCommit(KafkaTransactionState transaction) throws FlinkKafkaException { switch (semantic) { case EXACTLY_ONCE: case AT_LEAST_ONCE: @@ -1019,7 +1006,7 @@ protected void preCommit(FlinkKafkaProducer.KafkaTransactionState transaction) } @Override - protected void commit(FlinkKafkaProducer.KafkaTransactionState transaction) { + protected void commit(KafkaTransactionState transaction) { if (transaction.isTransactional()) { try { transaction.producer.commitTransaction(); @@ -1030,7 +1017,7 @@ protected void commit(FlinkKafkaProducer.KafkaTransactionState transaction) { } @Override - protected void recoverAndCommit(FlinkKafkaProducer.KafkaTransactionState transaction) { + protected void recoverAndCommit(KafkaTransactionState transaction) { if (transaction.isTransactional()) { FlinkKafkaInternalProducer producer = null; try { @@ -1063,7 +1050,7 @@ protected void recoverAndCommit(FlinkKafkaProducer.KafkaTransactionState transac } @Override - protected void abort(FlinkKafkaProducer.KafkaTransactionState transaction) { + protected void abort(KafkaTransactionState transaction) { if (transaction.isTransactional()) { transaction.producer.abortTransaction(); recycleTransactionalProducer(transaction.producer); @@ -1071,7 +1058,7 @@ protected void abort(FlinkKafkaProducer.KafkaTransactionState transaction) { } @Override - protected void recoverAndAbort(FlinkKafkaProducer.KafkaTransactionState transaction) { + protected void recoverAndAbort(KafkaTransactionState transaction) { if (transaction.isTransactional()) { FlinkKafkaInternalProducer producer = null; try { @@ -1099,8 +1086,7 @@ protected void acknowledgeMessage() { * * @param transaction */ - private void flush(FlinkKafkaProducer.KafkaTransactionState transaction) - throws FlinkKafkaException { + private void flush(KafkaTransactionState transaction) throws FlinkKafkaException { if (transaction.producer != null) { transaction.producer.flush(); } @@ -1122,8 +1108,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { // To avoid duplication only first subtask keeps track of next transactional id hint. // Otherwise all of the // subtasks would write exactly same information. - if (getRuntimeContext().getIndexOfThisSubtask() == 0 - && semantic == FlinkKafkaProducer.Semantic.EXACTLY_ONCE) { + if (getRuntimeContext().getIndexOfThisSubtask() == 0 && semantic == Semantic.EXACTLY_ONCE) { checkState( nextTransactionalIdHint != null, "nextTransactionalIdHint must be set for EXACTLY_ONCE"); @@ -1141,7 +1126,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } nextTransactionalIdHintState.add( - new FlinkKafkaProducer.NextTransactionalIdHint( + new NextTransactionalIdHint( getRuntimeContext().getNumberOfParallelSubtasks(), nextFreeTransactionalId)); } @@ -1149,13 +1134,13 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { @Override public void initializeState(FunctionInitializationContext context) throws Exception { - if (semantic != FlinkKafkaProducer.Semantic.NONE + if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { LOG.warn( "Using {} semantic, but checkpointing is not enabled. Switching to {} semantic.", semantic, - FlinkKafkaProducer.Semantic.NONE); - semantic = FlinkKafkaProducer.Semantic.NONE; + Semantic.NONE); + semantic = Semantic.NONE; } nextTransactionalIdHintState = @@ -1195,18 +1180,17 @@ public void initializeState(FunctionInitializationContext context) throws Except kafkaProducersPoolSize, SAFE_SCALE_DOWN_FACTOR); - if (semantic != FlinkKafkaProducer.Semantic.EXACTLY_ONCE) { + if (semantic != Semantic.EXACTLY_ONCE) { nextTransactionalIdHint = null; } else { - List transactionalIdHints = - new ArrayList<>(); + List transactionalIdHints = new ArrayList<>(); nextTransactionalIdHintState.get().forEach(transactionalIdHints::add); if (transactionalIdHints.size() > 1) { throw new IllegalStateException( "There should be at most one next transactional id hint written by the first subtask"); } else if (transactionalIdHints.size() == 0) { - nextTransactionalIdHint = new FlinkKafkaProducer.NextTransactionalIdHint(0, 0); + nextTransactionalIdHint = new NextTransactionalIdHint(0, 0); // this means that this is either: // (1) the first execution of this application @@ -1223,14 +1207,14 @@ public void initializeState(FunctionInitializationContext context) throws Except } @Override - protected Optional initializeUserContext() { - if (semantic != FlinkKafkaProducer.Semantic.EXACTLY_ONCE) { + protected Optional initializeUserContext() { + if (semantic != Semantic.EXACTLY_ONCE) { return Optional.empty(); } Set transactionalIds = generateNewTransactionalIds(); resetAvailableTransactionalIdsPool(transactionalIds); - return Optional.of(new FlinkKafkaProducer.KafkaTransactionContext(transactionalIds)); + return Optional.of(new KafkaTransactionContext(transactionalIds)); } private Set generateNewTransactionalIds() { @@ -1246,8 +1230,18 @@ private Set generateNewTransactionalIds() { } @Override - protected void finishRecoveringContext( - Collection handledTransactions) { + protected void finishProcessing(@Nullable KafkaTransactionState transaction) { + super.finishProcessing(transaction); + // TwoPhaseCommitSink sets transaction = null on final checkpoint and thus closing will leak + // the producer. For transactional producers, we track the producer in pendingTransactions. + if (transaction != null && !transaction.isTransactional()) { + transaction.producer.flush(); + transaction.producer.close(Duration.ZERO); + } + } + + @Override + protected void finishRecoveringContext(Collection handledTransactions) { cleanUpUserContext(handledTransactions); resetAvailableTransactionalIdsPool(getUserContext().get().transactionalIds); LOG.info("Recovered transactionalIds {}", getUserContext().get().transactionalIds); @@ -1264,8 +1258,7 @@ protected FlinkKafkaInternalProducer createProducer() { * @param handledTransactions transactions which were already committed or aborted and do not * need further handling */ - private void cleanUpUserContext( - Collection handledTransactions) { + private void cleanUpUserContext(Collection handledTransactions) { if (!getUserContext().isPresent()) { return; } @@ -1320,7 +1313,7 @@ private void abortTransactions(final Set transactionalIds) { } int getTransactionCoordinatorId() { - final FlinkKafkaProducer.KafkaTransactionState currentTransaction = currentTransaction(); + final KafkaTransactionState currentTransaction = currentTransaction(); if (currentTransaction == null || currentTransaction.producer == null) { throw new IllegalArgumentException(); } @@ -1329,7 +1322,7 @@ int getTransactionCoordinatorId() { @VisibleForTesting String getTransactionalId() { - final FlinkKafkaProducer.KafkaTransactionState currentTransaction = currentTransaction(); + final KafkaTransactionState currentTransaction = currentTransaction(); if (currentTransaction == null || currentTransaction.producer == null) { throw new IllegalArgumentException(); } @@ -1415,6 +1408,7 @@ private FlinkKafkaInternalProducer initProducer(boolean register } } } + LOG.error("InitProducer {} {}", producerConfig, producer); return producer; } @@ -1430,8 +1424,7 @@ protected void checkErroneous() throws FlinkKafkaException { } } - private void readObject(java.io.ObjectInputStream in) - throws IOException, ClassNotFoundException { + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); } @@ -1562,8 +1555,7 @@ public boolean equals(Object o) { return false; } - FlinkKafkaProducer.KafkaTransactionState that = - (FlinkKafkaProducer.KafkaTransactionState) o; + KafkaTransactionState that = (KafkaTransactionState) o; if (producerId != that.producerId) { return false; @@ -1609,8 +1601,7 @@ public boolean equals(Object o) { return false; } - FlinkKafkaProducer.KafkaTransactionContext that = - (FlinkKafkaProducer.KafkaTransactionContext) o; + KafkaTransactionContext that = (KafkaTransactionContext) o; return transactionalIds.equals(that.transactionalIds); } @@ -1621,14 +1612,11 @@ public int hashCode() { } } - /** - * {@link org.apache.flink.api.common.typeutils.TypeSerializer} for {@link - * FlinkKafkaProducer.KafkaTransactionState}. - */ + /** {@link TypeSerializer} for {@link KafkaTransactionState}. */ @VisibleForTesting @Internal public static class TransactionStateSerializer - extends TypeSerializerSingleton { + extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; @@ -1638,20 +1626,17 @@ public boolean isImmutableType() { } @Override - public FlinkKafkaProducer.KafkaTransactionState createInstance() { + public KafkaTransactionState createInstance() { return null; } @Override - public FlinkKafkaProducer.KafkaTransactionState copy( - FlinkKafkaProducer.KafkaTransactionState from) { + public KafkaTransactionState copy(KafkaTransactionState from) { return from; } @Override - public FlinkKafkaProducer.KafkaTransactionState copy( - FlinkKafkaProducer.KafkaTransactionState from, - FlinkKafkaProducer.KafkaTransactionState reuse) { + public KafkaTransactionState copy(KafkaTransactionState from, KafkaTransactionState reuse) { return from; } @@ -1661,8 +1646,7 @@ public int getLength() { } @Override - public void serialize( - FlinkKafkaProducer.KafkaTransactionState record, DataOutputView target) + public void serialize(KafkaTransactionState record, DataOutputView target) throws IOException { if (record.transactionalId == null) { target.writeBoolean(false); @@ -1675,21 +1659,18 @@ public void serialize( } @Override - public FlinkKafkaProducer.KafkaTransactionState deserialize(DataInputView source) - throws IOException { + public KafkaTransactionState deserialize(DataInputView source) throws IOException { String transactionalId = null; if (source.readBoolean()) { transactionalId = source.readUTF(); } long producerId = source.readLong(); short epoch = source.readShort(); - return new FlinkKafkaProducer.KafkaTransactionState( - transactionalId, producerId, epoch, null); + return new KafkaTransactionState(transactionalId, producerId, epoch, null); } @Override - public FlinkKafkaProducer.KafkaTransactionState deserialize( - FlinkKafkaProducer.KafkaTransactionState reuse, DataInputView source) + public KafkaTransactionState deserialize(KafkaTransactionState reuse, DataInputView source) throws IOException { return deserialize(source); } @@ -1708,15 +1689,14 @@ public void copy(DataInputView source, DataOutputView target) throws IOException // ----------------------------------------------------------------------------------- @Override - public TypeSerializerSnapshot - snapshotConfiguration() { + public TypeSerializerSnapshot snapshotConfiguration() { return new TransactionStateSerializerSnapshot(); } /** Serializer configuration snapshot for compatibility and format evolution. */ @SuppressWarnings("WeakerAccess") public static final class TransactionStateSerializerSnapshot - extends SimpleTypeSerializerSnapshot { + extends SimpleTypeSerializerSnapshot { public TransactionStateSerializerSnapshot() { super(TransactionStateSerializer::new); @@ -1724,14 +1704,11 @@ public TransactionStateSerializerSnapshot() { } } - /** - * {@link org.apache.flink.api.common.typeutils.TypeSerializer} for {@link - * FlinkKafkaProducer.KafkaTransactionContext}. - */ + /** {@link TypeSerializer} for {@link KafkaTransactionContext}. */ @VisibleForTesting @Internal public static class ContextStateSerializer - extends TypeSerializerSingleton { + extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; @@ -1741,20 +1718,18 @@ public boolean isImmutableType() { } @Override - public FlinkKafkaProducer.KafkaTransactionContext createInstance() { + public KafkaTransactionContext createInstance() { return null; } @Override - public FlinkKafkaProducer.KafkaTransactionContext copy( - FlinkKafkaProducer.KafkaTransactionContext from) { + public KafkaTransactionContext copy(KafkaTransactionContext from) { return from; } @Override - public FlinkKafkaProducer.KafkaTransactionContext copy( - FlinkKafkaProducer.KafkaTransactionContext from, - FlinkKafkaProducer.KafkaTransactionContext reuse) { + public KafkaTransactionContext copy( + KafkaTransactionContext from, KafkaTransactionContext reuse) { return from; } @@ -1764,8 +1739,7 @@ public int getLength() { } @Override - public void serialize( - FlinkKafkaProducer.KafkaTransactionContext record, DataOutputView target) + public void serialize(KafkaTransactionContext record, DataOutputView target) throws IOException { int numIds = record.transactionalIds.size(); target.writeInt(numIds); @@ -1775,20 +1749,18 @@ public void serialize( } @Override - public FlinkKafkaProducer.KafkaTransactionContext deserialize(DataInputView source) - throws IOException { + public KafkaTransactionContext deserialize(DataInputView source) throws IOException { int numIds = source.readInt(); Set ids = new HashSet<>(numIds); for (int i = 0; i < numIds; i++) { ids.add(source.readUTF()); } - return new FlinkKafkaProducer.KafkaTransactionContext(ids); + return new KafkaTransactionContext(ids); } @Override - public FlinkKafkaProducer.KafkaTransactionContext deserialize( - FlinkKafkaProducer.KafkaTransactionContext reuse, DataInputView source) - throws IOException { + public KafkaTransactionContext deserialize( + KafkaTransactionContext reuse, DataInputView source) throws IOException { return deserialize(source); } @@ -1870,10 +1842,7 @@ public int hashCode() { } } - /** - * {@link org.apache.flink.api.common.typeutils.TypeSerializer} for {@link - * FlinkKafkaProducer.NextTransactionalIdHint}. - */ + /** {@link TypeSerializer} for {@link NextTransactionalIdHint}. */ @VisibleForTesting @Internal public static class NextTransactionalIdHintSerializer diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java index 7b345bf7d..61c06a1a3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java @@ -99,23 +99,17 @@ public void resourceCleanUpAtLeastOnce() throws Exception { public void resourceCleanUp(FlinkKafkaProducer.Semantic semantic) throws Exception { String topic = "flink-kafka-producer-resource-cleanup-" + semantic; - final int allowedEpsilonThreadCountGrow = 50; - - Optional initialActiveThreads = Optional.empty(); - for (int i = 0; i < allowedEpsilonThreadCountGrow * 2; i++) { - try (OneInputStreamOperatorTestHarness testHarness1 = - createTestHarness(topic, 1, 1, 0, semantic)) { - testHarness1.setup(); - testHarness1.open(); - } + try (OneInputStreamOperatorTestHarness testHarness1 = + createTestHarness(topic, 1, 1, 0, semantic)) { + testHarness1.setup(); + testHarness1.open(); + testHarness1.snapshot(1L, 100L); + testHarness1.notifyOfCompletedCheckpoint(1L); - if (initialActiveThreads.isPresent()) { - assertThat(Thread.activeCount()) - .as("active threads count") - .isLessThan(initialActiveThreads.get() + allowedEpsilonThreadCountGrow); - } else { - initialActiveThreads = Optional.of(Thread.activeCount()); - } + // test the leak fixed by FLINK-36441 + testHarness1.getOneInputOperator().finish(); + testHarness1.snapshot(2L, 100L); + testHarness1.notifyOfCompletedCheckpoint(2L); } checkProducerLeak(); } From 28836c616d2efea1aec7a513adaa11282dc61241 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 1 Oct 2024 23:34:58 +0200 Subject: [PATCH 298/322] [FLINK-36441] Ensure producers are not leaked in tests Add leak check in all relevant tests. --- .../FlinkKafkaInternalProducerITCase.java | 7 ++ .../kafka/sink/KafkaCommitterTest.java | 9 ++- .../kafka/sink/KafkaSinkBuilderTest.java | 1 - .../connector/kafka/sink/KafkaSinkITCase.java | 39 +--------- .../kafka/sink/KafkaTransactionLogITCase.java | 5 +- .../sink/KafkaWriterFaultToleranceITCase.java | 78 ++++++++++--------- .../kafka/sink/KafkaWriterITCase.java | 9 +++ .../kafka/sink/KafkaWriterTestBase.java | 7 ++ .../kafka/source/KafkaSourceLegacyITCase.java | 8 ++ .../connector/kafka/testutils/KafkaUtil.java | 43 ++++++++++ .../kafka/FlinkKafkaProducerITCase.java | 9 +-- .../kafka/testutils/DataGenerators.java | 36 ++++----- 12 files changed, 142 insertions(+), 109 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java index a4685be07..55224c052 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java @@ -30,6 +30,7 @@ import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; @@ -45,6 +46,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -57,6 +59,11 @@ class FlinkKafkaInternalProducerITCase { private static final KafkaContainer KAFKA_CONTAINER = createKafkaContainer(FlinkKafkaInternalProducerITCase.class).withEmbeddedZookeeper(); + @AfterEach + public void check() { + checkProducerLeak(); + } + @Test void testInitTransactionId() { final String topic = "test-init-transactions"; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java index ea9d893ed..a768bfc7a 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.serialization.StringSerializer; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -32,16 +33,22 @@ import java.util.Collections; import java.util.Properties; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link KafkaCommitter}. */ @ExtendWith({TestLoggerExtension.class}) -public class KafkaCommitterTest { +class KafkaCommitterTest { private static final int PRODUCER_ID = 0; private static final short EPOCH = 0; private static final String TRANSACTIONAL_ID = "transactionalId"; + @AfterEach + public void check() { + checkProducerLeak(); + } + /** Causes a network error by inactive broker and tests that a retry will happen. */ @Test public void testRetryCommittableOnRetriableError() throws IOException, InterruptedException { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java index eeecc84df..3e2beb9ad 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java @@ -52,7 +52,6 @@ public void testPropertyHandling() { getBasicBuilder().setProperty("k1", "v1"), p -> { Arrays.stream(DEFAULT_KEYS).forEach(k -> assertThat(p).containsKey(k)); - p.containsKey("k1"); }); Properties testConf = new Properties(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java index a257a44d4..d3bef5425 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java @@ -86,7 +86,6 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -101,9 +100,9 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; /** Tests for using KafkaSink writing to a Kafka cluster. */ public class KafkaSinkITCase extends TestLogger { @@ -158,6 +157,7 @@ public void setUp() throws ExecutionException, InterruptedException, TimeoutExce @After public void tearDown() throws ExecutionException, InterruptedException, TimeoutException { + checkProducerLeak(); deleteTestTopic(topic); } @@ -329,7 +329,6 @@ private void executeWithMapper( builder.setTransactionalIdPrefix(transactionalIdPrefix); stream.sinkTo(builder.build()); env.execute(); - checkProducerLeak(); } private void testRecoveryWithAssertion( @@ -600,40 +599,6 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { public void initializeState(FunctionInitializationContext context) throws Exception {} } - private void checkProducerLeak() throws InterruptedException { - List> leaks = null; - for (int tries = 0; tries < 10; tries++) { - leaks = - Thread.getAllStackTraces().entrySet().stream() - .filter(this::findAliveKafkaThread) - .collect(Collectors.toList()); - if (leaks.isEmpty()) { - return; - } - Thread.sleep(1000); - } - - for (Map.Entry leak : leaks) { - leak.getKey().stop(); - } - fail( - "Detected producer leaks:\n" - + leaks.stream().map(this::format).collect(Collectors.joining("\n\n"))); - } - - private String format(Map.Entry leak) { - String stackTrace = - Arrays.stream(leak.getValue()) - .map(StackTraceElement::toString) - .collect(Collectors.joining("\n")); - return leak.getKey().getName() + ":\n" + stackTrace; - } - - private boolean findAliveKafkaThread(Map.Entry threadStackTrace) { - return threadStackTrace.getKey().getState() != Thread.State.TERMINATED - && threadStackTrace.getKey().getName().contains("kafka-producer-network-thread"); - } - /** * Exposes information about how man records have been emitted overall and finishes after * receiving the checkpoint completed event. diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java index 121955acb..aca541455 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java @@ -29,8 +29,6 @@ import org.junit.After; import org.junit.ClassRule; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import java.util.ArrayList; @@ -44,13 +42,13 @@ import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.Ongoing; import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.PrepareAbort; import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.PrepareCommit; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link KafkaTransactionLog} to retrieve abortable Kafka transactions. */ public class KafkaTransactionLogITCase extends TestLogger { - private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkITCase.class); private static final String TOPIC_NAME = "kafkaTransactionLogTest"; private static final String TRANSACTIONAL_ID_PREFIX = "kafka-log"; @@ -63,6 +61,7 @@ public class KafkaTransactionLogITCase extends TestLogger { @After public void tearDown() { openProducers.forEach(Producer::close); + checkProducerLeak(); } @Test diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java index e045d7dda..1cf1b5c07 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java @@ -63,20 +63,21 @@ void testWriteExceptionWhenKafkaUnavailable() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - final KafkaWriter writer = + try (KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup); + properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup)) { - writer.write(1, SINK_WRITER_CONTEXT); + writer.write(1, SINK_WRITER_CONTEXT); - KAFKA_CONTAINER.stop(); + KAFKA_CONTAINER.stop(); - try { - writer.getCurrentProducer().flush(); - assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) - .hasRootCauseExactlyInstanceOf(NetworkException.class); - } finally { - KAFKA_CONTAINER.start(); + try { + writer.getCurrentProducer().flush(); + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .hasRootCauseExactlyInstanceOf(NetworkException.class); + } finally { + KAFKA_CONTAINER.start(); + } } } @@ -86,17 +87,18 @@ void testFlushExceptionWhenKafkaUnavailable() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - final KafkaWriter writer = + try (KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup); - writer.write(1, SINK_WRITER_CONTEXT); - - KAFKA_CONTAINER.stop(); - try { - assertThatCode(() -> writer.flush(false)) - .hasRootCauseExactlyInstanceOf(NetworkException.class); - } finally { - KAFKA_CONTAINER.start(); + properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup)) { + writer.write(1, SINK_WRITER_CONTEXT); + + KAFKA_CONTAINER.stop(); + try { + assertThatCode(() -> writer.flush(false)) + .hasRootCauseExactlyInstanceOf(NetworkException.class); + } finally { + KAFKA_CONTAINER.start(); + } } } @@ -106,7 +108,7 @@ void testCloseExceptionWhenKafkaUnavailable() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - final KafkaWriter writer = + KafkaWriter writer = createWriterWithConfiguration( properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup); @@ -119,6 +121,9 @@ void testCloseExceptionWhenKafkaUnavailable() throws Exception { // closing producer resource throws exception first assertThatCode(() -> writer.close()) .hasRootCauseExactlyInstanceOf(NetworkException.class); + } catch (Exception e) { + writer.close(); + throw e; } finally { KAFKA_CONTAINER.start(); } @@ -130,26 +135,27 @@ void testMailboxExceptionWhenKafkaUnavailable() throws Exception { SinkInitContext sinkInitContext = new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); - final KafkaWriter writer = + try (KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.AT_LEAST_ONCE, sinkInitContext); + properties, DeliveryGuarantee.AT_LEAST_ONCE, sinkInitContext)) { - KAFKA_CONTAINER.stop(); + KAFKA_CONTAINER.stop(); - writer.write(1, SINK_WRITER_CONTEXT); + writer.write(1, SINK_WRITER_CONTEXT); - try { - writer.getCurrentProducer().flush(); + try { + writer.getCurrentProducer().flush(); - assertThatCode( - () -> { - while (sinkInitContext.getMailboxExecutor().tryYield()) { - // execute all mails - } - }) - .hasRootCauseExactlyInstanceOf(TimeoutException.class); - } finally { - KAFKA_CONTAINER.start(); + assertThatCode( + () -> { + while (sinkInitContext.getMailboxExecutor().tryYield()) { + // execute all mails + } + }) + .hasRootCauseExactlyInstanceOf(TimeoutException.class); + } finally { + KAFKA_CONTAINER.start(); + } } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index 0aef938c9..e363d837c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -165,6 +165,9 @@ void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { .as("the exception is not thrown again") .doesNotThrowAnyException(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + // async exception is checked and thrown on close + assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); } @Test @@ -191,6 +194,9 @@ void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { .as("the exception is not thrown again") .doesNotThrowAnyException(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + // async exception is checked and thrown on close + assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); } @Test @@ -225,6 +231,9 @@ void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { .as("the exception is not thrown again") .doesNotThrowAnyException(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + // async exception is checked and thrown on close + assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); } @Test diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java index c5e6e9199..479d6ab8f 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java @@ -36,6 +36,7 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInfo; import org.slf4j.Logger; @@ -55,6 +56,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.function.Consumer; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; /** Test base for KafkaWriter. */ @@ -84,6 +86,11 @@ public void setUp(TestInfo testInfo) { topic = testInfo.getDisplayName().replaceAll("\\W", ""); } + @AfterEach + public void check() { + checkProducerLeak(); + } + protected KafkaWriter createWriterWithConfiguration( Properties config, DeliveryGuarantee guarantee) throws IOException { return createWriterWithConfiguration(config, guarantee, createSinkWriterMetricGroup()); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java index 5cc0ddf63..e9c9cab81 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java @@ -23,10 +23,13 @@ import org.apache.flink.streaming.connectors.kafka.KafkaProducerTestBase; import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl; +import org.junit.After; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; + /** * An IT case class that runs all the IT cases of the legacy {@link * org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer} with the new {@link KafkaSource}. @@ -44,6 +47,11 @@ public static void prepare() throws Exception { .setProducerSemantic(FlinkKafkaProducer.Semantic.AT_LEAST_ONCE); } + @After + public void check() { + checkProducerLeak(); + } + @Test public void testFailOnNoBroker() throws Exception { runFailOnNoBrokerTest(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java index 3dbe9bbbf..7bf7bb007 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java @@ -33,12 +33,15 @@ import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; import java.util.stream.Collectors; +import static org.assertj.core.api.Assertions.fail; + /** Collection of methods to interact with a Kafka cluster. */ public class KafkaUtil { @@ -189,4 +192,44 @@ private static Set getAllPartitions( .map(info -> new TopicPartition(info.topic(), info.partition())) .collect(Collectors.toSet()); } + + public static void checkProducerLeak() { + List> leaks = null; + for (int tries = 0; tries < 10; tries++) { + leaks = + Thread.getAllStackTraces().entrySet().stream() + .filter(KafkaUtil::findAliveKafkaThread) + .collect(Collectors.toList()); + if (leaks.isEmpty()) { + return; + } + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + } + } + + for (Map.Entry leak : leaks) { + leak.getKey().stop(); + } + fail( + "Detected producer leaks:\n" + + leaks.stream() + .map(KafkaUtil::format) + .collect(Collectors.joining("\n\n"))); + } + + private static String format(Map.Entry leak) { + String stackTrace = + Arrays.stream(leak.getValue()) + .map(StackTraceElement::toString) + .collect(Collectors.joining("\n")); + return leak.getKey().getName() + ":\n" + stackTrace; + } + + private static boolean findAliveKafkaThread( + Map.Entry threadStackTrace) { + return threadStackTrace.getKey().getState() != Thread.State.TERMINATED + && threadStackTrace.getKey().getName().contains("kafka-producer-network-thread"); + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java index 61c06a1a3..8644a796c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java @@ -45,6 +45,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.apache.flink.util.Preconditions.checkState; import static org.assertj.core.api.Assertions.assertThat; @@ -816,12 +817,4 @@ private boolean isCausedBy(FlinkKafkaErrorCode expectedErrorCode, Throwable ex) } return false; } - - private void checkProducerLeak() { - for (Thread t : Thread.getAllStackTraces().keySet()) { - if (t.getName().contains("kafka-producer-network-thread")) { - fail("Detected producer leak. Thread name: " + t.getName()); - } - } - } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java index be3651e58..92978a783 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java @@ -151,20 +151,19 @@ public InfiniteStringsGenerator(KafkaTestEnvironment server, String topic) { @Override public void run() { // we manually feed data into the Kafka sink - OneInputStreamOperatorTestHarness testHarness = null; - try { - Properties producerProperties = - KafkaUtils.getPropertiesFromBrokerList(server.getBrokerConnectionString()); - producerProperties.setProperty("retries", "3"); - - StreamSink sink = - server.getProducerSink( - topic, - new SimpleStringSchema(), - producerProperties, - new FlinkFixedPartitioner<>()); - - testHarness = new OneInputStreamOperatorTestHarness<>(sink); + + Properties producerProperties = + KafkaUtils.getPropertiesFromBrokerList(server.getBrokerConnectionString()); + producerProperties.setProperty("retries", "3"); + + StreamSink sink = + server.getProducerSink( + topic, + new SimpleStringSchema(), + producerProperties, + new FlinkFixedPartitioner<>()); + try (OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(sink)) { testHarness.open(); final StringBuilder bld = new StringBuilder(); @@ -183,20 +182,11 @@ public void run() { } } catch (Throwable t) { this.error = t; - } finally { - if (testHarness != null) { - try { - testHarness.close(); - } catch (Throwable t) { - // ignore - } - } } } public void shutdown() { this.running = false; - this.interrupt(); } public Throwable getError() { From 5eeafd6a8cd99490dcbc9425c2bdb0b24dc279ec Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Fri, 4 Oct 2024 10:53:28 +0200 Subject: [PATCH 299/322] [FLINK-36441] Fix KafkaWriterITCase#testIncreasingRecordBasedCounters The test tried to assert on byte counts which are written async. Commit adds flushing and establishes a baseline so that metadata request don't interfere with assertions. --- .../kafka/sink/KafkaWriterITCase.java | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index e363d837c..a559e3a54 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -88,31 +88,40 @@ public void testIncreasingRecordBasedCounters() throws Exception { try (final KafkaWriter writer = createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.NONE, metricGroup)) { + getKafkaClientConfiguration(), + DeliveryGuarantee.AT_LEAST_ONCE, + metricGroup)) { final Counter numBytesOut = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); final Counter numRecordsOut = metricGroup.getIOMetricGroup().getNumRecordsOutCounter(); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); final Counter numRecordsSendErrors = metricGroup.getNumRecordsSendErrorsCounter(); - assertThat(numBytesOut.getCount()).isEqualTo(0L); - assertThat(numRecordsOut.getCount()).isEqualTo(0); + + // ApiVersionsRequest etc. is sent on initialization, so establish some baseline + writer.write(0, SINK_WRITER_CONTEXT); + writer.flush(false); // ensure data is actually written + timeService.trigger(); // sync byte count + long baselineCount = numBytesOut.getCount(); + assertThat(numRecordsOut.getCount()).isEqualTo(1); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0); assertThat(numRecordsSendErrors.getCount()).isEqualTo(0); // elements for which the serializer returns null should be silently skipped writer.write(null, SINK_WRITER_CONTEXT); - timeService.trigger(); - assertThat(numBytesOut.getCount()).isEqualTo(0L); - assertThat(numRecordsOut.getCount()).isEqualTo(0); + writer.flush(false); + timeService.trigger(); // sync byte count + assertThat(numBytesOut.getCount()).isEqualTo(baselineCount); + assertThat(numRecordsOut.getCount()).isEqualTo(1); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0); assertThat(numRecordsSendErrors.getCount()).isEqualTo(0); // but elements for which a non-null producer record is returned should count writer.write(1, SINK_WRITER_CONTEXT); - timeService.trigger(); - assertThat(numRecordsOut.getCount()).isEqualTo(1); + writer.flush(false); + timeService.trigger(); // sync byte count + assertThat(numBytesOut.getCount()).isGreaterThan(baselineCount); + assertThat(numRecordsOut.getCount()).isEqualTo(2); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0); assertThat(numRecordsSendErrors.getCount()).isEqualTo(0); - assertThat(numBytesOut.getCount()).isGreaterThan(0L); } } From 429fe0c48e4485ab12836ebfa1643c31e62e97c2 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 9 Oct 2024 14:25:35 +0200 Subject: [PATCH 300/322] [FLINK-35109] Fix SmokeKafkaITCase for later java versions Workaround for FLINK-36454. --- .../org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java index eef63e57c..a8c416b0b 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java @@ -101,6 +101,10 @@ private static Configuration getConfiguration() { final Configuration flinkConfig = new Configuration(); flinkConfig.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 3); flinkConfig.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + // Workaround for FLINK-36454 ; default config is entirely overwritten + flinkConfig.setString( + "env.java.opts.all", + "--add-exports=java.base/sun.net.util=ALL-UNNAMED --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.text=ALL-UNNAMED --add-opens=java.base/java.time=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.locks=ALL-UNNAMED"); return flinkConfig; } From aedc79051323ed3c75d54859cdaa5770b224f4ce Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Mon, 30 Sep 2024 14:33:46 +0200 Subject: [PATCH 301/322] [FLINK-35109] Bump to Flink 1.19 and support Flink 1.20 Also - adds the migration support tests up to 1.20. - bumps Kafka-client to 3.6.2 --- .github/workflows/push_pr.yml | 8 +- .github/workflows/weekly.yml | 21 +- flink-connector-kafka/pom.xml | 12 +- .../FlinkKafkaConsumerBaseMigrationTest.java | 2 +- .../FlinkKafkaProducerMigrationTest.java | 2 +- .../kafka/KafkaSerializerUpgradeTest.java | 2 +- .../table/KafkaChangelogTableITCase.java | 4 - .../testutils/ThreadContextClassLoader.java | 41 ++++ .../testutils/TypeSerializerMatchers.java | 225 ++++++++++++++++++ .../TypeSerializerUpgradeTestBase.java | 21 +- .../context-state-serializer-1.11/test-data | Bin 19 -> 19 bytes .../context-state-serializer-1.12/test-data | Bin 19 -> 19 bytes .../context-state-serializer-1.13/test-data | Bin 19 -> 19 bytes .../context-state-serializer-1.14/test-data | Bin 19 -> 19 bytes .../context-state-serializer-1.15/test-data | Bin 19 -> 19 bytes .../context-state-serializer-1.16/test-data | Bin 19 -> 19 bytes .../serializer-snapshot | Bin 0 -> 126 bytes .../context-state-serializer-1.18/test-data | Bin 0 -> 19 bytes .../serializer-snapshot | Bin 0 -> 126 bytes .../context-state-serializer-1.19/test-data | Bin 0 -> 19 bytes .../serializer-snapshot | Bin 0 -> 126 bytes .../context-state-serializer-1.20/test-data | Bin 0 -> 19 bytes ...ration-test-flink1.17-empty-state-snapshot | Bin 0 -> 1465 bytes ...consumer-migration-test-flink1.17-snapshot | Bin 0 -> 1519 bytes ...ration-test-flink1.18-empty-state-snapshot | Bin 0 -> 1466 bytes ...consumer-migration-test-flink1.18-snapshot | Bin 0 -> 1520 bytes ...ration-test-flink1.19-empty-state-snapshot | Bin 0 -> 1466 bytes ...consumer-migration-test-flink1.19-snapshot | Bin 0 -> 1520 bytes ...ration-test-flink1.20-empty-state-snapshot | Bin 0 -> 1466 bytes ...consumer-migration-test-flink1.20-snapshot | Bin 0 -> 1520 bytes ...gration-kafka-producer-flink-1.17-snapshot | Bin 0 -> 1242 bytes ...gration-kafka-producer-flink-1.18-snapshot | Bin 0 -> 1243 bytes ...gration-kafka-producer-flink-1.19-snapshot | Bin 0 -> 1243 bytes ...gration-kafka-producer-flink-1.20-snapshot | Bin 0 -> 1243 bytes .../serializer-snapshot | Bin 0 -> 134 bytes .../test-data | Bin 0 -> 17 bytes .../serializer-snapshot | Bin 0 -> 134 bytes .../test-data | Bin 0 -> 17 bytes .../serializer-snapshot | Bin 0 -> 134 bytes .../test-data | Bin 0 -> 17 bytes pom.xml | 9 +- 41 files changed, 293 insertions(+), 54 deletions(-) create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThreadContextClassLoader.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerMatchers.java create mode 100644 flink-connector-kafka/src/test/resources/context-state-serializer-1.18/serializer-snapshot create mode 100644 flink-connector-kafka/src/test/resources/context-state-serializer-1.18/test-data create mode 100644 flink-connector-kafka/src/test/resources/context-state-serializer-1.19/serializer-snapshot create mode 100644 flink-connector-kafka/src/test/resources/context-state-serializer-1.19/test-data create mode 100644 flink-connector-kafka/src/test/resources/context-state-serializer-1.20/serializer-snapshot create mode 100644 flink-connector-kafka/src/test/resources/context-state-serializer-1.20/test-data create mode 100644 flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-empty-state-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-empty-state-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.19-empty-state-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.19-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-empty-state-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.17-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.18-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.19-snapshot create mode 100644 flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.20-snapshot create mode 100644 flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/serializer-snapshot create mode 100644 flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/test-data create mode 100644 flink-connector-kafka/src/test/resources/transaction-state-serializer-1.19/serializer-snapshot create mode 100644 flink-connector-kafka/src/test/resources/transaction-state-serializer-1.19/test-data create mode 100644 flink-connector-kafka/src/test/resources/transaction-state-serializer-1.20/serializer-snapshot create mode 100644 flink-connector-kafka/src/test/resources/transaction-state-serializer-1.20/test-data diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 951579406..fd1028711 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -28,10 +28,10 @@ jobs: compile_and_test: strategy: matrix: - flink: [ 1.18.1 ] - jdk: [ '8, 11, 17' ] + flink: [ 1.19.1 ] + jdk: [ '8, 11, 17, 21' ] include: - - flink: 1.19.0 + - flink: 1.20.0 jdk: '8, 11, 17, 21' uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: @@ -40,7 +40,7 @@ jobs: python_test: strategy: matrix: - flink: [ 1.18.1, 1.19.0 ] + flink: [ 1.19.1, 1.20.0 ] uses: apache/flink-connector-shared-utils/.github/workflows/python_ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} \ No newline at end of file diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index 1caecd5c9..da4426dfd 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -30,34 +30,21 @@ jobs: strategy: matrix: flink_branches: [{ - flink: 1.18-SNAPSHOT, - branch: main - }, { flink: 1.19-SNAPSHOT, - jdk: '8, 11, 17, 21', branch: main }, { flink: 1.20-SNAPSHOT, - jdk: '8, 11, 17, 21', branch: main }, { - flink: 1.18.1, + flink: 1.19.1, branch: v3.2 }, { - flink: 1.19.0, - branch: v3.2, - jdk: '8, 11, 17, 21', - }, { - flink: 1.18.1, - branch: v3.1 - }, { - flink: 1.19.0, - branch: v3.1, - jdk: '8, 11, 17, 21', + flink: 1.20.0, + branch: v3.2 }] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: flink_version: ${{ matrix.flink_branches.flink }} connector_branch: ${{ matrix.flink_branches.branch }} - jdk_version: ${{ matrix.flink_branches.jdk || '8, 11, 17' }} + jdk_version: ${{ matrix.flink_branches.jdk || '8, 11, 17, 21' }} run_dependency_convergence: false \ No newline at end of file diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 4a10bdc9a..85d437413 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -39,7 +39,8 @@ under the License. FlinkKafkaProducerBaseTest --> --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.locks=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED + KafkaProducerExactlyOnceITCase --> --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED + @@ -81,10 +82,10 @@ under the License. ${kafka.version} - - com.google.guava - guava - + + com.google.guava + guava + @@ -211,6 +212,7 @@ under the License. ${flink.version} test + org.apache.flink flink-table-planner_${scala.binary.version} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java index 296545cad..47bce8bd9 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java @@ -92,7 +92,7 @@ public class FlinkKafkaConsumerBaseMigrationTest { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { - return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.v1_16); + return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.current()); } public FlinkKafkaConsumerBaseMigrationTest(FlinkVersion testMigrateVersion) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java index 8a413f423..32c380c70 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java @@ -42,7 +42,7 @@ public class FlinkKafkaProducerMigrationTest extends KafkaMigrationTestBase { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { - return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.v1_16); + return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.current()); } public FlinkKafkaProducerMigrationTest(FlinkVersion testMigrateVersion) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java index c9e82be5d..90ce2e5eb 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java @@ -20,9 +20,9 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerMatchers; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer; +import org.apache.flink.streaming.connectors.kafka.testutils.TypeSerializerMatchers; import org.apache.flink.streaming.connectors.kafka.testutils.TypeSerializerUpgradeTestBase; import org.hamcrest.Matcher; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java index e8bc9e373..632b74ac9 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java @@ -29,7 +29,6 @@ import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.flink.table.api.config.OptimizerConfigOptions; import org.apache.kafka.clients.producer.ProducerConfig; import org.junit.Before; @@ -65,7 +64,6 @@ public void testKafkaDebeziumChangelogSource() throws Exception { tableConf.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)); tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L); - tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); // ---------- Write the Debezium json into Kafka ------------------- List lines = readLines("debezium-data-schema-exclude.txt"); @@ -194,7 +192,6 @@ public void testKafkaCanalChangelogSource() throws Exception { tableConf.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)); tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L); - tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); // ---------- Write the Canal json into Kafka ------------------- List lines = readLines("canal-data.txt"); @@ -335,7 +332,6 @@ public void testKafkaMaxwellChangelogSource() throws Exception { tableConf.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)); tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L); - tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); // ---------- Write the Maxwell json into Kafka ------------------- List lines = readLines("maxwell-data.txt"); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThreadContextClassLoader.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThreadContextClassLoader.java new file mode 100644 index 000000000..cdaec31c1 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThreadContextClassLoader.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.testutils; + +import java.io.Closeable; + +/** + * Utility class to temporarily use a different classloader as the thread context classloader. + * + *

      Temporarily copied from flink-core to avoid dependency on flink-core. + */ +public class ThreadContextClassLoader implements Closeable { + + private final ClassLoader originalThreadContextClassLoader; + + public ThreadContextClassLoader(ClassLoader newThreadContextClassLoader) { + this.originalThreadContextClassLoader = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(newThreadContextClassLoader); + } + + @Override + public void close() { + Thread.currentThread().setContextClassLoader(originalThreadContextClassLoader); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerMatchers.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerMatchers.java new file mode 100644 index 000000000..0815e2990 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerMatchers.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.testutils; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; + +import org.hamcrest.CoreMatchers; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeDiagnosingMatcher; +import org.hamcrest.TypeSafeMatcher; + +import java.util.function.Predicate; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A Collection of useful {@link Matcher}s for {@link TypeSerializer} and {@link + * TypeSerializerSchemaCompatibility}. + * + *

      Note copied from Flink 1.19. Remove when we drop 1.19 support. + */ +public final class TypeSerializerMatchers { + + private TypeSerializerMatchers() {} + + // ------------------------------------------------------------------------------------------------------------- + // Matcher Factories + // ------------------------------------------------------------------------------------------------------------- + + /** + * Matches {@code compatibleAsIs} {@link TypeSerializerSchemaCompatibility}. + * + * @param element type + * @return a {@code Matcher} that matches {@code compatibleAsIs} {@link + * TypeSerializerSchemaCompatibility}. + */ + public static Matcher> isCompatibleAsIs() { + return propertyMatcher( + TypeSerializerSchemaCompatibility::isCompatibleAsIs, + "type serializer schema that is a compatible as is"); + } + + /** + * Matches {@code isIncompatible} {@link TypeSerializerSchemaCompatibility}. + * + * @param element type + * @return a {@code Matcher} that matches {@code isIncompatible} {@link + * TypeSerializerSchemaCompatibility}. + */ + public static Matcher> isIncompatible() { + return propertyMatcher( + TypeSerializerSchemaCompatibility::isIncompatible, + "type serializer schema that is incompatible"); + } + + /** + * Matches {@code isCompatibleAfterMigration} {@link TypeSerializerSchemaCompatibility}. + * + * @param element type + * @return a {@code Matcher} that matches {@code isCompatibleAfterMigration} {@link + * TypeSerializerSchemaCompatibility}. + */ + public static Matcher> isCompatibleAfterMigration() { + return propertyMatcher( + TypeSerializerSchemaCompatibility::isCompatibleAfterMigration, + "type serializer schema that is compatible after migration"); + } + + /** + * Matches {@code isCompatibleWithReconfiguredSerializer} {@link + * TypeSerializerSchemaCompatibility}. + * + * @param element type + * @return a {@code Matcher} that matches {@code isCompatibleWithReconfiguredSerializer} {@link + * TypeSerializerSchemaCompatibility}. + */ + public static + Matcher> isCompatibleWithReconfiguredSerializer() { + @SuppressWarnings("unchecked") + Matcher> anything = + (Matcher>) (Matcher) CoreMatchers.anything(); + + return new CompatibleAfterReconfiguration<>(anything); + } + + /** + * Matches {@code isCompatibleWithReconfiguredSerializer} {@link + * TypeSerializerSchemaCompatibility}. + * + * @param reconfiguredSerializerMatcher matches the reconfigured serializer. + * @param element type + * @return a {@code Matcher} that matches {@code isCompatibleWithReconfiguredSerializer} {@link + * TypeSerializerSchemaCompatibility}. + */ + public static + Matcher> isCompatibleWithReconfiguredSerializer( + Matcher> reconfiguredSerializerMatcher) { + + return new CompatibleAfterReconfiguration<>(reconfiguredSerializerMatcher); + } + + /** + * Matches if the expected {@code TypeSerializerSchemaCompatibility} has the same compatibility + * as {@code expectedCompatibility}. + * + * @param expectedCompatibility the compatibility to match to. + * @param element type. + * @return a {@code Matcher} that matches if it has the same compatibility as {@code + * expectedCompatibility}. + */ + public static Matcher> hasSameCompatibilityAs( + TypeSerializerSchemaCompatibility expectedCompatibility) { + + return new SchemaCompatibilitySameAs<>(expectedCompatibility); + } + + // ------------------------------------------------------------------------------------------------------------- + // Helpers + // ------------------------------------------------------------------------------------------------------------- + + private static Matcher propertyMatcher( + Predicate predicate, String matcherDescription) { + return new TypeSafeMatcher() { + + @Override + protected boolean matchesSafely(T item) { + return predicate.test(item); + } + + @Override + public void describeTo(Description description) { + description.appendText(matcherDescription); + } + }; + } + + // ------------------------------------------------------------------------------------------------------------- + // Matchers + // ------------------------------------------------------------------------------------------------------------- + + private static final class CompatibleAfterReconfiguration + extends TypeSafeDiagnosingMatcher> { + + private final Matcher> reconfiguredSerializerMatcher; + + private CompatibleAfterReconfiguration( + Matcher> reconfiguredSerializerMatcher) { + this.reconfiguredSerializerMatcher = checkNotNull(reconfiguredSerializerMatcher); + } + + @Override + protected boolean matchesSafely( + TypeSerializerSchemaCompatibility item, Description mismatchDescription) { + if (!item.isCompatibleWithReconfiguredSerializer()) { + mismatchDescription.appendText( + "serializer schema is not compatible with a reconfigured serializer"); + return false; + } + TypeSerializer reconfiguredSerializer = item.getReconfiguredSerializer(); + if (!reconfiguredSerializerMatcher.matches(reconfiguredSerializer)) { + reconfiguredSerializerMatcher.describeMismatch( + reconfiguredSerializer, mismatchDescription); + return false; + } + return true; + } + + @Override + public void describeTo(Description description) { + description + .appendText("type serializer schema that is compatible after reconfiguration,") + .appendText("with a reconfigured serializer matching ") + .appendDescriptionOf(reconfiguredSerializerMatcher); + } + } + + private static class SchemaCompatibilitySameAs + extends TypeSafeMatcher> { + + private final TypeSerializerSchemaCompatibility expectedCompatibility; + + private SchemaCompatibilitySameAs( + TypeSerializerSchemaCompatibility expectedCompatibility) { + this.expectedCompatibility = checkNotNull(expectedCompatibility); + } + + @Override + protected boolean matchesSafely( + TypeSerializerSchemaCompatibility testResultCompatibility) { + if (expectedCompatibility.isCompatibleAsIs()) { + return testResultCompatibility.isCompatibleAsIs(); + } else if (expectedCompatibility.isIncompatible()) { + return testResultCompatibility.isIncompatible(); + } else if (expectedCompatibility.isCompatibleAfterMigration()) { + return testResultCompatibility.isCompatibleAfterMigration(); + } else if (expectedCompatibility.isCompatibleWithReconfiguredSerializer()) { + return testResultCompatibility.isCompatibleWithReconfiguredSerializer(); + } + return false; + } + + @Override + public void describeTo(Description description) { + description.appendText("same compatibility as ").appendValue(expectedCompatibility); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java index 90e218382..ab3baadef 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java @@ -20,9 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.typeutils.ClassRelocator; -import org.apache.flink.api.common.typeutils.ThreadContextClassLoader; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerMatchers; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil; @@ -61,7 +59,7 @@ @TestInstance(TestInstance.Lifecycle.PER_CLASS) public abstract class TypeSerializerUpgradeTestBase { - public static final FlinkVersion CURRENT_VERSION = FlinkVersion.v1_17; + public static final FlinkVersion CURRENT_VERSION = FlinkVersion.current(); public static final Set MIGRATION_VERSIONS = FlinkVersion.rangeOf(FlinkVersion.v1_11, CURRENT_VERSION); @@ -136,9 +134,6 @@ public TypeSerializer createPriorSerializer() { try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(setupClassloader)) { return delegateSetup.createPriorSerializer(); - } catch (IOException e) { - throw new RuntimeException( - "Error creating prior serializer via ClassLoaderSafePreUpgradeSetup.", e); } } @@ -147,9 +142,6 @@ public PreviousElementT createTestData() { try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(setupClassloader)) { return delegateSetup.createTestData(); - } catch (IOException e) { - throw new RuntimeException( - "Error creating test data via ThreadContextClassLoader.", e); } } } @@ -179,10 +171,6 @@ public TypeSerializer createUpgradedSerializer() { try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(verifierClassloader)) { return delegateVerifier.createUpgradedSerializer(); - } catch (IOException e) { - throw new RuntimeException( - "Error creating upgraded serializer via ClassLoaderSafeUpgradeVerifier.", - e); } } @@ -191,9 +179,6 @@ public Matcher testDataMatcher() { try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(verifierClassloader)) { return delegateVerifier.testDataMatcher(); - } catch (IOException e) { - throw new RuntimeException( - "Error creating expected test data via ClassLoaderSafeUpgradeVerifier.", e); } } @@ -203,10 +188,6 @@ public Matcher testDataMatcher() { try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(verifierClassloader)) { return delegateVerifier.schemaCompatibilityMatcher(version); - } catch (IOException e) { - throw new RuntimeException( - "Error creating schema compatibility matcher via ClassLoaderSafeUpgradeVerifier.", - e); } } } diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/test-data index 3efe488b18be8cab3ffc86511ae5a20321effd7f..d13f52b8c771029793a85c406fc90f3d1380cb6b 100644 GIT binary patch literal 19 acmZQzU|?ooHn*^3U^X!|V_-HkG6nz*^8xAr literal 19 acmZQzU|?ooHZ(G3U^X!|V_-J7umk`NKLP3h diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/test-data index 3efe488b18be8cab3ffc86511ae5a20321effd7f..d13f52b8c771029793a85c406fc90f3d1380cb6b 100644 GIT binary patch literal 19 acmZQzU|?ooHn*^3U^X!|V_-HkG6nz*^8xAr literal 19 acmZQzU|?ooHZ(G3U^X!|V_-J7umk`NKLP3h diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/test-data index 3efe488b18be8cab3ffc86511ae5a20321effd7f..d13f52b8c771029793a85c406fc90f3d1380cb6b 100644 GIT binary patch literal 19 acmZQzU|?ooHn*^3U^X!|V_-HkG6nz*^8xAr literal 19 acmZQzU|?ooHZ(G3U^X!|V_-J7umk`NKLP3h diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/test-data index 3efe488b18be8cab3ffc86511ae5a20321effd7f..d13f52b8c771029793a85c406fc90f3d1380cb6b 100644 GIT binary patch literal 19 acmZQzU|?ooHn*^3U^X!|V_-HkG6nz*^8xAr literal 19 acmZQzU|?ooHZ(G3U^X!|V_-J7umk`NKLP3h diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/test-data index 3efe488b18be8cab3ffc86511ae5a20321effd7f..d13f52b8c771029793a85c406fc90f3d1380cb6b 100644 GIT binary patch literal 19 acmZQzU|?ooHn*^3U^X!|V_-HkG6nz*^8xAr literal 19 acmZQzU|?ooHZ(G3U^X!|V_-J7umk`NKLP3h diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/test-data index 3efe488b18be8cab3ffc86511ae5a20321effd7f..d13f52b8c771029793a85c406fc90f3d1380cb6b 100644 GIT binary patch literal 19 acmZQzU|?ooHn*^3U^X!|V_-HkG6nz*^8xAr literal 19 acmZQzU|?ooHZ(G3U^X!|V_-J7umk`NKLP3h diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.18/serializer-snapshot b/flink-connector-kafka/src/test/resources/context-state-serializer-1.18/serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..85a71dc6ba48967d2886e2d23db00f3ace9f3e00 GIT binary patch literal 126 zcmZ|F!3lsc3_#JKt9Z^5f)_7BCkQoaX-y=JAWp9i;O+By0MLM|tw;yOjd3YSGFCUH uJ;{nuML52qVx}b%4j&l5oJ{94vURHgh8=%|}I0JIM-2QB~r literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.18/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.18/test-data new file mode 100644 index 0000000000000000000000000000000000000000..3efe488b18be8cab3ffc86511ae5a20321effd7f GIT binary patch literal 19 acmZQzU|?ooHZ(G3U^X!|V_-J7umk`NKLP3h literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.19/serializer-snapshot b/flink-connector-kafka/src/test/resources/context-state-serializer-1.19/serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..85a71dc6ba48967d2886e2d23db00f3ace9f3e00 GIT binary patch literal 126 zcmZ|F!3lsc3_#JKt9Z^5f)_7BCkQoaX-y=JAWp9i;O+By0MLM|tw;yOjd3YSGFCUH uJ;{nuML52qVx}b%4j&l5oJ{94vURHgh8=%|}I0JIM-2QB~r literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.19/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.19/test-data new file mode 100644 index 0000000000000000000000000000000000000000..3efe488b18be8cab3ffc86511ae5a20321effd7f GIT binary patch literal 19 acmZQzU|?ooHZ(G3U^X!|V_-J7umk`NKLP3h literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.20/serializer-snapshot b/flink-connector-kafka/src/test/resources/context-state-serializer-1.20/serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..85a71dc6ba48967d2886e2d23db00f3ace9f3e00 GIT binary patch literal 126 zcmZ|F!3lsc3_#JKt9Z^5f)_7BCkQoaX-y=JAWp9i;O+By0MLM|tw;yOjd3YSGFCUH uJ;{nuML52qVx}b%4j&l5oJ{94vURHgh8=%|}I0JIM-2QB~r literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.20/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.20/test-data new file mode 100644 index 0000000000000000000000000000000000000000..3efe488b18be8cab3ffc86511ae5a20321effd7f GIT binary patch literal 19 acmZQzU|?ooHZ(G3U^X!|V_-J7umk`NKLP3h literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-empty-state-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-empty-state-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..79ca38871d54646bf0a29ab74a5245ef27cf9888 GIT binary patch literal 1465 zcmcIkzi$&U6n>W`3lazfYo!iRC4|m_P_C^I!!!{4*!d@zkPtf(_BFq%mZD0y>&wG()$3FC&s$+ zPtXF=!Y;MiLB~&8oo<}?NziSEancDlc9XER)7@@01E}oom~-K9{iz@A2HiO5gnkr0 z4?5;=2^vE75Y90pEHS+AUp-8w4`{zcn=Z5 zIVD<1h59({W5l?ol4FzOmc1wTE!&f)1StRd^6pVyCu z;V3sm24@oaOV<>Fogt`doKwP5MnZ0e1Shm1vdpN`P@h>n7P6>r#FXtXyPOihNCN)v zv5&*qc&0{WI|ek|h-`MX4D(*Ckzu7wA%+re7CGA~fu?jvufHw6{fgf&fD=MlnMpA+ mEi`QS*< zXpyqWOHpd3nc$v?V@0*6G-|3GW8?N6vbT5M_Z-Mw#%os!&5H#O3u`l8u@DtKoSrUt zMVzLHGsUYj%{725H!j(K1~Eo9jsugG<#N@pSE`L#-LID$b3v_M4Q5yBL1n42SeYwB zc4f)zd&Z~k_~A;qQ7cyiKMd}ctLCctiOx3#3251aUezWy)Vd@Y->)r>s1ZP)&_h?z4`9pjTkZ8*- zK^Dw1l86hz0WhT{#SY^w5(&;}q=i(Zjd2?Z{T~p%HA*gIA*YPCE$OL-4AEunHpv67|?J!wAtkoEYCPmj=p}nhQ(1-TRBeU6Q>0kmiQBe{K-WpUWxBs|r7{RLZ l>BD#ToP*6T{cQY4tIpf)U+%cGM!(um*FUen>i-(7e*h{2^8o+= literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-empty-state-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-empty-state-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..0fc1a5555c4ee18363c72d756d145de97ca36480 GIT binary patch literal 1466 zcmcIkL2DC16n>i~9t1_vYoUiw5ZOsAlC+42G+E2iZBvr1f|u|z%%)>DGt100Cf@u7 zo)x@!_UhH6-u)Av1o197vuT@@)=HtyVP|G}?|a{S^SuWEOK`NjqCI%9Y-Gl%pP@3` za4CG5B$^pt8#GLN&e(%;f1_5d*KotH(K`AMYY*0ZT&H!PR@bZjI@?HSzY0(at@X8k zqvn{-Y^A*ybQ*EH)9c2KIOw&)Zrlks_v5g=+uLck0x0e8+H>x3>uDp}4|?686E>po zdC;+c3(%BmKrlmkz(|sEF|-#>Uf>vsnP%+J@Kh6ZC=4Gl5+7zM>oUbL<*%6P3e5C@ zG!|$HZhUtIDZuIlfX;mevIcjb{myAQpREgg&X5?YsU+L>b1snrglE9CQ4B|13<#A% zFlwaIWQfTS2^WSbfi}l&caPm;?j64+K=Id?caO6?ud9P~qS(Gdnhvjhg?JVY;NC(N zajYa6Fkv5_lC9a^LWAXoQaE+t?_bW|`*{g_3t#y714>%K<^rj?Sg9P1q#(hBQ7cy| zno=ZuP(GFi_7^qVxcPbK!^)da1@I!cu1~aVU^TKp6{e+C1gBai<|QgtadN91V=?t; zl3607bBX+=YjVNP5p=nmG0GE8Lt&?cpsXp=)T+`_pIbeVDz9$Dwd*gpyaK>P0si;c zvv4k+tx?g90Ry)qmt7@AF{m_C)Y=wetl&FcRsbdUYW literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..36381498b767cc8dc18d1ba1f2c8dd04ebaddc0b GIT binary patch literal 1520 zcmcIkF>4e-6n?Y0vk(l5Rze`f5fItAm_!p4i_K*XOE$T1w=p2XF}s<)jJG$l%uM3N zN`Ha1f{nF}g{_rnDOgzA2-+w}8Y{t>yCioZF%n3&n4Q^u?|a{S@0$kzL$LFpg4rM# z(IRJwn-i&-W`esSO%>Iy(nM2*jE&QCC7YK_DIMPXdd{6C7ZvVjncR+s&n7gcMGdgMiZV=kyiwP1E73aU$u#p;|7 z#g!$q?;4-K>xC|b2gZ&7wC+8S3AnWSr%TKF%pc)%fJ94f z3A|vILy0&L8~{^VQqpF;i4(y&O|*~-w@BI|m~l-dCnm=QdyZ^#mv?&$P}qL|{C3{W z>*QchvMB{_Qf?Zb;Q8Krj%ZTyq$FE5{MF;xqhH6cH}KA@ zpHNg1W(P?1#hQqATX5`mXkz3lct!A(HA}nlAn~APz2UcuFGrugaR7yIR_$q7!&qp5 z%2WxX2$m?3>W4(qvXxtT7mLbUo!k&^{6gD12ivdpN` zQ14sa5wcs|kSTk=?1Ri?=|~`_9=jLL#xpf4*fF5tLTIxqXM{J)m5eB53b85SVmD_y zCGZ~I&XZ3g&pwbBLx2J(C^IQK<_hgyC4)Y&Up_ROotE||P)roXq2SGag-ZL6`-Ty$ mqR$_HIF}KLzxJ~69P!eTJg%y#OF}5IAdRQs;9~F1)Vxds9A5~dHr6I>~mD@R8FSyx+M z`!{NWY0S2pEvH@eo9$l5uli1}?sojPyVv#I=3(!kS$81YJ=EvK;nuUN*L8Xwr|nif z_l48ee^XH7Vt{argJXiih_a!+P;`PP7^zuI#)?J~i7``jM36s@Bhn#);)uQ?qQh`3 zkGax7X}Iy-6eI;Z7XTXfIYx-F zGe!c%g+xOf4l$xk5y7y|v0?7MdCb=GEdesWzPx)9CwW~RtPw@y3TYa=<`v>;IDmT_ zRm2m)(SR`h@B!MN?{%zjt||ncneg{7XK($yguRAOef$AgDPV7d)LN_oAB{Le&XfdN zuAJ343+W)Yln3r_YPNdw^TCJhH=j~qd2n5>w5(vq(?EG5xmE-XBoEa~EJ)tSEx*Jf z;_)=rL`LTl`AgR%f?XrXTqh<04Qb#qJtbH`YCMXxDmC@B)l)8#>Uva~{xZu-0Zawp z{~miD&cxF-%9t^r;I?P7%SV_E^0f#{sS7a?a3{&xObN82JAM6a>+M(kJ`I)&8L215 nRJYKol{EUsxpi(fizi)Apgxuad7KT_70R8yo*G7r4A$Epi}~d! literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.19-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.19-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..ca2179cac8eb25a206b944ee7571503938491b09 GIT binary patch literal 1520 zcmcIkziSjh6n?w8vk(l5Rze`f5fItA{5WEwSZpq9ShC55yNv-6j>$NC8EzS^iKg5Z zXxS`B5^*6s1g5m4xW#w_5y3enT1bVOIB8FMy`UF1y5L`uqO|^IjmWK^v%kP@h7ica3VOX_O+~GA~HZl zs)SJlBT6Lt0hY9A$WIub7Sa<)?f z?bGc({y6sZJ$^m{P6#<=CPmj=q5Z35(1-SGM`p9z(!m7U#EPOw@W!A*g@eZf!w5E< lPanR!mmR$MrJs%eXv2BC^UED|?$NLI)Ai3=ulm0R>mRk^^0)v1 literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-empty-state-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-empty-state-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..78ba15781b599180fdcfbca6e66b1c8015ec88fa GIT binary patch literal 1466 zcmcIk&ubGw6n>ke9t1_vYoUiw5ZOs>lbRwP(qt`5woOU43SPp?I-5?jnOSD0G4bYK z;90?oXRls8>fL|BlOWy&XSP3*(pV|fIqb|V?|tukZ@%{cAP>jODVT!;i&~^Ca#NI= zX(qTU;#g7bDvg>d#~M3O#PY_AhimKTK3I)=Zn<2oxEok4yOoU!tDMtI0#0ldAb7ABFM@aM}rH7g&ipWE**^rX(FimKON`?u0MP-L$ zs`iC8KzX?F-4-MV%NGDz_al&1xclsPM$7taUEp($#6V62X`0VmB02ESfhjF14jJ#0 zNN`RgEu<*Hm&eIX3M*w2!%a@|FOFUtiumPP4qO4%UjId4)7BUh4|+3>?6{ znJVH)3euEY_sNi4A}Ma`w*8OV}Iu+{YiVpd@V0keZ4$62qb3#2eGd z$W`#V;4$kLPvn8US2 o#-@c%Tgjl$oOkDDbNr;!3DiSHNg#NCTA||E>zQG6oWXSa1GK&6TmS$7 literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..995b70e585d57305de822cf042f57e5e03ca7be9 GIT binary patch literal 1520 zcmcIkL2DC16n?u+JO~y=PeP%GP!QQk8$%_Ehi$r+rQ4A`>Ou+7A<@Cma zDaA7yI~f*=D$1F|lSC5bNQD)VmbS5n4tam~eb0hId9z%Zn~!E4Qo5En<ON^Q}D z{Q9ci4~);>af5ZQ+3@PV8~Asa+&`ascr+`~w1XQZoWDv{_i zMLPtA-7F;yA}CJjLn0asXL5@x4KxAgzL^5qFmnu`aUX(|;L^tL0WIURbd1j-5^b@~ z(Xw8SBw~Yq2uvzLaEG!MiaBE>R$NHb#z`9^$`la{>l~NOIW)~(+v_brZs*;zTiGD5 z(}Oj=TRuga2CsICcp47i%2*X~PjJ*COgBD8OT+gfR=D5_fwxWgt0%L!exATy!`m-^ zKwb)17$Y?jYs@(htt_GJr4UxwMZk2b8gJHPBnYmnm>s-0<|wBuf&`A~T*-mY71O5m``nW`>-xD3m-EW3 z@M~eb>9LX}h*C=C^!g)+fRUw`ajE**SjqRS>iZ0Lf>F=IE9&kVvcK1mB};0?=PVnq zFojkEU-Hq6i0F_n7E}hxb(8~YfgbXTWP!rggT6IY2)_J!%$m_U|5;P|BRap*_)YEF z|Bni7sX2aX(ubT$HSD+5|LrbRRw86(s5gWV?Dct=U?LJ!FPc@{)-BY?L$~$c1aA*o myPH{(GC8a~2Yku#?NMJYqg;+sO_u=MW!iAcfa^K$zn>?CqEK)E literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.18-snapshot b/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.18-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..68c9d63268d2ae438e744457e4b771c35d0634e2 GIT binary patch literal 1243 zcmbtT-A>yu6t+vt-K1V{i?oa0O^#Y+KEw{Fc+P5;x6iSsmt6>_pH}5G1ZX? z+kU-bo*aUY61cpOe6+{fPZ$d7R7A55@69!-biBM;OjBeTu|>RA9~9dh;wWKU+gkS$bgIwQ@O1HC+O{U#5+A8E`Y_{qVU3#DG%Q literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.19-snapshot b/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.19-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..03b1a24a17bdedc09cb5e1cdab80657ce8f2dce4 GIT binary patch literal 1243 zcmbtTPjAyO6t}yE+r$oBkvQ~*9Hnivt;cT5M6|B05*H*c<&n6nmDa9e4-n!LaOAj8 zwVn0>I3T_QNZ28x%LLsbo|MFX{=GlH-*XT`Tc|YLG>Zm$D%b~2bPhs6qPY~XAUq{= zE;M=9!3JX=V~uO%C=Hs~&5iFXMlG+mmjdj*C9^%>Cw{lrCcWN%O1f$4cX#(0>v(`r zb7;aI-Cwaxj?uHxIEeam6is8=r@{0njA;}eOlUY7PESWi0dgiovn~R5UiJr*U>XNe z*dK(igUEOr=(EhvTv$OeXYOplMdm73PEP z*3d{pZz~g~FN`iR%Q-Cg$A=(d0V_3^+Vr!v*6+8h_cPpy$AgfrsRtX#zS%$)R@|MP zi(f?c4X90~bz5dvp z1g`J(`rbF!*WL=a7U|p~D-y=knk^VlEU8jTmWP;gE|B~oc6WR?)?&;X9ja+Heb zG^A;BprygjmGPn@txG^tMi%VyEr@`Vm5hbb{Z!ZbJ<9qz!$vUbdw5OV-9YyH2C`(u ztl25g<|{2}l)&d=6cb7J#d671pk0SKpc$AcuSpgtY~7h_Q-k3B*ZZs)qx02<(r?lE znZ_?_H~)Q9XiLxWzb3sexYEOZS^e+sLS-#NW_A6B5Q0x9B1|xm39^gk6t}j6T6t*0 s{7&#@?`QlYOHw9>ljo>D<(S`^%T+Izqg2x+z}scoXqN%oIqyH82Q!IM&j0`b literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/serializer-snapshot b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..b95eacb633e5812b3ee57980a0dd34732d9d81cc GIT binary patch literal 134 zcmaLNu?@mN3`SuqtU~QEXlRfw3;Z%pVqLH!pJ@=oqbxx8r0)^YMBm7jaX>cZWjF6U yDitsDl_lI&Dsphg(!sZLxS#pRc)msRIwyQU%EDWrv}qc4`~0QmIK~!5M3*1FD=|s{ literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/test-data b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/test-data new file mode 100644 index 0000000000000000000000000000000000000000..0936509e94a5ecf733366d7275336f06d56d70a1 GIT binary patch literal 17 UcmZQ%U@cZWjF6U yDitsDl_lI&Dsphg(!sZLxS#pRc)msRIwyQU%EDWrv}qc4`~0QmIK~!5M3*1FD=|s{ literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.19/test-data b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.19/test-data new file mode 100644 index 0000000000000000000000000000000000000000..0936509e94a5ecf733366d7275336f06d56d70a1 GIT binary patch literal 17 UcmZQ%U@cZWjF6U yDitsDl_lI&Dsphg(!sZLxS#pRc)msRIwyQU%EDWrv}qc4`~0QmIK~!5M3*1FD=|s{ literal 0 HcmV?d00001 diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.20/test-data b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.20/test-data new file mode 100644 index 0000000000000000000000000000000000000000..0936509e94a5ecf733366d7275336f06d56d70a1 GIT binary patch literal 17 UcmZQ%U@ - 1.18.0 + 1.19.1 3.4.0 7.4.4 @@ -403,6 +403,13 @@ under the License. 2.1 + + + org.yaml + snakeyaml + 2.2 + + org.apache.commons From 7f1ac6db59360b4bb1ee7a3fea4eac3399524326 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 10 Oct 2024 15:37:53 +0200 Subject: [PATCH 302/322] [hotfix] Add maven wrapper --- .mvn/wrapper/maven-wrapper.properties | 19 ++ mvnw | 259 ++++++++++++++++++++++++++ mvnw.cmd | 149 +++++++++++++++ 3 files changed, 427 insertions(+) create mode 100644 .mvn/wrapper/maven-wrapper.properties create mode 100755 mvnw create mode 100644 mvnw.cmd diff --git a/.mvn/wrapper/maven-wrapper.properties b/.mvn/wrapper/maven-wrapper.properties new file mode 100644 index 000000000..38d23edda --- /dev/null +++ b/.mvn/wrapper/maven-wrapper.properties @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +wrapperVersion=3.3.3-SNAPSHOT +distributionType=only-script +distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.9/apache-maven-3.9.9-bin.zip diff --git a/mvnw b/mvnw new file mode 100755 index 000000000..684df0068 --- /dev/null +++ b/mvnw @@ -0,0 +1,259 @@ +#!/bin/sh +# ---------------------------------------------------------------------------- +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# ---------------------------------------------------------------------------- + +# ---------------------------------------------------------------------------- +# Apache Maven Wrapper startup batch script, version 3.3.3-SNAPSHOT +# +# Optional ENV vars +# ----------------- +# JAVA_HOME - location of a JDK home dir, required when download maven via java source +# MVNW_REPOURL - repo url base for downloading maven distribution +# MVNW_USERNAME/MVNW_PASSWORD - user and password for downloading maven +# MVNW_VERBOSE - true: enable verbose log; debug: trace the mvnw script; others: silence the output +# ---------------------------------------------------------------------------- + +set -euf +[ "${MVNW_VERBOSE-}" != debug ] || set -x + +# OS specific support. +native_path() { printf %s\\n "$1"; } +case "$(uname)" in +CYGWIN* | MINGW*) + [ -z "${JAVA_HOME-}" ] || JAVA_HOME="$(cygpath --unix "$JAVA_HOME")" + native_path() { cygpath --path --windows "$1"; } + ;; +esac + +# set JAVACMD and JAVACCMD +set_java_home() { + # For Cygwin and MinGW, ensure paths are in Unix format before anything is touched + if [ -n "${JAVA_HOME-}" ]; then + if [ -x "$JAVA_HOME/jre/sh/java" ]; then + # IBM's JDK on AIX uses strange locations for the executables + JAVACMD="$JAVA_HOME/jre/sh/java" + JAVACCMD="$JAVA_HOME/jre/sh/javac" + else + JAVACMD="$JAVA_HOME/bin/java" + JAVACCMD="$JAVA_HOME/bin/javac" + + if [ ! -x "$JAVACMD" ] || [ ! -x "$JAVACCMD" ]; then + echo "The JAVA_HOME environment variable is not defined correctly, so mvnw cannot run." >&2 + echo "JAVA_HOME is set to \"$JAVA_HOME\", but \"\$JAVA_HOME/bin/java\" or \"\$JAVA_HOME/bin/javac\" does not exist." >&2 + return 1 + fi + fi + else + JAVACMD="$( + 'set' +e + 'unset' -f command 2>/dev/null + 'command' -v java + )" || : + JAVACCMD="$( + 'set' +e + 'unset' -f command 2>/dev/null + 'command' -v javac + )" || : + + if [ ! -x "${JAVACMD-}" ] || [ ! -x "${JAVACCMD-}" ]; then + echo "The java/javac command does not exist in PATH nor is JAVA_HOME set, so mvnw cannot run." >&2 + return 1 + fi + fi +} + +# hash string like Java String::hashCode +hash_string() { + str="${1:-}" h=0 + while [ -n "$str" ]; do + char="${str%"${str#?}"}" + h=$(((h * 31 + $(LC_CTYPE=C printf %d "'$char")) % 4294967296)) + str="${str#?}" + done + printf %x\\n $h +} + +verbose() { :; } +[ "${MVNW_VERBOSE-}" != true ] || verbose() { printf %s\\n "${1-}"; } + +die() { + printf %s\\n "$1" >&2 + exit 1 +} + +trim() { + # MWRAPPER-139: + # Trims trailing and leading whitespace, carriage returns, tabs, and linefeeds. + # Needed for removing poorly interpreted newline sequences when running in more + # exotic environments such as mingw bash on Windows. + printf "%s" "${1}" | tr -d '[:space:]' +} + +# parse distributionUrl and optional distributionSha256Sum, requires .mvn/wrapper/maven-wrapper.properties +while IFS="=" read -r key value; do + case "${key-}" in + distributionUrl) distributionUrl=$(trim "${value-}") ;; + distributionSha256Sum) distributionSha256Sum=$(trim "${value-}") ;; + esac +done <"${0%/*}/.mvn/wrapper/maven-wrapper.properties" +[ -n "${distributionUrl-}" ] || die "cannot read distributionUrl property in ${0%/*}/.mvn/wrapper/maven-wrapper.properties" + +case "${distributionUrl##*/}" in +maven-mvnd-*bin.*) + MVN_CMD=mvnd.sh _MVNW_REPO_PATTERN=/maven/mvnd/ + case "${PROCESSOR_ARCHITECTURE-}${PROCESSOR_ARCHITEW6432-}:$(uname -a)" in + *AMD64:CYGWIN* | *AMD64:MINGW*) distributionPlatform=windows-amd64 ;; + :Darwin*x86_64) distributionPlatform=darwin-amd64 ;; + :Darwin*arm64) distributionPlatform=darwin-aarch64 ;; + :Linux*x86_64*) distributionPlatform=linux-amd64 ;; + *) + echo "Cannot detect native platform for mvnd on $(uname)-$(uname -m), use pure java version" >&2 + distributionPlatform=linux-amd64 + ;; + esac + distributionUrl="${distributionUrl%-bin.*}-$distributionPlatform.zip" + ;; +maven-mvnd-*) MVN_CMD=mvnd.sh _MVNW_REPO_PATTERN=/maven/mvnd/ ;; +*) MVN_CMD="mvn${0##*/mvnw}" _MVNW_REPO_PATTERN=/org/apache/maven/ ;; +esac + +# apply MVNW_REPOURL and calculate MAVEN_HOME +# maven home pattern: ~/.m2/wrapper/dists/{apache-maven-,maven-mvnd--}/ +[ -z "${MVNW_REPOURL-}" ] || distributionUrl="$MVNW_REPOURL$_MVNW_REPO_PATTERN${distributionUrl#*"$_MVNW_REPO_PATTERN"}" +distributionUrlName="${distributionUrl##*/}" +distributionUrlNameMain="${distributionUrlName%.*}" +distributionUrlNameMain="${distributionUrlNameMain%-bin}" +MAVEN_USER_HOME="${MAVEN_USER_HOME:-${HOME}/.m2}" +MAVEN_HOME="${MAVEN_USER_HOME}/wrapper/dists/${distributionUrlNameMain-}/$(hash_string "$distributionUrl")" + +exec_maven() { + unset MVNW_VERBOSE MVNW_USERNAME MVNW_PASSWORD MVNW_REPOURL || : + exec "$MAVEN_HOME/bin/$MVN_CMD" "$@" || die "cannot exec $MAVEN_HOME/bin/$MVN_CMD" +} + +if [ -d "$MAVEN_HOME" ]; then + verbose "found existing MAVEN_HOME at $MAVEN_HOME" + exec_maven "$@" +fi + +case "${distributionUrl-}" in +*?-bin.zip | *?maven-mvnd-?*-?*.zip) ;; +*) die "distributionUrl is not valid, must match *-bin.zip or maven-mvnd-*.zip, but found '${distributionUrl-}'" ;; +esac + +# prepare tmp dir +if TMP_DOWNLOAD_DIR="$(mktemp -d)" && [ -d "$TMP_DOWNLOAD_DIR" ]; then + clean() { rm -rf -- "$TMP_DOWNLOAD_DIR"; } + trap clean HUP INT TERM EXIT +else + die "cannot create temp dir" +fi + +mkdir -p -- "${MAVEN_HOME%/*}" + +# Download and Install Apache Maven +verbose "Couldn't find MAVEN_HOME, downloading and installing it ..." +verbose "Downloading from: $distributionUrl" +verbose "Downloading to: $TMP_DOWNLOAD_DIR/$distributionUrlName" + +# select .zip or .tar.gz +if ! command -v unzip >/dev/null; then + distributionUrl="${distributionUrl%.zip}.tar.gz" + distributionUrlName="${distributionUrl##*/}" +fi + +# verbose opt +__MVNW_QUIET_WGET=--quiet __MVNW_QUIET_CURL=--silent __MVNW_QUIET_UNZIP=-q __MVNW_QUIET_TAR='' +[ "${MVNW_VERBOSE-}" != true ] || __MVNW_QUIET_WGET='' __MVNW_QUIET_CURL='' __MVNW_QUIET_UNZIP='' __MVNW_QUIET_TAR=v + +# normalize http auth +case "${MVNW_PASSWORD:+has-password}" in +'') MVNW_USERNAME='' MVNW_PASSWORD='' ;; +has-password) [ -n "${MVNW_USERNAME-}" ] || MVNW_USERNAME='' MVNW_PASSWORD='' ;; +esac + +if [ -z "${MVNW_USERNAME-}" ] && command -v wget >/dev/null; then + verbose "Found wget ... using wget" + wget ${__MVNW_QUIET_WGET:+"$__MVNW_QUIET_WGET"} "$distributionUrl" -O "$TMP_DOWNLOAD_DIR/$distributionUrlName" || die "wget: Failed to fetch $distributionUrl" +elif [ -z "${MVNW_USERNAME-}" ] && command -v curl >/dev/null; then + verbose "Found curl ... using curl" + curl ${__MVNW_QUIET_CURL:+"$__MVNW_QUIET_CURL"} -f -L -o "$TMP_DOWNLOAD_DIR/$distributionUrlName" "$distributionUrl" || die "curl: Failed to fetch $distributionUrl" +elif set_java_home; then + verbose "Falling back to use Java to download" + javaSource="$TMP_DOWNLOAD_DIR/Downloader.java" + targetZip="$TMP_DOWNLOAD_DIR/$distributionUrlName" + cat >"$javaSource" <<-END + public class Downloader extends java.net.Authenticator + { + protected java.net.PasswordAuthentication getPasswordAuthentication() + { + return new java.net.PasswordAuthentication( System.getenv( "MVNW_USERNAME" ), System.getenv( "MVNW_PASSWORD" ).toCharArray() ); + } + public static void main( String[] args ) throws Exception + { + setDefault( new Downloader() ); + java.nio.file.Files.copy( java.net.URI.create( args[0] ).toURL().openStream(), java.nio.file.Paths.get( args[1] ).toAbsolutePath().normalize() ); + } + } + END + # For Cygwin/MinGW, switch paths to Windows format before running javac and java + verbose " - Compiling Downloader.java ..." + "$(native_path "$JAVACCMD")" "$(native_path "$javaSource")" || die "Failed to compile Downloader.java" + verbose " - Running Downloader.java ..." + "$(native_path "$JAVACMD")" -cp "$(native_path "$TMP_DOWNLOAD_DIR")" Downloader "$distributionUrl" "$(native_path "$targetZip")" +fi + +# If specified, validate the SHA-256 sum of the Maven distribution zip file +if [ -n "${distributionSha256Sum-}" ]; then + distributionSha256Result=false + if [ "$MVN_CMD" = mvnd.sh ]; then + echo "Checksum validation is not supported for maven-mvnd." >&2 + echo "Please disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." >&2 + exit 1 + elif command -v sha256sum >/dev/null; then + if echo "$distributionSha256Sum $TMP_DOWNLOAD_DIR/$distributionUrlName" | sha256sum -c >/dev/null 2>&1; then + distributionSha256Result=true + fi + elif command -v shasum >/dev/null; then + if echo "$distributionSha256Sum $TMP_DOWNLOAD_DIR/$distributionUrlName" | shasum -a 256 -c >/dev/null 2>&1; then + distributionSha256Result=true + fi + else + echo "Checksum validation was requested but neither 'sha256sum' or 'shasum' are available." >&2 + echo "Please install either command, or disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." >&2 + exit 1 + fi + if [ $distributionSha256Result = false ]; then + echo "Error: Failed to validate Maven distribution SHA-256, your Maven distribution might be compromised." >&2 + echo "If you updated your Maven version, you need to update the specified distributionSha256Sum property." >&2 + exit 1 + fi +fi + +# unzip and move +if command -v unzip >/dev/null; then + unzip ${__MVNW_QUIET_UNZIP:+"$__MVNW_QUIET_UNZIP"} "$TMP_DOWNLOAD_DIR/$distributionUrlName" -d "$TMP_DOWNLOAD_DIR" || die "failed to unzip" +else + tar xzf${__MVNW_QUIET_TAR:+"$__MVNW_QUIET_TAR"} "$TMP_DOWNLOAD_DIR/$distributionUrlName" -C "$TMP_DOWNLOAD_DIR" || die "failed to untar" +fi +printf %s\\n "$distributionUrl" >"$TMP_DOWNLOAD_DIR/$distributionUrlNameMain/mvnw.url" +mv -- "$TMP_DOWNLOAD_DIR/$distributionUrlNameMain" "$MAVEN_HOME" || [ -d "$MAVEN_HOME" ] || die "fail to move MAVEN_HOME" + +clean || : +exec_maven "$@" diff --git a/mvnw.cmd b/mvnw.cmd new file mode 100644 index 000000000..f6af60a91 --- /dev/null +++ b/mvnw.cmd @@ -0,0 +1,149 @@ +<# : batch portion +@REM ---------------------------------------------------------------------------- +@REM Licensed to the Apache Software Foundation (ASF) under one +@REM or more contributor license agreements. See the NOTICE file +@REM distributed with this work for additional information +@REM regarding copyright ownership. The ASF licenses this file +@REM to you under the Apache License, Version 2.0 (the +@REM "License"); you may not use this file except in compliance +@REM with the License. You may obtain a copy of the License at +@REM +@REM http://www.apache.org/licenses/LICENSE-2.0 +@REM +@REM Unless required by applicable law or agreed to in writing, +@REM software distributed under the License is distributed on an +@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +@REM KIND, either express or implied. See the License for the +@REM specific language governing permissions and limitations +@REM under the License. +@REM ---------------------------------------------------------------------------- + +@REM ---------------------------------------------------------------------------- +@REM Apache Maven Wrapper startup batch script, version 3.3.3-SNAPSHOT +@REM +@REM Optional ENV vars +@REM MVNW_REPOURL - repo url base for downloading maven distribution +@REM MVNW_USERNAME/MVNW_PASSWORD - user and password for downloading maven +@REM MVNW_VERBOSE - true: enable verbose log; others: silence the output +@REM ---------------------------------------------------------------------------- + +@IF "%__MVNW_ARG0_NAME__%"=="" (SET __MVNW_ARG0_NAME__=%~nx0) +@SET __MVNW_CMD__= +@SET __MVNW_ERROR__= +@SET __MVNW_PSMODULEP_SAVE=%PSModulePath% +@SET PSModulePath= +@FOR /F "usebackq tokens=1* delims==" %%A IN (`powershell -noprofile "& {$scriptDir='%~dp0'; $script='%__MVNW_ARG0_NAME__%'; icm -ScriptBlock ([Scriptblock]::Create((Get-Content -Raw '%~f0'))) -NoNewScope}"`) DO @( + IF "%%A"=="MVN_CMD" (set __MVNW_CMD__=%%B) ELSE IF "%%B"=="" (echo %%A) ELSE (echo %%A=%%B) +) +@SET PSModulePath=%__MVNW_PSMODULEP_SAVE% +@SET __MVNW_PSMODULEP_SAVE= +@SET __MVNW_ARG0_NAME__= +@SET MVNW_USERNAME= +@SET MVNW_PASSWORD= +@IF NOT "%__MVNW_CMD__%"=="" (%__MVNW_CMD__% %*) +@echo Cannot start maven from wrapper >&2 && exit /b 1 +@GOTO :EOF +: end batch / begin powershell #> + +$ErrorActionPreference = "Stop" +if ($env:MVNW_VERBOSE -eq "true") { + $VerbosePreference = "Continue" +} + +# calculate distributionUrl, requires .mvn/wrapper/maven-wrapper.properties +$distributionUrl = (Get-Content -Raw "$scriptDir/.mvn/wrapper/maven-wrapper.properties" | ConvertFrom-StringData).distributionUrl +if (!$distributionUrl) { + Write-Error "cannot read distributionUrl property in $scriptDir/.mvn/wrapper/maven-wrapper.properties" +} + +switch -wildcard -casesensitive ( $($distributionUrl -replace '^.*/','') ) { + "maven-mvnd-*" { + $USE_MVND = $true + $distributionUrl = $distributionUrl -replace '-bin\.[^.]*$',"-windows-amd64.zip" + $MVN_CMD = "mvnd.cmd" + break + } + default { + $USE_MVND = $false + $MVN_CMD = $script -replace '^mvnw','mvn' + break + } +} + +# apply MVNW_REPOURL and calculate MAVEN_HOME +# maven home pattern: ~/.m2/wrapper/dists/{apache-maven-,maven-mvnd--}/ +if ($env:MVNW_REPOURL) { + $MVNW_REPO_PATTERN = if ($USE_MVND) { "/org/apache/maven/" } else { "/maven/mvnd/" } + $distributionUrl = "$env:MVNW_REPOURL$MVNW_REPO_PATTERN$($distributionUrl -replace '^.*'+$MVNW_REPO_PATTERN,'')" +} +$distributionUrlName = $distributionUrl -replace '^.*/','' +$distributionUrlNameMain = $distributionUrlName -replace '\.[^.]*$','' -replace '-bin$','' +$MAVEN_HOME_PARENT = "$HOME/.m2/wrapper/dists/$distributionUrlNameMain" +if ($env:MAVEN_USER_HOME) { + $MAVEN_HOME_PARENT = "$env:MAVEN_USER_HOME/wrapper/dists/$distributionUrlNameMain" +} +$MAVEN_HOME_NAME = ([System.Security.Cryptography.MD5]::Create().ComputeHash([byte[]][char[]]$distributionUrl) | ForEach-Object {$_.ToString("x2")}) -join '' +$MAVEN_HOME = "$MAVEN_HOME_PARENT/$MAVEN_HOME_NAME" + +if (Test-Path -Path "$MAVEN_HOME" -PathType Container) { + Write-Verbose "found existing MAVEN_HOME at $MAVEN_HOME" + Write-Output "MVN_CMD=$MAVEN_HOME/bin/$MVN_CMD" + exit $? +} + +if (! $distributionUrlNameMain -or ($distributionUrlName -eq $distributionUrlNameMain)) { + Write-Error "distributionUrl is not valid, must end with *-bin.zip, but found $distributionUrl" +} + +# prepare tmp dir +$TMP_DOWNLOAD_DIR_HOLDER = New-TemporaryFile +$TMP_DOWNLOAD_DIR = New-Item -Itemtype Directory -Path "$TMP_DOWNLOAD_DIR_HOLDER.dir" +$TMP_DOWNLOAD_DIR_HOLDER.Delete() | Out-Null +trap { + if ($TMP_DOWNLOAD_DIR.Exists) { + try { Remove-Item $TMP_DOWNLOAD_DIR -Recurse -Force | Out-Null } + catch { Write-Warning "Cannot remove $TMP_DOWNLOAD_DIR" } + } +} + +New-Item -Itemtype Directory -Path "$MAVEN_HOME_PARENT" -Force | Out-Null + +# Download and Install Apache Maven +Write-Verbose "Couldn't find MAVEN_HOME, downloading and installing it ..." +Write-Verbose "Downloading from: $distributionUrl" +Write-Verbose "Downloading to: $TMP_DOWNLOAD_DIR/$distributionUrlName" + +$webclient = New-Object System.Net.WebClient +if ($env:MVNW_USERNAME -and $env:MVNW_PASSWORD) { + $webclient.Credentials = New-Object System.Net.NetworkCredential($env:MVNW_USERNAME, $env:MVNW_PASSWORD) +} +[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12 +$webclient.DownloadFile($distributionUrl, "$TMP_DOWNLOAD_DIR/$distributionUrlName") | Out-Null + +# If specified, validate the SHA-256 sum of the Maven distribution zip file +$distributionSha256Sum = (Get-Content -Raw "$scriptDir/.mvn/wrapper/maven-wrapper.properties" | ConvertFrom-StringData).distributionSha256Sum +if ($distributionSha256Sum) { + if ($USE_MVND) { + Write-Error "Checksum validation is not supported for maven-mvnd. `nPlease disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." + } + Import-Module $PSHOME\Modules\Microsoft.PowerShell.Utility -Function Get-FileHash + if ((Get-FileHash "$TMP_DOWNLOAD_DIR/$distributionUrlName" -Algorithm SHA256).Hash.ToLower() -ne $distributionSha256Sum) { + Write-Error "Error: Failed to validate Maven distribution SHA-256, your Maven distribution might be compromised. If you updated your Maven version, you need to update the specified distributionSha256Sum property." + } +} + +# unzip and move +Expand-Archive "$TMP_DOWNLOAD_DIR/$distributionUrlName" -DestinationPath "$TMP_DOWNLOAD_DIR" | Out-Null +Rename-Item -Path "$TMP_DOWNLOAD_DIR/$distributionUrlNameMain" -NewName $MAVEN_HOME_NAME | Out-Null +try { + Move-Item -Path "$TMP_DOWNLOAD_DIR/$MAVEN_HOME_NAME" -Destination $MAVEN_HOME_PARENT | Out-Null +} catch { + if (! (Test-Path -Path "$MAVEN_HOME" -PathType Container)) { + Write-Error "fail to move MAVEN_HOME" + } +} finally { + try { Remove-Item $TMP_DOWNLOAD_DIR -Recurse -Force | Out-Null } + catch { Write-Warning "Cannot remove $TMP_DOWNLOAD_DIR" } +} + +Write-Output "MVN_CMD=$MAVEN_HOME/bin/$MVN_CMD" From 6dc74db6c544efb1e49fb774c0d551ce5ca08935 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 10 Oct 2024 15:53:56 +0200 Subject: [PATCH 303/322] Addressing Chesnay's feedback --- .mvn/wrapper/maven-wrapper.properties | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/.mvn/wrapper/maven-wrapper.properties b/.mvn/wrapper/maven-wrapper.properties index 38d23edda..405de879d 100644 --- a/.mvn/wrapper/maven-wrapper.properties +++ b/.mvn/wrapper/maven-wrapper.properties @@ -14,6 +14,11 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -wrapperVersion=3.3.3-SNAPSHOT -distributionType=only-script -distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.9/apache-maven-3.9.9-bin.zip + +wrapperVersion=3.3.2 + +# updating the Maven version requires updates to certain documentation and verification logic +distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.8.6/apache-maven-3.8.6-bin.zip +distributionSha256Sum=ccf20a80e75a17ffc34d47c5c95c98c39d426ca17d670f09cd91e877072a9309 +wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.3.2/maven-wrapper-3.3.2.jar +wrapperSha256Sum=3d8f20ce6103913be8b52aef6d994e0c54705fb527324ceb9b835b338739c7a8 From c2e11a6e22f0252cecf0d3d519d9fabb08fc102a Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 10 Oct 2024 23:16:28 +0200 Subject: [PATCH 304/322] [FLINK-35109] Update version to 3.4-SNAPSHOT --- .../flink-end-to-end-tests-common-kafka/pom.xml | 2 +- .../flink-streaming-kafka-test-base/pom.xml | 2 +- .../flink-streaming-kafka-test/pom.xml | 2 +- flink-connector-kafka-e2e-tests/pom.xml | 2 +- flink-connector-kafka/pom.xml | 2 +- flink-python/pom.xml | 2 +- flink-sql-connector-kafka/pom.xml | 2 +- pom.xml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index c6faf1149..e414db962 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 3.3-SNAPSHOT + 3.4-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml index c393fe80c..974e9d421 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 3.3-SNAPSHOT + 3.4-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml index a9dadfa6e..87498bea7 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 3.3-SNAPSHOT + 3.4-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/pom.xml b/flink-connector-kafka-e2e-tests/pom.xml index 00661d0bd..12c881197 100644 --- a/flink-connector-kafka-e2e-tests/pom.xml +++ b/flink-connector-kafka-e2e-tests/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 3.3-SNAPSHOT + 3.4-SNAPSHOT pom diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 85d437413..ae4cc71b8 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 3.3-SNAPSHOT + 3.4-SNAPSHOT flink-connector-kafka diff --git a/flink-python/pom.xml b/flink-python/pom.xml index 978db5f5d..dcb6117fc 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 3.3-SNAPSHOT + 3.4-SNAPSHOT flink-connector-kafka-python diff --git a/flink-sql-connector-kafka/pom.xml b/flink-sql-connector-kafka/pom.xml index 0f724d1ce..506e55d66 100644 --- a/flink-sql-connector-kafka/pom.xml +++ b/flink-sql-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 3.3-SNAPSHOT + 3.4-SNAPSHOT flink-sql-connector-kafka diff --git a/pom.xml b/pom.xml index 8a8575be1..50e235df8 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 3.3-SNAPSHOT + 3.4-SNAPSHOT Flink : Connectors : Kafka : Parent pom 2022 From 2dfdae6e8e44828a9c2b6db833348b65d9379b43 Mon Sep 17 00:00:00 2001 From: Peter Huang Date: Mon, 14 Oct 2024 01:47:22 -0700 Subject: [PATCH 305/322] [FLINK-34467] bump flink version to 1.20.0 (#111) --- .gitignore | 1 + .../86dfd459-67a9-4b26-9b5c-0b0bbf22681a | 15 +- .../c0d94764-76a0-4c50-b617-70b1754c4612 | 210 ++++++++++++++++++ .../d853eb69-8c04-4246-9a5e-4f5911286b1d | 2 +- .../archunit-violations/stored.rules | 8 +- pom.xml | 2 +- 6 files changed, 234 insertions(+), 4 deletions(-) create mode 100644 flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 diff --git a/.gitignore b/.gitignore index 901fd674f..485c27aa6 100644 --- a/.gitignore +++ b/.gitignore @@ -38,6 +38,7 @@ tools/releasing/release tools/japicmp-output # Generated file, do not store in git +flink-connector-kafka/.idea flink-python/pyflink/datastream/connectors/kafka_connector_version.py flink-python/apache_flink_connectors_kafka.egg-info/ flink-python/.tox/ diff --git a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a index 4406d5c5f..07efd19a9 100644 --- a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a +++ b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a @@ -20,6 +20,12 @@ org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only o * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.sink.KafkaWriterFaultToleranceITCase does not satisfy: only one of the following predicates match:\ @@ -28,6 +34,12 @@ org.apache.flink.connector.kafka.sink.KafkaWriterFaultToleranceITCase does not s * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match: +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.source.KafkaSourceITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ @@ -101,6 +113,7 @@ org.apache.flink.connector.kafka.sink.KafkaTransactionLogITCase does not satisfy * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match:\ +org.apache.flink.connector.kafka.sink.KafkaWriterFaultToleranceITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ @@ -165,4 +178,4 @@ org.apache.flink.streaming.connectors.kafka.shuffle.KafkaShuffleITCase does not * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule diff --git a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 new file mode 100644 index 000000000..20326f5ec --- /dev/null +++ b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 @@ -0,0 +1,210 @@ +Class is annotated with in (KafkaSourceEnumerator.java:0) +Class is annotated with in (KafkaSourceEnumerator.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber, org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, java.util.Properties, org.apache.flink.api.connector.source.Boundedness, org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState, org.apache.flink.connector.kafka.dynamic.source.enumerator.StoppableKafkaEnumContextProxy$StoppableKafkaEnumContextProxyFactory)> is annotated with in (DynamicKafkaSourceEnumerator.java:0) +Constructor (org.apache.flink.api.connector.source.SourceReaderContext, org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema, java.util.Properties)> calls constructor (int)> in (DynamicKafkaSourceReader.java:114) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1733) +Constructor (java.util.Set)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (java.lang.String, long, short, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1879) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1630) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.api.common.typeutils.TypeSerializer)> is annotated with in (KafkaShuffleFetcher.java:0) +Field has generic type > with type argument depending on in (KafkaClusterMetricGroupManager.java:0) +Field has type in (DynamicKafkaSourceReader.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (KafkaShuffleFetcher.java:0) +Field has type in (FlinkKafkaShuffleProducer.java:0) +Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Method is annotated with in (DynamicKafkaSource.java:0) +Method calls method in (KafkaClusterMetricGroupManager.java:73) +Method calls method in (KafkaClusterMetricGroupManager.java:62) +Method checks instanceof in (KafkaClusterMetricGroupManager.java:42) +Method calls constructor (int)> in (DynamicKafkaSourceReader.java:475) +Method calls method in (DynamicKafkaSourceReader.java:474) +Method calls method in (DynamicKafkaSourceReader.java:485) +Method has return type in (DynamicKafkaSourceReader.java:0) +Method is annotated with in (DynamicKafkaSourceReader.java:0) +Method is annotated with in (DynamicKafkaSourceReader.java:0) +Method calls method in (DynamicKafkaSourceReader.java:383) +Method calls method in (DynamicKafkaSourceReader.java:381) +Method calls method in (DynamicKafkaSourceReader.java:496) +Method is annotated with in (KafkaSink.java:0) +Method calls method in (KafkaSinkBuilder.java:152) +Method is annotated with in (KafkaWriter.java:0) +Method is annotated with in (KafkaWriter.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSourceEnumStateSerializer.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaPartitionSplitReader.java:0) +Method is annotated with in (KafkaPartitionSplitReader.java:0) +Method is annotated with in (KafkaSourceReader.java:0) +Method is annotated with in (KafkaSourceReader.java:0) +Method calls method in (AbstractFetcher.java:604) +Method calls method in (AbstractFetcher.java:608) +Method calls method in (AbstractFetcher.java:608) +Method calls method in (AbstractFetcher.java:593) +Method calls method in (AbstractFetcher.java:593) +Method calls constructor ([B)> in (KafkaShuffleFetcher.java:240) +Method calls method in (KafkaShuffleFetcher.java:244) +Method calls method in (KafkaShuffleFetcher.java:245) +Method calls method in (KafkaShuffleFetcher.java:254) +Method calls method in (KafkaShuffleFetcher.java:251) +Method calls method in (KafkaShuffleFetcher.java:255) +Method calls method in (KafkaShuffleFetcher.java:238) +Method gets field in (KafkaShuffleFetcher.java:244) +Method gets field in (KafkaShuffleFetcher.java:245) +Method gets field in (KafkaShuffleFetcher.java:253) +Method gets field in (KafkaShuffleFetcher.java:250) +Method gets field in (KafkaShuffleFetcher.java:254) +Method is annotated with in (KafkaShuffleFetcher.java:0) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:186) +Method calls method in (FlinkKafkaShuffleProducer.java:205) +Method calls method in (FlinkKafkaShuffleProducer.java:204) +Method calls method in (FlinkKafkaShuffleProducer.java:190) +Method calls method in (FlinkKafkaShuffleProducer.java:193) +Method calls method in (FlinkKafkaShuffleProducer.java:195) +Method calls method in (FlinkKafkaShuffleProducer.java:196) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:212) +Method calls method in (FlinkKafkaShuffleProducer.java:225) +Method calls method in (FlinkKafkaShuffleProducer.java:224) +Method calls method in (FlinkKafkaShuffleProducer.java:216) +Method calls method in (FlinkKafkaShuffleProducer.java:217) +Method calls method in (FlinkKafkaShuffleProducer.java:218) +Method calls method in (FlinkKafkaShuffleProducer.java:219) +Method has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Method calls method in (KafkaConnectorOptionsUtil.java:543) +Method calls method in (KafkaConnectorOptionsUtil.java:587) +Method calls method in (KafkaDynamicSink.java:386) +Method has return type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (KafkaDynamicSink.java:0) +Method calls method in (KafkaDynamicSource.java:566) +Constructor (java.lang.String, org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext, java.lang.Runnable)> calls constructor (java.lang.String)> in (StoppableKafkaEnumContextProxy.java:95) +Constructor (java.util.function.Function, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.kafka.sink.HeaderProvider)> calls method in (KafkaRecordSerializationSchemaBuilder.java:322) +Constructor (java.util.function.Function, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.kafka.sink.HeaderProvider)> calls method in (KafkaRecordSerializationSchemaBuilder.java:323) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:51) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:53) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:54) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:134) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:135) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:136) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:137) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:138) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:173) +Constructor (java.lang.String)> calls method in (KafkaWriterState.java:28) +Constructor (java.lang.Object, java.util.function.Consumer)> calls method in (Recyclable.java:31) +Constructor (java.lang.Object, java.util.function.Consumer)> calls method in (Recyclable.java:32) +Constructor (int, int, java.util.function.Function, java.util.function.Consumer)> calls method in (TransactionAborter.java:60) +Constructor (java.util.Set)> calls method in (FlinkKafkaProducer.java:1591) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> calls method in (AbstractFetcher.java:593) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> calls method in (AbstractFetcher.java:595) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaConsumerThread, org.apache.flink.streaming.connectors.kafka.internals.KafkaCommitCallback)> calls method in (KafkaConsumerThread.java:540) +Constructor (int, org.apache.flink.api.common.serialization.DeserializationSchema, [I, org.apache.flink.api.common.serialization.DeserializationSchema, [I, boolean, [Lorg.apache.flink.streaming.connectors.kafka.table.DynamicKafkaDeserializationSchema$MetadataConverter;, org.apache.flink.api.common.typeinfo.TypeInformation, boolean)> calls method in (DynamicKafkaDeserializationSchema.java:72) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:71) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:75) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:87) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:181) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:161) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:163) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:166) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:167) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:168) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:176) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:179) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:210) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:194) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:198) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:201) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:203) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:216) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:218) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:220) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:224) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:226) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:70) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:71) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:72) +Method calls method in (DynamicKafkaSourceBuilder.java:291) +Method calls method in (DynamicKafkaSourceBuilder.java:293) +Method calls method in (DynamicKafkaSourceBuilder.java:295) +Method calls method in (DynamicKafkaSourceBuilder.java:299) +Method calls method in (DynamicKafkaSourceBuilder.java:100) +Method calls method in (DynamicKafkaSourceBuilder.java:73) +Method calls method in (DynamicKafkaSourceBuilder.java:86) +Method calls method in (DynamicKafkaSourceEnumStateSerializer.java:142) +Method calls method in (DynamicKafkaSourceEnumerator.java:514) +Method calls method in (StoppableKafkaEnumContextProxy.java:237) +Method calls method in (StoppableKafkaEnumContextProxy.java:259) +Method calls method in (DynamicKafkaSourceReader.java:418) +Method calls method in (DynamicKafkaSourceReader.java:228) +Method calls method in (FlinkKafkaInternalProducer.java:100) +Method calls method in (FlinkKafkaInternalProducer.java:109) +Method calls method in (FlinkKafkaInternalProducer.java:296) +Method calls method in (FlinkKafkaInternalProducer.java:297) +Method calls method in (FlinkKafkaInternalProducer.java:174) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:268) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:269) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:283) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:279) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:218) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:154) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:112) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:99) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:124) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:123) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:139) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:137) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:204) +Method calls method in (KafkaSerializerWrapper.java:71) +Method calls method in (KafkaSerializerWrapper.java:88) +Method calls method in (KafkaSinkBuilder.java:194) +Method calls method in (KafkaSinkBuilder.java:202) +Method calls method in (KafkaSinkBuilder.java:198) +Method calls method in (KafkaSinkBuilder.java:111) +Method calls method in (KafkaSinkBuilder.java:97) +Method calls method in (KafkaSinkBuilder.java:123) +Method calls method in (KafkaSinkBuilder.java:133) +Method calls method in (KafkaSinkBuilder.java:151) +Method calls method in (KafkaSinkBuilder.java:175) +Method calls method in (KafkaSinkBuilder.java:176) +Method calls method in (KafkaWriter.java:244) +Method calls method in (KafkaWriter.java:245) +Method calls method in (KafkaWriter.java:246) +Method calls method in (KafkaWriter.java:311) +Method calls method in (Recyclable.java:36) +Method calls method in (KafkaSourceBuilder.java:513) +Method calls method in (KafkaSourceBuilder.java:518) +Method calls method in (KafkaSourceBuilder.java:522) +Method calls method in (KafkaSourceBuilder.java:524) +Method calls method in (KafkaSourceBuilder.java:203) +Method calls method in (ReaderHandledOffsetsInitializer.java:75) +Method calls method in (SpecifiedOffsetsInitializer.java:105) +Method calls method in (KafkaPartitionSplitReader.java:540) +Method calls method in (KafkaPartitionSplitReader.java:359) +Method calls method in (KafkaValueOnlyDeserializerWrapper.java:65) +Method calls method in (KafkaShuffleFetcher.java:280) +Method calls method in (KafkaConnectorOptionsUtil.java:500) +Method calls method in (KafkaConnectorOptionsUtil.java:567) +Method calls method in (KafkaConnectorOptionsUtil.java:480) +Method calls method in (ReducingUpsertWriter.java:177) diff --git a/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d b/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d index b591e33c6..dfb76a0c5 100644 --- a/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d +++ b/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d @@ -2,4 +2,4 @@ org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema.open(org.ap org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema.serialize(java.lang.Object, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext, java.lang.Long): Argument leaf type org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer.getPartitionOffsets(java.util.Collection, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer$PartitionOffsetsRetriever): Argument leaf type org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer$PartitionOffsetsRetriever does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.invoke(org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$KafkaTransactionState, java.lang.Object, org.apache.flink.streaming.api.functions.sink.SinkFunction$Context): Argument leaf type org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$KafkaTransactionState does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated -org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition.dropLeaderData(java.util.List): Argument leaf type org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated \ No newline at end of file +org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition.dropLeaderData(java.util.List): Argument leaf type org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated diff --git a/flink-connector-kafka/archunit-violations/stored.rules b/flink-connector-kafka/archunit-violations/stored.rules index 372199016..122522db7 100644 --- a/flink-connector-kafka/archunit-violations/stored.rules +++ b/flink-connector-kafka/archunit-violations/stored.rules @@ -1,10 +1,16 @@ # -#Fri Feb 10 21:57:58 CET 2023 +#Sat Aug 10 18:37:22 PDT 2024 Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @Public\ must\ be\ annotated\ with\ @Public.=690729ae-756f-4ccc-831b-2667db3fdae5 Connector\ production\ code\ must\ not\ depend\ on\ non-public\ API\ outside\ of\ connector\ packages=3030609a-c417-456d-af43-1799a4445197 +Classes\ in\ API\ packages\ should\ have\ at\ least\ one\ API\ visibility\ annotation.=661bd4bd-792c-4ba4-af39-87b7d16dcf75 ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=86dfd459-67a9-4b26-9b5c-0b0bbf22681a Production\ code\ must\ not\ call\ methods\ annotated\ with\ @VisibleForTesting=27a0a5e4-29c2-4069-b381-952746c90862 Options\ for\ connectors\ and\ formats\ should\ reside\ in\ a\ consistent\ package\ and\ be\ public\ API.=3b06cb2f-1036-4d54-a358-c37f163611ee Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=cdd3929e-b7bc-4541-a0ae-46ed4fbb2001 +Options\ for\ connectors\ and\ formats\ should\ reside\ in\ a\ consistent\ package\ and\ be\ public\ API.=3b06cb2f-1036-4d54-a358-c37f163611ee +Connector\ production\ code\ must\ not\ depend\ on\ non-public\ API\ outside\ of\ connector\ packages=3030609a-c417-456d-af43-1799a4445197 +Production\ code\ must\ not\ call\ methods\ annotated\ with\ @VisibleForTesting=27a0a5e4-29c2-4069-b381-952746c90862 +Connector\ production\ code\ must\ depend\ only\ on\ public\ API\ when\ outside\ of\ connector\ packages=c0d94764-76a0-4c50-b617-70b1754c4612 +Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @Public\ must\ be\ annotated\ with\ @Public.=690729ae-756f-4ccc-831b-2667db3fdae5 Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @PublicEvolving\ must\ be\ annotated\ with\ @Public(Evolving).=d853eb69-8c04-4246-9a5e-4f5911286b1d Classes\ in\ API\ packages\ should\ have\ at\ least\ one\ API\ visibility\ annotation.=661bd4bd-792c-4ba4-af39-87b7d16dcf75 diff --git a/pom.xml b/pom.xml index 50e235df8..41d9aefc4 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,7 @@ under the License. - 1.19.1 + 1.20.0 3.4.0 7.4.4 From 0fed445e80149fb44c51d54054411a0e7107614b Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Mon, 14 Oct 2024 11:32:26 +0200 Subject: [PATCH 306/322] [FLINK-35109] Updating build configurations for v3.3 --- .github/workflows/push_pr.yml | 9 +++------ .github/workflows/weekly.yml | 9 +++------ 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index fd1028711..69854c344 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -28,11 +28,8 @@ jobs: compile_and_test: strategy: matrix: - flink: [ 1.19.1 ] + flink: [ 1.20.0 ] jdk: [ '8, 11, 17, 21' ] - include: - - flink: 1.20.0 - jdk: '8, 11, 17, 21' uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: flink_version: ${{ matrix.flink }} @@ -40,7 +37,7 @@ jobs: python_test: strategy: matrix: - flink: [ 1.19.1, 1.20.0 ] + flink: [ 1.20.0 ] uses: apache/flink-connector-shared-utils/.github/workflows/python_ci.yml@ci_utils with: - flink_version: ${{ matrix.flink }} \ No newline at end of file + flink_version: ${{ matrix.flink }} diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index da4426dfd..cea005757 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -30,21 +30,18 @@ jobs: strategy: matrix: flink_branches: [{ - flink: 1.19-SNAPSHOT, - branch: main - }, { flink: 1.20-SNAPSHOT, branch: main }, { flink: 1.19.1, - branch: v3.2 + branch: v3.3 }, { flink: 1.20.0, - branch: v3.2 + branch: v3.3 }] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: flink_version: ${{ matrix.flink_branches.flink }} connector_branch: ${{ matrix.flink_branches.branch }} jdk_version: ${{ matrix.flink_branches.jdk || '8, 11, 17, 21' }} - run_dependency_convergence: false \ No newline at end of file + run_dependency_convergence: false From 727327df7f0f2358c03e85022eae20079290a847 Mon Sep 17 00:00:00 2001 From: "pawel.leszczynski" Date: Thu, 14 Nov 2024 09:40:19 +0100 Subject: [PATCH 307/322] [FLINK-34466] Lineage interfaces for kafka connector (#130) Signed-off-by: Pawel Leszczynski --- .../lineage/DefaultKafkaDatasetFacet.java | 65 +++++++ .../DefaultKafkaDatasetIdentifier.java | 59 ++++++ .../lineage/DefaultTypeDatasetFacet.java | 44 +++++ .../kafka/lineage/KafkaDatasetFacet.java | 16 ++ .../lineage/KafkaDatasetFacetProvider.java | 16 ++ .../kafka/lineage/KafkaDatasetIdentifier.java | 30 +++ .../KafkaDatasetIdentifierProvider.java | 16 ++ .../connector/kafka/lineage/LineageUtil.java | 118 +++++++++++ .../kafka/lineage/TypeDatasetFacet.java | 11 ++ .../lineage/TypeDatasetFacetProvider.java | 16 ++ .../sink/KafkaRecordSerializationSchema.java | 5 +- ...KafkaRecordSerializationSchemaBuilder.java | 98 +++++++++- .../flink/connector/kafka/sink/KafkaSink.java | 54 +++++- .../connector/kafka/source/KafkaSource.java | 41 +++- .../subscriber/KafkaSubscriber.java | 4 + .../subscriber/PartitionSetSubscriber.java | 16 +- .../subscriber/TopicListSubscriber.java | 11 +- .../subscriber/TopicPatternSubscriber.java | 11 +- .../kafka/lineage/LineageUtilTest.java | 74 +++++++ ...aRecordSerializationSchemaBuilderTest.java | 146 ++++++++++++++ .../connector/kafka/sink/KafkaSinkTest.java | 144 ++++++++++++++ .../kafka/source/KafkaSourceTest.java | 183 ++++++++++++++++++ .../subscriber/KafkaSubscriberTest.java | 12 +- 23 files changed, 1178 insertions(+), 12 deletions(-) create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetFacet.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetIdentifier.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultTypeDatasetFacet.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacet.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacetProvider.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifier.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifierProvider.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacet.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacetProvider.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/lineage/LineageUtilTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTest.java diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetFacet.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetFacet.java new file mode 100644 index 000000000..e1c682345 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetFacet.java @@ -0,0 +1,65 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; + +import java.util.Objects; +import java.util.Properties; + +/** Default implementation of {@link KafkaDatasetFacet}. */ +@PublicEvolving +public class DefaultKafkaDatasetFacet implements KafkaDatasetFacet { + + public static final String KAFKA_FACET_NAME = "kafka"; + + private Properties properties; + + private final KafkaDatasetIdentifier topicIdentifier; + + public DefaultKafkaDatasetFacet(KafkaDatasetIdentifier topicIdentifier, Properties properties) { + this(topicIdentifier); + + this.properties = new Properties(); + KafkaPropertiesUtil.copyProperties(properties, this.properties); + } + + public DefaultKafkaDatasetFacet(KafkaDatasetIdentifier topicIdentifier) { + this.topicIdentifier = topicIdentifier; + } + + public void setProperties(Properties properties) { + this.properties = new Properties(); + KafkaPropertiesUtil.copyProperties(properties, this.properties); + } + + public Properties getProperties() { + return properties; + } + + public KafkaDatasetIdentifier getTopicIdentifier() { + return topicIdentifier; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultKafkaDatasetFacet that = (DefaultKafkaDatasetFacet) o; + return Objects.equals(properties, that.properties) + && Objects.equals(topicIdentifier, that.topicIdentifier); + } + + @Override + public int hashCode() { + return Objects.hash(properties, topicIdentifier); + } + + @Override + public String name() { + return KAFKA_FACET_NAME; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetIdentifier.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetIdentifier.java new file mode 100644 index 000000000..cd97b7ff4 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetIdentifier.java @@ -0,0 +1,59 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Objects; +import java.util.regex.Pattern; + +/** Default implementation of {@link KafkaDatasetIdentifier}. */ +@PublicEvolving +public class DefaultKafkaDatasetIdentifier implements KafkaDatasetIdentifier { + + @Nullable private final List topics; + @Nullable private final Pattern topicPattern; + + private DefaultKafkaDatasetIdentifier( + @Nullable List fixedTopics, @Nullable Pattern topicPattern) { + this.topics = fixedTopics; + this.topicPattern = topicPattern; + } + + public static DefaultKafkaDatasetIdentifier ofPattern(Pattern pattern) { + return new DefaultKafkaDatasetIdentifier(null, pattern); + } + + public static DefaultKafkaDatasetIdentifier ofTopics(List fixedTopics) { + return new DefaultKafkaDatasetIdentifier(fixedTopics, null); + } + + @Nullable + public List getTopics() { + return topics; + } + + @Nullable + public Pattern getTopicPattern() { + return topicPattern; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultKafkaDatasetIdentifier that = (DefaultKafkaDatasetIdentifier) o; + return Objects.equals(topics, that.topics) + && Objects.equals(topicPattern, that.topicPattern); + } + + @Override + public int hashCode() { + return Objects.hash(topics, topicPattern); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultTypeDatasetFacet.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultTypeDatasetFacet.java new file mode 100644 index 000000000..d9475d77a --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultTypeDatasetFacet.java @@ -0,0 +1,44 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; + +import java.util.Objects; + +/** Default implementation of {@link KafkaDatasetFacet}. */ +@PublicEvolving +public class DefaultTypeDatasetFacet implements TypeDatasetFacet { + + public static final String TYPE_FACET_NAME = "type"; + + private final TypeInformation typeInformation; + + public DefaultTypeDatasetFacet(TypeInformation typeInformation) { + this.typeInformation = typeInformation; + } + + public TypeInformation getTypeInformation() { + return typeInformation; + } + + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultTypeDatasetFacet that = (DefaultTypeDatasetFacet) o; + return Objects.equals(typeInformation, that.typeInformation); + } + + @Override + public int hashCode() { + return Objects.hash(typeInformation); + } + + @Override + public String name() { + return TYPE_FACET_NAME; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacet.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacet.java new file mode 100644 index 000000000..c0d3d0b73 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacet.java @@ -0,0 +1,16 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; + +import java.util.Properties; + +/** Facet definition to contain all Kafka specific information on Kafka sources and sinks. */ +@PublicEvolving +public interface KafkaDatasetFacet extends LineageDatasetFacet { + Properties getProperties(); + + KafkaDatasetIdentifier getTopicIdentifier(); + + void setProperties(Properties properties); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacetProvider.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacetProvider.java new file mode 100644 index 000000000..76fe41b82 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacetProvider.java @@ -0,0 +1,16 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; + +import java.util.Optional; + +/** Contains method to extract {@link KafkaDatasetFacet}. */ +@PublicEvolving +public interface KafkaDatasetFacetProvider { + + /** + * Returns a Kafka dataset facet or empty in case an implementing class is not able to identify + * a dataset. + */ + Optional getKafkaDatasetFacet(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifier.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifier.java new file mode 100644 index 000000000..19f7082e2 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifier.java @@ -0,0 +1,30 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Objects; +import java.util.regex.Pattern; + +/** Kafka dataset identifier which can contain either a list of topics or a topic pattern. */ +@PublicEvolving +public interface KafkaDatasetIdentifier { + @Nullable + List getTopics(); + + @Nullable + Pattern getTopicPattern(); + + /** + * Assigns lineage dataset's name which is topic pattern if it is present or comma separated + * list of topics. + */ + default String toLineageName() { + if (getTopicPattern() != null) { + return getTopicPattern().toString(); + } + return String.join(",", Objects.requireNonNull(getTopics())); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifierProvider.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifierProvider.java new file mode 100644 index 000000000..1389fea58 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifierProvider.java @@ -0,0 +1,16 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; + +import java.util.Optional; + +/** Contains method which allows extracting topic identifier. */ +@PublicEvolving +public interface KafkaDatasetIdentifierProvider { + + /** + * Gets Kafka dataset identifier or empty in case a class implementing is not able to extract + * dataset identifier. + */ + Optional getDatasetIdentifier(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java new file mode 100644 index 000000000..086303e09 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.streaming.api.lineage.LineageDataset; +import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; +import org.apache.flink.streaming.api.lineage.LineageVertex; +import org.apache.flink.streaming.api.lineage.SourceLineageVertex; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; + +/** Utility class with useful methods for managing lineage objects. */ +public class LineageUtil { + + private static final String KAFKA_DATASET_PREFIX = "kafka://"; + private static final String COMMA = ","; + private static final String SEMICOLON = ";"; + + public static LineageDataset datasetOf(String namespace, KafkaDatasetFacet kafkaDatasetFacet) { + return datasetOf(namespace, kafkaDatasetFacet, Collections.emptyList()); + } + + public static LineageDataset datasetOf( + String namespace, KafkaDatasetFacet kafkaDatasetFacet, TypeDatasetFacet typeFacet) { + return datasetOf(namespace, kafkaDatasetFacet, Collections.singletonList(typeFacet)); + } + + private static LineageDataset datasetOf( + String namespace, + KafkaDatasetFacet kafkaDatasetFacet, + List facets) { + return new LineageDataset() { + @Override + public String name() { + return kafkaDatasetFacet.getTopicIdentifier().toLineageName(); + } + + @Override + public String namespace() { + return namespace; + } + + @Override + public Map facets() { + Map facetMap = new HashMap<>(); + facetMap.put(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME, kafkaDatasetFacet); + facetMap.putAll( + facets.stream() + .collect( + Collectors.toMap(LineageDatasetFacet::name, item -> item))); + return facetMap; + } + }; + } + + public static String namespaceOf(Properties properties) { + String bootstrapServers = properties.getProperty("bootstrap.servers"); + + if (bootstrapServers == null) { + return KAFKA_DATASET_PREFIX; + } + + if (bootstrapServers.contains(COMMA)) { + bootstrapServers = bootstrapServers.split(COMMA)[0]; + } else if (bootstrapServers.contains(SEMICOLON)) { + bootstrapServers = bootstrapServers.split(SEMICOLON)[0]; + } + + return String.format(KAFKA_DATASET_PREFIX + bootstrapServers); + } + + public static SourceLineageVertex sourceLineageVertexOf(Collection datasets) { + return new SourceLineageVertex() { + @Override + public Boundedness boundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public List datasets() { + return datasets.stream().collect(Collectors.toList()); + } + }; + } + + public static LineageVertex lineageVertexOf(Collection datasets) { + return new LineageVertex() { + @Override + public List datasets() { + return datasets.stream().collect(Collectors.toList()); + } + }; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacet.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacet.java new file mode 100644 index 000000000..1e64f5819 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacet.java @@ -0,0 +1,11 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; + +/** Facet definition to contain type information of source and sink. */ +@PublicEvolving +public interface TypeDatasetFacet extends LineageDatasetFacet { + TypeInformation getTypeInformation(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacetProvider.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacetProvider.java new file mode 100644 index 000000000..016a1bb84 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacetProvider.java @@ -0,0 +1,16 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; + +import java.util.Optional; + +/** Contains method to extract {@link TypeDatasetFacet}. */ +@PublicEvolving +public interface TypeDatasetFacetProvider { + + /** + * Returns a type dataset facet or `Optional.empty` in case an implementing class is not able to + * resolve type. + */ + Optional getTypeDatasetFacet(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java index 9d081c755..f56a7da54 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java @@ -29,7 +29,10 @@ /** * A serialization schema which defines how to convert a value of type {@code T} to {@link - * ProducerRecord}. + * ProducerRecord}. {@link org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider} can + * be implemented to provide Kafka specific lineage metadata, while {@link + * org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider} can be implemented to provide + * lineage metadata with type information. * * @param the type of values being serialized */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java index e9fc413b2..0fba3a364 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java @@ -19,16 +19,32 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.DefaultTypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider; +import org.apache.flink.connector.kafka.source.KafkaSource; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import com.google.common.reflect.TypeToken; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.io.Serializable; +import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import java.util.OptionalInt; import java.util.function.Function; @@ -79,6 +95,7 @@ */ @PublicEvolving public class KafkaRecordSerializationSchemaBuilder { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); @Nullable private Function topicSelector; @Nullable private SerializationSchema valueSerializationSchema; @@ -122,7 +139,8 @@ public KafkaRecordSerializationSchemaBuilder setPartitioner( public KafkaRecordSerializationSchemaBuilder setTopic(String topic) { checkState(this.topicSelector == null, "Topic selector already set."); checkNotNull(topic); - this.topicSelector = new CachingTopicSelector<>((e) -> topic); + + this.topicSelector = new ConstantTopicSelector<>(topic); return this; } @@ -283,7 +301,29 @@ private void checkKeySerializerNotSet() { checkState(keySerializationSchema == null, "Key serializer already set."); } - private static class CachingTopicSelector implements Function, Serializable { + private static class ConstantTopicSelector + implements Function, Serializable, KafkaDatasetIdentifierProvider { + + private String topic; + + ConstantTopicSelector(String topic) { + this.topic = topic; + } + + @Override + public String apply(IN in) { + return topic; + } + + @Override + public Optional getDatasetIdentifier() { + return Optional.of( + DefaultKafkaDatasetIdentifier.ofTopics(Collections.singletonList(topic))); + } + } + + private static class CachingTopicSelector + implements Function, KafkaDatasetIdentifierProvider, Serializable { private static final int CACHE_RESET_SIZE = 5; private final Map cache; @@ -303,10 +343,21 @@ public String apply(IN in) { } return topic; } + + @Override + public Optional getDatasetIdentifier() { + if (topicSelector instanceof KafkaDatasetIdentifierProvider) { + return ((KafkaDatasetIdentifierProvider) topicSelector).getDatasetIdentifier(); + } else { + return Optional.empty(); + } + } } private static class KafkaRecordSerializationSchemaWrapper - implements KafkaRecordSerializationSchema { + implements KafkaDatasetFacetProvider, + KafkaRecordSerializationSchema, + TypeDatasetFacetProvider { private final SerializationSchema valueSerializationSchema; private final Function topicSelector; private final KafkaPartitioner partitioner; @@ -369,5 +420,46 @@ public ProducerRecord serialize( value, headerProvider != null ? headerProvider.getHeaders(element) : null); } + + @Override + public Optional getKafkaDatasetFacet() { + if (!(topicSelector instanceof KafkaDatasetIdentifierProvider)) { + LOG.info("Cannot identify topics. Not an TopicsIdentifierProvider"); + return Optional.empty(); + } + + Optional topicsIdentifier = + ((KafkaDatasetIdentifierProvider) (topicSelector)).getDatasetIdentifier(); + + if (!topicsIdentifier.isPresent()) { + LOG.info("No topics' identifiers provided"); + return Optional.empty(); + } + + return Optional.of(new DefaultKafkaDatasetFacet(topicsIdentifier.get())); + } + + @Override + public Optional getTypeDatasetFacet() { + if (this.valueSerializationSchema instanceof ResultTypeQueryable) { + return Optional.of( + new DefaultTypeDatasetFacet( + ((ResultTypeQueryable) this.valueSerializationSchema) + .getProducedType())); + } else { + // gets type information from serialize method signature + TypeToken serializationSchemaType = + TypeToken.of(valueSerializationSchema.getClass()); + Class parameterType = + serializationSchemaType + .resolveType(SerializationSchema.class.getTypeParameters()[0]) + .getRawType(); + if (parameterType != Object.class) { + return Optional.of( + new DefaultTypeDatasetFacet(TypeInformation.of(parameterType))); + } + } + return Optional.empty(); + } } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java index d5b1c3700..d3d3c89df 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java @@ -22,11 +22,22 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.Committer; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider; +import org.apache.flink.connector.kafka.lineage.LineageUtil; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.lineage.LineageVertex; +import org.apache.flink.streaming.api.lineage.LineageVertexProvider; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Collection; import java.util.Collections; +import java.util.Optional; import java.util.Properties; /** @@ -54,8 +65,9 @@ */ @PublicEvolving public class KafkaSink - implements TwoPhaseCommittingStatefulSink { - + implements LineageVertexProvider, + TwoPhaseCommittingStatefulSink { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSink.class); private final DeliveryGuarantee deliveryGuarantee; private final KafkaRecordSerializationSchema recordSerializer; @@ -132,4 +144,42 @@ public SimpleVersionedSerializer getWriterStateSerializer() { protected Properties getKafkaProducerConfig() { return kafkaProducerConfig; } + + @Override + public LineageVertex getLineageVertex() { + // enrich dataset facet with properties + Optional kafkaDatasetFacet; + if (recordSerializer instanceof KafkaDatasetFacetProvider) { + kafkaDatasetFacet = + ((KafkaDatasetFacetProvider) recordSerializer).getKafkaDatasetFacet(); + + if (!kafkaDatasetFacet.isPresent()) { + LOG.info("Provider did not return kafka dataset facet"); + return LineageUtil.sourceLineageVertexOf(Collections.emptyList()); + } + kafkaDatasetFacet.get().setProperties(this.kafkaProducerConfig); + } else { + LOG.info( + "recordSerializer does not implement KafkaDatasetFacetProvider: {}", + recordSerializer); + return LineageUtil.sourceLineageVertexOf(Collections.emptyList()); + } + + String namespace = LineageUtil.namespaceOf(kafkaProducerConfig); + + Optional typeDatasetFacet = Optional.empty(); + if (recordSerializer instanceof TypeDatasetFacetProvider) { + typeDatasetFacet = ((TypeDatasetFacetProvider) recordSerializer).getTypeDatasetFacet(); + } + + if (typeDatasetFacet.isPresent()) { + return LineageUtil.sourceLineageVertexOf( + Collections.singleton( + LineageUtil.datasetOf( + namespace, kafkaDatasetFacet.get(), typeDatasetFacet.get()))); + } + + return LineageUtil.sourceLineageVertexOf( + Collections.singleton(LineageUtil.datasetOf(namespace, kafkaDatasetFacet.get()))); + } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java index 54f5f856c..39302751c 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java @@ -33,6 +33,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.DefaultTypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; +import org.apache.flink.connector.kafka.lineage.LineageUtil; import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumStateSerializer; import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator; @@ -48,15 +53,20 @@ import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.lineage.LineageVertexProvider; +import org.apache.flink.streaming.api.lineage.SourceLineageVertex; import org.apache.flink.util.UserCodeClassLoader; import org.apache.flink.util.function.SerializableSupplier; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.io.IOException; import java.util.Collection; +import java.util.Collections; import java.util.Optional; import java.util.Properties; import java.util.function.Consumer; @@ -87,8 +97,10 @@ */ @PublicEvolving public class KafkaSource - implements Source, + implements LineageVertexProvider, + Source, ResultTypeQueryable { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); private static final long serialVersionUID = -8755372893283732098L; // Users can choose only one of the following ways to specify the topics to consume from. private final KafkaSubscriber subscriber; @@ -251,4 +263,31 @@ KafkaSubscriber getKafkaSubscriber() { OffsetsInitializer getStoppingOffsetsInitializer() { return stoppingOffsetsInitializer; } + + @Override + public SourceLineageVertex getLineageVertex() { + if (!(subscriber instanceof KafkaDatasetIdentifierProvider)) { + LOG.info("unable to determine topic identifier"); + return LineageUtil.sourceLineageVertexOf(Collections.emptyList()); + } + + Optional topicsIdentifier = + ((KafkaDatasetIdentifierProvider) subscriber).getDatasetIdentifier(); + + if (!topicsIdentifier.isPresent()) { + LOG.info("No topics' identifier returned from subscriber"); + return LineageUtil.sourceLineageVertexOf(Collections.emptyList()); + } + + DefaultKafkaDatasetFacet kafkaDatasetFacet = + new DefaultKafkaDatasetFacet(topicsIdentifier.get(), props); + + String namespace = LineageUtil.namespaceOf(props); + return LineageUtil.sourceLineageVertexOf( + Collections.singletonList( + LineageUtil.datasetOf( + namespace, + kafkaDatasetFacet, + new DefaultTypeDatasetFacet(getProducedType())))); + } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java index 1b819fb23..37de884af 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java @@ -39,6 +39,10 @@ * *

      The KafkaSubscriber provides a unified interface for the Kafka source to support all these * three types of subscribing mode. + * + *

      When implementing a subscriber, {@link + * org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider} can be implemented to + * provide lineage metadata with source topics. */ @PublicEvolving public interface KafkaSubscriber extends Serializable { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java index 3423b0f90..9cd50fb20 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.kafka.source.enumerator.subscriber; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; + import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.common.TopicPartition; @@ -26,13 +29,14 @@ import java.util.HashSet; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getTopicMetadata; /** A subscriber for a partition set. */ -class PartitionSetSubscriber implements KafkaSubscriber { +class PartitionSetSubscriber implements KafkaSubscriber, KafkaDatasetIdentifierProvider { private static final long serialVersionUID = 390970375272146036L; private static final Logger LOG = LoggerFactory.getLogger(PartitionSetSubscriber.class); private final Set subscribedPartitions; @@ -73,4 +77,14 @@ && partitionExistsInTopic( private boolean partitionExistsInTopic(TopicPartition partition, TopicDescription topic) { return topic.partitions().size() > partition.partition(); } + + @Override + public Optional getDatasetIdentifier() { + return Optional.of( + DefaultKafkaDatasetIdentifier.ofTopics( + subscribedPartitions.stream() + .map(TopicPartition::topic) + .distinct() + .collect(Collectors.toList()))); + } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java index b2ad844ab..e86ade0fa 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.kafka.source.enumerator.subscriber; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; + import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.common.TopicPartition; @@ -28,6 +31,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getTopicMetadata; @@ -36,7 +40,7 @@ * A subscriber to a fixed list of topics. The subscribed topics must have existed in the Kafka * cluster, otherwise an exception will be thrown. */ -class TopicListSubscriber implements KafkaSubscriber { +class TopicListSubscriber implements KafkaSubscriber, KafkaDatasetIdentifierProvider { private static final long serialVersionUID = -6917603843104947866L; private static final Logger LOG = LoggerFactory.getLogger(TopicListSubscriber.class); private final List topics; @@ -60,4 +64,9 @@ public Set getSubscribedTopicPartitions(AdminClient adminClient) return subscribedPartitions; } + + @Override + public Optional getDatasetIdentifier() { + return Optional.of(DefaultKafkaDatasetIdentifier.ofTopics(topics)); + } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java index 985ca7137..208959e27 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.kafka.source.enumerator.subscriber; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; + import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.common.TopicPartition; @@ -27,13 +30,14 @@ import java.util.HashSet; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.regex.Pattern; import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getTopicMetadata; /** A subscriber to a topic pattern. */ -class TopicPatternSubscriber implements KafkaSubscriber { +class TopicPatternSubscriber implements KafkaSubscriber, KafkaDatasetIdentifierProvider { private static final long serialVersionUID = -7471048577725467797L; private static final Logger LOG = LoggerFactory.getLogger(TopicPatternSubscriber.class); private final Pattern topicPattern; @@ -60,4 +64,9 @@ public Set getSubscribedTopicPartitions(AdminClient adminClient) return subscribedTopicPartitions; } + + @Override + public Optional getDatasetIdentifier() { + return Optional.of(DefaultKafkaDatasetIdentifier.ofPattern(topicPattern)); + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/lineage/LineageUtilTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/lineage/LineageUtilTest.java new file mode 100644 index 000000000..869399896 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/lineage/LineageUtilTest.java @@ -0,0 +1,74 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.streaming.api.lineage.LineageDataset; +import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; +import org.apache.flink.streaming.api.lineage.SourceLineageVertex; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.Properties; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link LineageUtil}. */ +public class LineageUtilTest { + @Test + public void testSourceLineageVertexOf() { + LineageDataset dataset = new TestingLineageDataset(); + SourceLineageVertex sourceLineageVertex = + LineageUtil.sourceLineageVertexOf(Collections.singletonList(dataset)); + + assertThat(sourceLineageVertex.boundedness()).isEqualTo(Boundedness.CONTINUOUS_UNBOUNDED); + assertThat(sourceLineageVertex.datasets()).containsExactly(dataset); + } + + @Test + public void testDatasetNamespaceOf() { + Properties properties = new Properties(); + properties.put("bootstrap.servers", "my-kafka-host"); + + assertThat(LineageUtil.namespaceOf(properties)).isEqualTo("kafka://my-kafka-host"); + } + + @Test + public void testDatasetNamespaceOfWithSemicolon() { + Properties properties = new Properties(); + properties.put("bootstrap.servers", "my-kafka-host1;my-kafka-host2"); + + assertThat(LineageUtil.namespaceOf(properties)).isEqualTo("kafka://my-kafka-host1"); + } + + @Test + public void testDatasetNamespaceOfWithComma() { + Properties properties = new Properties(); + properties.put("bootstrap.servers", "my-kafka-host1,my-kafka-host2"); + + assertThat(LineageUtil.namespaceOf(properties)).isEqualTo("kafka://my-kafka-host1"); + } + + @Test + public void testDatasetNamespaceWhenNoBootstrapServersProperty() { + Properties properties = new Properties(); + assertThat(LineageUtil.namespaceOf(properties)).isEqualTo("kafka://"); + } + + private static class TestingLineageDataset implements LineageDataset { + @Override + public String name() { + return null; + } + + @Override + public String namespace() { + return null; + } + + @Override + public Map facets() { + return null; + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java index 701f9c8aa..4d1437288 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java @@ -19,6 +19,15 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider; import org.apache.flink.connector.testutils.formats.DummyInitializationContext; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.util.TestLogger; @@ -31,6 +40,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.Before; import org.junit.Test; @@ -40,6 +50,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; @@ -53,6 +64,13 @@ public class KafkaRecordSerializationSchemaBuilderTest extends TestLogger { private static Map configurableConfiguration; private static Map configuration; + + private interface TestingTopicSelector + extends TopicSelector, KafkaDatasetIdentifierProvider {} + + private interface SerializationSchemaWithResultQueryable + extends SerializationSchema, ResultTypeQueryable {} + private static boolean isKeySerializer; @Before @@ -256,6 +274,134 @@ public void testSerializeRecordWithTimestamp() { assertThat(recordWithInvalidTimestamp.timestamp()).isNull(); } + @Test + public void testGetLineageDatasetFacetsWhenTopicSelectorNotKafkaTopicsIdentifierProvider() { + SerializationSchema serializationSchema = new SimpleStringSchema(); + KafkaRecordSerializationSchema schema = + KafkaRecordSerializationSchema.builder() + .setTopicSelector((TopicSelector) o -> DEFAULT_TOPIC) + .setValueSerializationSchema(serializationSchema) + .setKeySerializationSchema(serializationSchema) + .build(); + + assertThat(schema) + .asInstanceOf(InstanceOfAssertFactories.type(KafkaDatasetFacetProvider.class)) + .returns(Optional.empty(), KafkaDatasetFacetProvider::getKafkaDatasetFacet); + } + + @Test + public void testGetLineageDatasetFacetsWhenNoTopicsIdentifiersFound() { + SerializationSchema serializationSchema = new SimpleStringSchema(); + KafkaRecordSerializationSchema schema = + KafkaRecordSerializationSchema.builder() + .setTopicSelector( + new TestingTopicSelector() { + @Override + public Optional + getDatasetIdentifier() { + return Optional.empty(); + } + + @Override + public String apply(Object o) { + return DEFAULT_TOPIC; + } + }) + .setValueSerializationSchema(serializationSchema) + .setKeySerializationSchema(serializationSchema) + .build(); + assertThat(schema) + .asInstanceOf(InstanceOfAssertFactories.type(KafkaDatasetFacetProvider.class)) + .returns(Optional.empty(), KafkaDatasetFacetProvider::getKafkaDatasetFacet); + } + + @Test + public void testGetLineageDatasetFacetsValueSerializationSchemaIsResultTypeQueryable() { + TypeInformation stringTypeInformation = TypeInformation.of(String.class); + SerializationSchemaWithResultQueryable serializationSchema = + new SerializationSchemaWithResultQueryable() { + + @Override + public TypeInformation getProducedType() { + return stringTypeInformation; + } + + @Override + public byte[] serialize(String o) { + return new byte[0]; + } + }; + + KafkaRecordSerializationSchema schema = + KafkaRecordSerializationSchema.builder() + .setTopicSelector( + new TestingTopicSelector() { + @Override + public Optional + getDatasetIdentifier() { + return Optional.of( + DefaultKafkaDatasetIdentifier.ofTopics( + Arrays.asList("topic1", "topic2"))); + } + + @Override + public String apply(Object o) { + return DEFAULT_TOPIC; + } + }) + .setValueSerializationSchema(serializationSchema) + .setKeySerializationSchema(serializationSchema) + .build(); + + Optional kafkaDatasetFacet = + ((KafkaDatasetFacetProvider) schema).getKafkaDatasetFacet(); + + assertThat(kafkaDatasetFacet).isPresent(); + assertThat(kafkaDatasetFacet.get().getTopicIdentifier().getTopics()) + .containsExactly("topic1", "topic2"); + assertThat(((TypeDatasetFacetProvider) schema).getTypeDatasetFacet()) + .isPresent() + .get() + .extracting(TypeDatasetFacet::getTypeInformation) + .isEqualTo(stringTypeInformation); + } + + @Test + public void testGetLineageDatasetFacets() { + KafkaRecordSerializationSchema schema = + KafkaRecordSerializationSchema.builder() + .setTopicSelector( + new TestingTopicSelector() { + @Override + public Optional + getDatasetIdentifier() { + return Optional.of( + DefaultKafkaDatasetIdentifier.ofTopics( + Arrays.asList("topic1", "topic2"))); + } + + @Override + public String apply(Object o) { + return DEFAULT_TOPIC; + } + }) + .setValueSerializationSchema(new SimpleStringSchema()) + .setKeySerializationSchema(new SimpleStringSchema()) + .build(); + + Optional kafkaDatasetFacet = + ((KafkaDatasetFacetProvider) schema).getKafkaDatasetFacet(); + + assertThat(kafkaDatasetFacet).isPresent(); + assertThat(kafkaDatasetFacet.get().getTopicIdentifier().getTopics()) + .containsExactly("topic1", "topic2"); + assertThat(((TypeDatasetFacetProvider) schema).getTypeDatasetFacet()) + .isPresent() + .get() + .extracting(TypeDatasetFacet::getTypeInformation) + .isEqualTo(BasicTypeInfo.STRING_TYPE_INFO); + } + private static void assertOnlyOneSerializerAllowed( List< Function< diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java new file mode 100644 index 000000000..1efb6ec7d --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java @@ -0,0 +1,144 @@ +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.DefaultTypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider; +import org.apache.flink.streaming.api.lineage.LineageVertex; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.jetbrains.annotations.Nullable; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Optional; +import java.util.Properties; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link KafkaSink}. */ +public class KafkaSinkTest { + + Properties kafkaProperties; + + @BeforeEach + void setup() { + kafkaProperties = new Properties(); + kafkaProperties.put("bootstrap.servers", "host1;host2"); + } + + @Test + public void testGetLineageVertexWhenSerializerNotAnKafkaDatasetFacetProvider() { + KafkaRecordSerializationSchema recordSerializer = + new KafkaRecordSerializationSchemaWithoutKafkaDatasetProvider(); + KafkaSink sink = + new KafkaSink( + DeliveryGuarantee.EXACTLY_ONCE, new Properties(), "", recordSerializer); + + assertThat(sink.getLineageVertex().datasets()).isEmpty(); + } + + @Test + public void testGetLineageVertexWhenNoKafkaDatasetFacetReturnedFromSerializer() { + KafkaRecordSerializationSchema recordSerializer = + new KafkaRecordSerializationSchemaWithEmptyKafkaDatasetProvider(); + + KafkaSink sink = + new KafkaSink( + DeliveryGuarantee.EXACTLY_ONCE, new Properties(), "", recordSerializer); + + assertThat(sink.getLineageVertex().datasets()).isEmpty(); + } + + @Test + public void testGetLineageVertex() { + KafkaRecordSerializationSchema recordSerializer = + new TestingKafkaRecordSerializationSchema(); + + KafkaSink sink = + new KafkaSink( + DeliveryGuarantee.EXACTLY_ONCE, kafkaProperties, "", recordSerializer); + + LineageVertex lineageVertex = sink.getLineageVertex(); + + assertThat(lineageVertex.datasets().get(0).namespace()).isEqualTo("kafka://host1"); + assertThat(lineageVertex.datasets().get(0).name()).isEqualTo("topic1"); + + assertThat( + lineageVertex + .datasets() + .get(0) + .facets() + .get(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME)) + .hasFieldOrPropertyWithValue("properties", kafkaProperties) + .hasFieldOrPropertyWithValue( + "topicIdentifier", + DefaultKafkaDatasetIdentifier.ofTopics( + Collections.singletonList("topic1"))); + + assertThat( + lineageVertex + .datasets() + .get(0) + .facets() + .get(DefaultTypeDatasetFacet.TYPE_FACET_NAME)) + .hasFieldOrPropertyWithValue("typeInformation", TypeInformation.of(String.class)); + } + + private static class KafkaRecordSerializationSchemaWithoutKafkaDatasetProvider + implements KafkaRecordSerializationSchema { + @Nullable + @Override + public ProducerRecord serialize( + Object element, KafkaSinkContext context, Long timestamp) { + return null; + } + } + + private static class KafkaRecordSerializationSchemaWithEmptyKafkaDatasetProvider + implements KafkaRecordSerializationSchema, KafkaDatasetFacetProvider { + @Nullable + @Override + public ProducerRecord serialize( + Object element, KafkaSinkContext context, Long timestamp) { + return null; + } + + @Override + public Optional getKafkaDatasetFacet() { + return Optional.empty(); + } + } + + private static class TestingKafkaRecordSerializationSchema + implements KafkaRecordSerializationSchema, + KafkaDatasetFacetProvider, + TypeDatasetFacetProvider { + + @Override + public Optional getKafkaDatasetFacet() { + return Optional.of( + new DefaultKafkaDatasetFacet( + DefaultKafkaDatasetIdentifier.ofTopics( + Collections.singletonList("topic1")))); + } + + @Nullable + @Override + public ProducerRecord serialize( + Object element, KafkaSinkContext context, Long timestamp) { + return null; + } + + @Override + public Optional getTypeDatasetFacet() { + return Optional.of(new DefaultTypeDatasetFacet(TypeInformation.of(String.class))); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTest.java new file mode 100644 index 000000000..259668c5d --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTest.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.source; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.DefaultTypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.streaming.api.lineage.LineageDataset; +import org.apache.flink.streaming.api.lineage.LineageVertex; +import org.apache.flink.util.Collector; + +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link KafkaSource}. */ +public class KafkaSourceTest { + Properties kafkaProperties; + + @BeforeEach + void setup() { + kafkaProperties = new Properties(); + kafkaProperties.put("bootstrap.servers", "host1;host2"); + } + + @Test + public void testGetLineageVertexWhenSubscriberNotAnKafkaDatasetFacetProvider() { + KafkaSource source = + new KafkaSourceBuilder() + .setKafkaSubscriber( + new KafkaSubscriber() { + @Override + public Set getSubscribedTopicPartitions( + AdminClient adminClient) { + return null; + } + }) + .setProperties(kafkaProperties) + .setGroupId("") + .setDeserializer( + new KafkaRecordDeserializationSchema() { + @Override + public TypeInformation getProducedType() { + return null; + } + + @Override + public void deserialize( + ConsumerRecord record, + Collector out) + throws IOException {} + }) + .setUnbounded(OffsetsInitializer.committedOffsets()) + .build(); + + assertThat(source.getLineageVertex()) + .extracting(LineageVertex::datasets) + .asList() + .isEmpty(); + } + + @Test + public void testGetLineageVertexWhenNoKafkaTopicsIdentifier() { + KafkaSource source = + new KafkaSourceBuilder() + .setKafkaSubscriber( + new TestingKafkaSubscriber() { + @Override + public Optional + getDatasetIdentifier() { + return Optional.empty(); + } + }) + .setProperties(kafkaProperties) + .setGroupId("") + .setDeserializer( + new KafkaRecordDeserializationSchema() { + @Override + public void deserialize( + ConsumerRecord record, + Collector out) + throws IOException {} + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(String.class); + } + }) + .setUnbounded(OffsetsInitializer.committedOffsets()) + .build(); + assertThat(source.getLineageVertex()) + .extracting(LineageVertex::datasets) + .asList() + .isEmpty(); + } + + @Test + public void testGetLineageVertex() { + TypeInformation typeInformation = TypeInformation.of(String.class); + KafkaSource source = + new KafkaSourceBuilder() + .setKafkaSubscriber(new TestingKafkaSubscriber()) + .setProperties(kafkaProperties) + .setGroupId("") + .setDeserializer( + new KafkaRecordDeserializationSchema() { + @Override + public void deserialize( + ConsumerRecord record, + Collector out) + throws IOException {} + + @Override + public TypeInformation getProducedType() { + return typeInformation; + } + }) + .setUnbounded(OffsetsInitializer.committedOffsets()) + .build(); + + LineageVertex lineageVertex = source.getLineageVertex(); + assertThat(lineageVertex.datasets()).hasSize(1); + LineageDataset dataset = lineageVertex.datasets().get(0); + + assertThat(dataset.namespace()).isEqualTo("kafka://host1"); + assertThat(dataset.name()).isEqualTo("topic1"); + + assertThat(dataset.facets()).containsKey(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME); + DefaultKafkaDatasetFacet kafkaFacet = + (DefaultKafkaDatasetFacet) + dataset.facets().get(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME); + + assertThat(kafkaFacet.getProperties()).containsEntry("bootstrap.servers", "host1;host2"); + + assertThat(dataset.facets()).containsKey(DefaultTypeDatasetFacet.TYPE_FACET_NAME); + assertThat(dataset.facets().get(DefaultTypeDatasetFacet.TYPE_FACET_NAME)) + .hasFieldOrPropertyWithValue("typeInformation", TypeInformation.of(String.class)); + } + + private static class TestingKafkaSubscriber + implements KafkaSubscriber, KafkaDatasetIdentifierProvider { + @Override + public Optional getDatasetIdentifier() { + return Optional.of( + DefaultKafkaDatasetIdentifier.ofTopics(Collections.singletonList("topic1"))); + } + + @Override + public Set getSubscribedTopicPartitions(AdminClient adminClient) { + return null; + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java index 258c1c0ab..4c5a50243 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java @@ -18,6 +18,8 @@ package org.apache.flink.connector.kafka.source.enumerator.subscriber; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv; import org.apache.kafka.clients.admin.AdminClient; @@ -71,6 +73,8 @@ public void testTopicListSubscriber() { new HashSet<>(KafkaSourceTestEnv.getPartitionsForTopics(topics)); assertThat(subscribedPartitions).isEqualTo(expectedSubscribedPartitions); + assertThat(((KafkaDatasetIdentifierProvider) subscriber).getDatasetIdentifier().get()) + .isEqualTo(DefaultKafkaDatasetIdentifier.ofTopics(topics)); } @Test @@ -86,8 +90,8 @@ public void testNonExistingTopic() { @Test public void testTopicPatternSubscriber() { - KafkaSubscriber subscriber = - KafkaSubscriber.getTopicPatternSubscriber(Pattern.compile("pattern.*")); + Pattern pattern = Pattern.compile("pattern.*"); + KafkaSubscriber subscriber = KafkaSubscriber.getTopicPatternSubscriber(pattern); final Set subscribedPartitions = subscriber.getSubscribedTopicPartitions(adminClient); @@ -96,6 +100,8 @@ public void testTopicPatternSubscriber() { KafkaSourceTestEnv.getPartitionsForTopics(Collections.singleton(TOPIC2))); assertThat(subscribedPartitions).isEqualTo(expectedSubscribedPartitions); + assertThat(((KafkaDatasetIdentifierProvider) subscriber).getDatasetIdentifier().get()) + .isEqualTo(DefaultKafkaDatasetIdentifier.ofPattern(pattern)); } @Test @@ -111,6 +117,8 @@ public void testPartitionSetSubscriber() { subscriber.getSubscribedTopicPartitions(adminClient); assertThat(subscribedPartitions).isEqualTo(partitions); + assertThat(((KafkaDatasetIdentifierProvider) subscriber).getDatasetIdentifier().get()) + .isEqualTo(DefaultKafkaDatasetIdentifier.ofTopics(topics)); } @Test From 59baacccdc4f58ac6a0511ccda79050b16341d00 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Mon, 25 Nov 2024 17:10:52 +0100 Subject: [PATCH 308/322] [FLINK-35109] Update weekly CI to verify 3.4 release branch --- .github/workflows/weekly.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index cea005757..3d7341cc4 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -38,6 +38,9 @@ jobs: }, { flink: 1.20.0, branch: v3.3 + }, { + flink: 1.20.0, + branch: v3.4 }] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: From f6a077a9dd8d1d5e43fc545cc9baab227d8438a0 Mon Sep 17 00:00:00 2001 From: Mingliang Liu Date: Tue, 3 Dec 2024 05:11:18 -0800 Subject: [PATCH 309/322] [FLINK-36780] Kafka source disable partition discovery unexpectedly (#136) --- .../kafka/source/KafkaSourceBuilder.java | 7 +++--- .../kafka/source/KafkaSourceBuilderTest.java | 23 +++++++++++++++++++ 2 files changed, 26 insertions(+), 4 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java index 78a4b0b60..0709afe0b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java @@ -474,10 +474,9 @@ private void parseAndSetRequiredProperties() { true); // If the source is bounded, do not run periodic partition discovery. - maybeOverride( - KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), - "-1", - boundedness == Boundedness.BOUNDED); + if (boundedness == Boundedness.BOUNDED) { + maybeOverride(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "-1", true); + } // If the client id prefix is not set, reuse the consumer group id as the client id prefix, // or generate a random string if consumer group id is not specified. diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java index 2829f01e0..ca777bc73 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java @@ -217,6 +217,29 @@ public void testSettingInvalidCustomDeserializers( .hasMessageContaining(expectedError); } + @Test + public void testDefaultPartitionDiscovery() { + final KafkaSource kafkaSource = getBasicBuilder().build(); + // Commit on checkpoint and auto commit should be disabled because group.id is not specified + assertThat( + kafkaSource + .getConfiguration() + .get(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS)) + .isEqualTo(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.defaultValue()); + } + + @Test + public void testPeriodPartitionDiscovery() { + final KafkaSource kafkaSource = + getBasicBuilder().setBounded(OffsetsInitializer.latest()).build(); + // Commit on checkpoint and auto commit should be disabled because group.id is not specified + assertThat( + kafkaSource + .getConfiguration() + .get(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS)) + .isEqualTo(-1L); + } + private KafkaSourceBuilder getBasicBuilder() { return new KafkaSourceBuilder() .setBootstrapServers("testServer") From 20809dae282751182dcc9d82c758ca70d9192587 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Fri, 7 Feb 2025 21:54:15 +0100 Subject: [PATCH 310/322] [hotfix] Remote dead code --- .../sink/TransactionsToAbortChecker.java | 95 ------------------- .../sink/TransactionToAbortCheckerTest.java | 89 ----------------- 2 files changed, 184 deletions(-) delete mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionsToAbortChecker.java delete mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionsToAbortChecker.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionsToAbortChecker.java deleted file mode 100644 index 2d79b3506..000000000 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionsToAbortChecker.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.kafka.sink; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -class TransactionsToAbortChecker { - - private static final int MINIMUM_CHECKPOINT_OFFSET = 1; - - private final int numberOfParallelSubtasks; - private final Map subtaskIdCheckpointOffsetMapping; - private final int subtaskId; - - TransactionsToAbortChecker( - int numberOfParallelSubtasks, - Map subtaskIdCheckpointOffsetMapping, - int subtaskId) { - this.subtaskId = subtaskId; - this.numberOfParallelSubtasks = numberOfParallelSubtasks; - this.subtaskIdCheckpointOffsetMapping = subtaskIdCheckpointOffsetMapping; - } - - /** - * Iterates through all open transactions and filters for the following attributes. - * - *
        - *
      1. If the minimum checkpointOffset for the subtask is {@link #MINIMUM_CHECKPOINT_OFFSET} - * and [openSubtaskId % {@link #numberOfParallelSubtasks} == {@link #subtaskId}] return - * all transactions from this subtask - *
      2. If the subtaskId is part of the recovered states {@link - * #subtaskIdCheckpointOffsetMapping} and the checkpointOffset >= the recovered offSet - * also return this transactionalId - *
      - * - * @param openTransactions Mapping of {subtaskId: {checkpointOffset: transactionalId}} - * @return transactionalIds which must be aborted - */ - public List getTransactionsToAbort(Map> openTransactions) { - final List transactionalIdsToAbort = new ArrayList<>(); - for (final Map.Entry> subtaskOffsetMapping : - openTransactions.entrySet()) { - final Map checkpointOffsetTransactionalIdMapping = - subtaskOffsetMapping.getValue(); - // All transactions from this subtask have been closed - if (checkpointOffsetTransactionalIdMapping.isEmpty()) { - continue; - } - // Abort all open transactions if checkpointOffset 0 is open implying that no checkpoint - // finished. - // Cut the transactions in ranges to speed up abort process - if (Collections.min(checkpointOffsetTransactionalIdMapping.keySet()) - == MINIMUM_CHECKPOINT_OFFSET - && subtaskOffsetMapping.getKey() % numberOfParallelSubtasks == subtaskId) { - transactionalIdsToAbort.addAll(checkpointOffsetTransactionalIdMapping.values()); - } else { - // Check all open transactions against recovered ones and close if the open - // transaction is equal or higher to the offset - for (final Map.Entry offsetTransactionId : - checkpointOffsetTransactionalIdMapping.entrySet()) { - if (!hasSameSubtaskWithHigherCheckpoint( - subtaskOffsetMapping.getKey(), offsetTransactionId.getKey())) { - continue; - } - transactionalIdsToAbort.add(offsetTransactionId.getValue()); - } - } - } - return transactionalIdsToAbort; - } - - private boolean hasSameSubtaskWithHigherCheckpoint( - int openSubtaskIndex, long openCheckpointOffset) { - return subtaskIdCheckpointOffsetMapping.containsKey(openSubtaskIndex) - && subtaskIdCheckpointOffsetMapping.get(openSubtaskIndex) <= openCheckpointOffset; - } -} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java deleted file mode 100644 index 897a85911..000000000 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.kafka.sink; - -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link TransactionsToAbortChecker}. */ -public class TransactionToAbortCheckerTest extends TestLogger { - - public static final String ABORT = "abort"; - public static final String KEEP = "keep"; - - @Test - public void testMustAbortTransactionsWithSameSubtaskIdAndHigherCheckpointOffset() { - Map offsetMapping = new HashMap<>(2); - offsetMapping.put(0, 1L); - offsetMapping.put(2, 3L); - final TransactionsToAbortChecker checker = - new TransactionsToAbortChecker(2, offsetMapping, 0); - - // abort recovered subtasksId with equal or higher checkpoint offset - final Map> openTransactions = new HashMap<>(3); - final Map subtask0 = new HashMap<>(); - subtask0.put(1L, ABORT); - subtask0.put(2L, ABORT); - openTransactions.put(0, subtask0); - final Map subtask2 = new HashMap<>(); - subtask2.put(3L, ABORT); - subtask2.put(4L, ABORT); - openTransactions.put(2, subtask2); - final Map subtask3 = new HashMap<>(); - subtask3.put(3L, KEEP); - subtask3.put(4L, KEEP); - openTransactions.put(3, subtask3); - - final List transactionsToAbort = checker.getTransactionsToAbort(openTransactions); - assertThat(transactionsToAbort).hasSize(4); - assertThatAbortCorrectTransaction(transactionsToAbort); - } - - @Test - public void testMustAbortTransactionsIfLowestCheckpointOffsetIsMinimumOffset() { - final TransactionsToAbortChecker checker = - new TransactionsToAbortChecker(2, Collections.singletonMap(0, 1L), 0); - - // abort recovered subtasksId with equal or higher checkpoint offset - final Map> openTransactions = new HashMap<>(5); - final Map subtask0 = new HashMap<>(); - subtask0.put(1L, ABORT); - subtask0.put(2L, ABORT); - openTransactions.put(0, subtask0); - openTransactions.put(2, Collections.singletonMap(1L, ABORT)); - openTransactions.put(3, Collections.singletonMap(1L, KEEP)); - openTransactions.put(4, Collections.singletonMap(1L, ABORT)); - openTransactions.put(5, Collections.singletonMap(1L, KEEP)); - - final List transactionsToAbort = checker.getTransactionsToAbort(openTransactions); - assertThat(transactionsToAbort).hasSize(4); - assertThatAbortCorrectTransaction(transactionsToAbort); - } - - private static void assertThatAbortCorrectTransaction(List abortedTransactions) { - assertThat(abortedTransactions.stream().allMatch(t -> t.equals(ABORT))).isTrue(); - } -} From 8bb953df7eea371870233e380480a777f6c28fc6 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Fri, 7 Feb 2025 22:15:10 +0100 Subject: [PATCH 311/322] [FLINK-37281] Refactor KafkaSinkITCase Add more test coverage for unchained cases and separate the behavioral components from data capture and assertions. Also reduces the need to convey information with fields. --- .../connector/kafka/sink/KafkaSinkITCase.java | 467 ++++++++++-------- 1 file changed, 263 insertions(+), 204 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java index d3bef5425..3ddc4b154 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java @@ -17,15 +17,18 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; @@ -43,68 +46,80 @@ import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; -import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.test.util.TestUtils; -import org.apache.flink.testutils.junit.SharedObjects; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.testutils.junit.SharedObjectsExtension; import org.apache.flink.testutils.junit.SharedReference; import org.apache.flink.util.TestLogger; +import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; + import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.CreateTopicsResult; import org.apache.kafka.clients.admin.DeleteTopicsResult; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Nested; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.containers.Network; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import org.testcontainers.utility.DockerImageName; import javax.annotation.Nullable; import java.io.File; -import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.UUID; +import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import java.util.stream.LongStream; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.assertj.core.api.Assertions.assertThat; /** Tests for using KafkaSink writing to a Kafka cluster. */ +@Testcontainers public class KafkaSinkITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkITCase.class); @@ -115,23 +130,27 @@ public class KafkaSinkITCase extends TestLogger { private static AdminClient admin; private String topic; - private SharedReference emittedRecordsCount; - private SharedReference emittedRecordsWithCheckpoint; - private SharedReference failed; - private SharedReference lastCheckpointedRecord; - @ClassRule + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(2) + .setNumberSlotsPerTaskManager(8) + .setConfiguration(new Configuration()) + .build()); + + @Container public static final KafkaContainer KAFKA_CONTAINER = createKafkaContainer(KafkaSinkITCase.class) .withEmbeddedZookeeper() .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); - @Rule public final SharedObjects sharedObjects = SharedObjects.create(); - - @Rule public final TemporaryFolder temp = new TemporaryFolder(); + @RegisterExtension + public final SharedObjectsExtension sharedObjects = SharedObjectsExtension.create(); - @BeforeClass + @BeforeAll public static void setupAdmin() { Map properties = new HashMap<>(); properties.put( @@ -140,34 +159,29 @@ public static void setupAdmin() { admin = AdminClient.create(properties); } - @AfterClass + @AfterAll public static void teardownAdmin() { admin.close(); } - @Before - public void setUp() throws ExecutionException, InterruptedException, TimeoutException { - emittedRecordsCount = sharedObjects.add(new AtomicLong()); - emittedRecordsWithCheckpoint = sharedObjects.add(new AtomicLong()); - failed = sharedObjects.add(new AtomicBoolean(false)); - lastCheckpointedRecord = sharedObjects.add(new AtomicLong(0)); + @BeforeEach + public void setUp() throws ExecutionException, InterruptedException { topic = UUID.randomUUID().toString(); createTestTopic(topic, 1, TOPIC_REPLICATION_FACTOR); } - @After + @AfterEach public void tearDown() throws ExecutionException, InterruptedException, TimeoutException { checkProducerLeak(); deleteTestTopic(topic); } /** Integration test based on connector testing framework. */ + @SuppressWarnings("unused") @Nested class IntegrationTests extends SinkTestSuiteBase { // Defines test environment on Flink MiniCluster - @SuppressWarnings("unused") - @TestEnv - MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); // Defines external system @TestExternalSystem @@ -178,14 +192,12 @@ class IntegrationTests extends SinkTestSuiteBase { DockerImageName.parse(DockerImageVersions.KAFKA))) .build(); - @SuppressWarnings("unused") @TestSemantics CheckpointingMode[] semantics = new CheckpointingMode[] { CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE }; - @SuppressWarnings("unused") @TestContext KafkaSinkExternalContextFactory sinkContext = new KafkaSinkExternalContextFactory(kafka.getContainer(), Collections.emptyList()); @@ -193,97 +205,108 @@ class IntegrationTests extends SinkTestSuiteBase { @Test public void testWriteRecordsToKafkaWithAtLeastOnceGuarantee() throws Exception { - writeRecordsToKafka(DeliveryGuarantee.AT_LEAST_ONCE, emittedRecordsCount); + writeRecordsToKafka(DeliveryGuarantee.AT_LEAST_ONCE); } @Test public void testWriteRecordsToKafkaWithNoneGuarantee() throws Exception { - writeRecordsToKafka(DeliveryGuarantee.NONE, emittedRecordsCount); + writeRecordsToKafka(DeliveryGuarantee.NONE); } - @Test - public void testWriteRecordsToKafkaWithExactlyOnceGuarantee() throws Exception { - writeRecordsToKafka(DeliveryGuarantee.EXACTLY_ONCE, emittedRecordsWithCheckpoint); + @ParameterizedTest(name = "chained={0}") + @ValueSource(booleans = {true, false}) + public void testWriteRecordsToKafkaWithExactlyOnceGuarantee(boolean chained) throws Exception { + writeRecordsToKafka(DeliveryGuarantee.EXACTLY_ONCE, chained); } @Test public void testRecoveryWithAtLeastOnceGuarantee() throws Exception { - testRecoveryWithAssertion( - DeliveryGuarantee.AT_LEAST_ONCE, - 1, - (records) -> assertThat(records).contains(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L)); + testRecoveryWithAssertion(DeliveryGuarantee.AT_LEAST_ONCE, 1); } - @Test - public void testRecoveryWithExactlyOnceGuarantee() throws Exception { - testRecoveryWithAssertion( - DeliveryGuarantee.EXACTLY_ONCE, - 1, - (records) -> - assertThat(records) - .contains( - (LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new)))); + @ParameterizedTest(name = "chained={0}") + @ValueSource(booleans = {true, false}) + public void testRecoveryWithExactlyOnceGuarantee(boolean chained) throws Exception { + testRecoveryWithAssertion(DeliveryGuarantee.EXACTLY_ONCE, 1, chained); } - @Test - public void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints() throws Exception { - testRecoveryWithAssertion( - DeliveryGuarantee.EXACTLY_ONCE, - 2, - (records) -> - assertThat(records) - .contains( - LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new))); + @ParameterizedTest(name = "chained={0}") + @ValueSource(booleans = {true, false}) + public void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints(boolean chained) + throws Exception { + testRecoveryWithAssertion(DeliveryGuarantee.EXACTLY_ONCE, 2, chained); } - @Test - public void testAbortTransactionsOfPendingCheckpointsAfterFailure() throws Exception { + @ParameterizedTest(name = "chained={0}") + @ValueSource(booleans = {true, false}) + public void testAbortTransactionsOfPendingCheckpointsAfterFailure( + boolean chained, + @TempDir File checkpointDir, + @InjectMiniCluster MiniCluster miniCluster, + @InjectClusterClient ClusterClient clusterClient) + throws Exception { // Run a first job failing during the async phase of a checkpoint to leave some // lingering transactions - final Configuration config = new Configuration(); + final Configuration config = createConfiguration(4); config.setString(StateBackendOptions.STATE_BACKEND, "filesystem"); - final File checkpointDir = temp.newFolder(); config.setString( CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); config.set( ExecutionCheckpointingOptions.EXTERNALIZED_CHECKPOINT, CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); config.set(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS, 2); + JobID firstJobId = null; + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentLinkedDeque<>()); try { - executeWithMapper(new FailAsyncCheckpointMapper(1), config, "firstPrefix"); + firstJobId = + executeWithMapper( + new FailAsyncCheckpointMapper(1), + checkpointedRecords, + config, + chained, + "firstPrefix", + clusterClient); } catch (Exception e) { - assertThat(e.getCause().getCause().getMessage()) - .contains("Exceeded checkpoint tolerable failure"); + assertThat(e).hasStackTraceContaining("Exceeded checkpoint tolerable failure"); } - final File completedCheckpoint = TestUtils.getMostRecentCompletedCheckpoint(checkpointDir); + final Optional completedCheckpoint = + CommonTestUtils.getLatestCompletedCheckpointPath(firstJobId, miniCluster); - config.set(SavepointConfigOptions.SAVEPOINT_PATH, completedCheckpoint.toURI().toString()); + assertThat(completedCheckpoint).isPresent(); + config.set(SavepointConfigOptions.SAVEPOINT_PATH, completedCheckpoint.get()); // Run a second job which aborts all lingering transactions and new consumer should // immediately see the newly written records - failed.get().set(true); + SharedReference failed = sharedObjects.add(new AtomicBoolean(true)); executeWithMapper( - new FailingCheckpointMapper(failed, lastCheckpointedRecord), config, "newPrefix"); - final List> collectedRecords = - drainAllRecordsFromTopic(topic, true); - assertThat(deserializeValues(collectedRecords)) - .contains( - LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new)); + new FailingCheckpointMapper(failed), + checkpointedRecords, + config, + chained, + "newPrefix", + clusterClient); + final List committedRecords = + deserializeValues(drainAllRecordsFromTopic(topic, true)); + assertThat(committedRecords).containsExactlyInAnyOrderElementsOf(checkpointedRecords.get()); } - @Test - public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint() throws Exception { + @ParameterizedTest(name = "chained={0}") + @ValueSource(booleans = {true, false}) + public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint( + boolean chained, @InjectClusterClient ClusterClient clusterClient) throws Exception { // Run a first job opening 5 transactions one per subtask and fail in async checkpoint phase - final Configuration config = new Configuration(); - config.set(CoreOptions.DEFAULT_PARALLELISM, 5); try { - executeWithMapper(new FailAsyncCheckpointMapper(0), config, null); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentLinkedDeque<>()); + Configuration config = createConfiguration(5); + executeWithMapper( + new FailAsyncCheckpointMapper(0), + checkpointedRecords, + config, + chained, + null, + clusterClient); } catch (Exception e) { assertThat(e.getCause().getCause().getMessage()) .contains("Exceeded checkpoint tolerable failure"); @@ -291,29 +314,46 @@ public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint() throws Exce assertThat(deserializeValues(drainAllRecordsFromTopic(topic, true))).isEmpty(); // Second job aborts all transactions from previous runs with higher parallelism - config.set(CoreOptions.DEFAULT_PARALLELISM, 1); - failed.get().set(true); + SharedReference failed = sharedObjects.add(new AtomicBoolean(true)); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentLinkedDeque<>()); + Configuration config = createConfiguration(1); executeWithMapper( - new FailingCheckpointMapper(failed, lastCheckpointedRecord), config, null); - final List> collectedRecords = - drainAllRecordsFromTopic(topic, true); - assertThat(deserializeValues(collectedRecords)) - .contains( - LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new)); + new FailingCheckpointMapper(failed), + checkpointedRecords, + config, + chained, + null, + clusterClient); + final List committedRecords = + deserializeValues(drainAllRecordsFromTopic(topic, true)); + assertThat(committedRecords).containsExactlyInAnyOrderElementsOf(checkpointedRecords.get()); + } + + private static Configuration createConfiguration(int parallelism) { + final Configuration config = new Configuration(); + config.set(CoreOptions.DEFAULT_PARALLELISM, parallelism); + return config; } - private void executeWithMapper( + private JobID executeWithMapper( MapFunction mapper, + SharedReference> checkpointedRecords, Configuration config, - @Nullable String transactionalIdPrefix) + boolean chained, + @Nullable String transactionalIdPrefix, + @InjectClusterClient ClusterClient clusterClient) throws Exception { - final StreamExecutionEnvironment env = new LocalStreamEnvironment(config); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); env.enableCheckpointing(100L); + if (!chained) { + env.disableOperatorChaining(); + } env.setRestartStrategy(RestartStrategies.noRestart()); final DataStreamSource source = env.fromSequence(1, 10); - final DataStream stream = source.map(mapper); + final DataStream stream = + source.map(mapper).map(new RecordFetcher(checkpointedRecords)); final KafkaSinkBuilder builder = new KafkaSinkBuilder() .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) @@ -328,20 +368,33 @@ private void executeWithMapper( } builder.setTransactionalIdPrefix(transactionalIdPrefix); stream.sinkTo(builder.build()); - env.execute(); + final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + JobID jobID = clusterClient.submitJob(jobGraph).get(); + clusterClient.requestJobResult(jobID).get(); + return jobID; } private void testRecoveryWithAssertion( - DeliveryGuarantee guarantee, - int maxConcurrentCheckpoints, - java.util.function.Consumer> recordsAssertion) + DeliveryGuarantee guarantee, int maxConcurrentCheckpoints) throws Exception { + testRecoveryWithAssertion(guarantee, maxConcurrentCheckpoints, true); + } + + private void testRecoveryWithAssertion( + DeliveryGuarantee guarantee, int maxConcurrentCheckpoints, boolean chained) throws Exception { - final StreamExecutionEnvironment env = new LocalStreamEnvironment(); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(createConfiguration(1)); + if (!chained) { + env.disableOperatorChaining(); + } env.enableCheckpointing(300L); env.getCheckpointConfig().setMaxConcurrentCheckpoints(maxConcurrentCheckpoints); DataStreamSource source = env.fromSequence(1, 10); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentLinkedDeque<>()); DataStream stream = - source.map(new FailingCheckpointMapper(failed, lastCheckpointedRecord)); + source.map(new FailingCheckpointMapper(sharedObjects.add(new AtomicBoolean(false)))) + .map(new RecordFetcher(checkpointedRecords)); stream.sinkTo( new KafkaSinkBuilder() @@ -356,42 +409,53 @@ private void testRecoveryWithAssertion( .build()); env.execute(); - final List> collectedRecords = - drainAllRecordsFromTopic(topic, guarantee == DeliveryGuarantee.EXACTLY_ONCE); - recordsAssertion.accept(deserializeValues(collectedRecords)); - checkProducerLeak(); + List committedRecords = + deserializeValues( + drainAllRecordsFromTopic( + topic, guarantee == DeliveryGuarantee.EXACTLY_ONCE)); + + if (guarantee == DeliveryGuarantee.AT_LEAST_ONCE) { + assertThat(committedRecords).containsAll(checkpointedRecords.get()); + } else if (guarantee == DeliveryGuarantee.EXACTLY_ONCE) { + assertThat(committedRecords) + .containsExactlyInAnyOrderElementsOf(checkpointedRecords.get()); + } } - private void writeRecordsToKafka( - DeliveryGuarantee deliveryGuarantee, SharedReference expectedRecords) + private void writeRecordsToKafka(DeliveryGuarantee deliveryGuarantee) throws Exception { + writeRecordsToKafka(deliveryGuarantee, true); + } + + private void writeRecordsToKafka(DeliveryGuarantee deliveryGuarantee, boolean chained) throws Exception { - final StreamExecutionEnvironment env = new LocalStreamEnvironment(); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(createConfiguration(1)); + if (!chained) { + env.disableOperatorChaining(); + } env.enableCheckpointing(100L); - final DataStream source = - env.addSource( - new InfiniteIntegerSource( - emittedRecordsCount, emittedRecordsWithCheckpoint)); - source.sinkTo( - new KafkaSinkBuilder() - .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers()) - .setDeliveryGuarantee(deliveryGuarantee) - .setRecordSerializer( - KafkaRecordSerializationSchema.builder() - .setTopic(topic) - .setValueSerializationSchema(new RecordSerializer()) - .build()) - .setTransactionalIdPrefix("kafka-sink") - .build()); + final DataStream source = env.addSource(new InfiniteIntegerSource()); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentLinkedDeque<>()); + source.map(new RecordFetcher(checkpointedRecords)) + .sinkTo( + new KafkaSinkBuilder() + .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers()) + .setDeliveryGuarantee(deliveryGuarantee) + .setRecordSerializer( + KafkaRecordSerializationSchema.builder() + .setTopic(topic) + .setValueSerializationSchema(new RecordSerializer()) + .build()) + .setTransactionalIdPrefix("kafka-sink") + .build()); env.execute(); - final List> collectedRecords = - drainAllRecordsFromTopic( - topic, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE); - final long recordsCount = expectedRecords.get().get(); - assertThat(recordsCount).isEqualTo(collectedRecords.size()); - assertThat(deserializeValues(collectedRecords)) - .contains(LongStream.range(1, recordsCount + 1).boxed().toArray(Long[]::new)); - checkProducerLeak(); + final List collectedRecords = + deserializeValues( + drainAllRecordsFromTopic( + topic, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE)); + assertThat(collectedRecords).containsExactlyInAnyOrderElementsOf(checkpointedRecords.get()); } private static List deserializeValues(List> records) { @@ -420,7 +484,7 @@ private static Properties getKafkaClientConfiguration() { } private void createTestTopic(String topic, int numPartitions, short replicationFactor) - throws ExecutionException, InterruptedException, TimeoutException { + throws ExecutionException, InterruptedException { final CreateTopicsResult result = admin.createTopics( Collections.singletonList( @@ -428,8 +492,7 @@ private void createTestTopic(String topic, int numPartitions, short replicationF result.all().get(); } - private void deleteTestTopic(String topic) - throws ExecutionException, InterruptedException, TimeoutException { + private void deleteTestTopic(String topic) throws ExecutionException, InterruptedException { final DeleteTopicsResult result = admin.deleteTopics(Collections.singletonList(topic)); result.all().get(); } @@ -450,6 +513,44 @@ public byte[] serialize(Long element) { } } + private static class RecordFetcher + implements MapFunction, CheckpointedFunction, CheckpointListener { + private final SharedReference> checkpointedRecords; + private List seenRecords = new ArrayList<>(); + private static final ListStateDescriptor STATE_DESCRIPTOR = + new ListStateDescriptor<>("committed-records", BasicTypeInfo.LONG_TYPE_INFO); + private ListState snapshottedRecords; + + private RecordFetcher(SharedReference> checkpointedRecords) { + this.checkpointedRecords = checkpointedRecords; + } + + @Override + public Long map(Long value) { + seenRecords.add(value); + return value; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + // sync with shared object, this guaranteed to sync because of final checkpoint + checkpointedRecords.get().clear(); + checkpointedRecords.get().addAll(Lists.newArrayList(snapshottedRecords.get())); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + LOG.info("snapshotState {} @ {}", seenRecords, context.getCheckpointId()); + snapshottedRecords.addAll(seenRecords); + seenRecords.clear(); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + snapshottedRecords = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR); + } + } + private static class FailAsyncCheckpointMapper implements MapFunction, CheckpointedFunction { private static final ListStateDescriptor stateDescriptor = @@ -514,7 +615,7 @@ public int getLength() { } @Override - public void serialize(Integer record, DataOutputView target) throws IOException { + public void serialize(Integer record, DataOutputView target) { if (record != -1) { return; } @@ -522,17 +623,17 @@ public void serialize(Integer record, DataOutputView target) throws IOException } @Override - public Integer deserialize(DataInputView source) throws IOException { + public Integer deserialize(DataInputView source) { return 1; } @Override - public Integer deserialize(Integer reuse, DataInputView source) throws IOException { + public Integer deserialize(Integer reuse, DataInputView source) { return 1; } @Override - public void copy(DataInputView source, DataOutputView target) throws IOException {} + public void copy(DataInputView source, DataOutputView target) {} @Override public TypeSerializerSnapshot snapshotConfiguration() { @@ -549,54 +650,33 @@ public SlowSerializerSnapshot() { /** Fails after a checkpoint is taken and the next record was emitted. */ private static class FailingCheckpointMapper - implements MapFunction, CheckpointListener, CheckpointedFunction { + implements MapFunction, CheckpointListener { private final SharedReference failed; - private final SharedReference lastCheckpointedRecord; - - private volatile long lastSeenRecord; - private volatile long checkpointedRecord; - private volatile long lastCheckpointId = 0; - private final AtomicInteger emittedBetweenCheckpoint = new AtomicInteger(0); + private long lastCheckpointId = 0; + private int emittedBetweenCheckpoint = 0; - FailingCheckpointMapper( - SharedReference failed, - SharedReference lastCheckpointedRecord) { + FailingCheckpointMapper(SharedReference failed) { this.failed = failed; - this.lastCheckpointedRecord = lastCheckpointedRecord; } @Override public Long map(Long value) throws Exception { - lastSeenRecord = value; - if (lastCheckpointId >= 1 - && emittedBetweenCheckpoint.get() > 0 - && !failed.get().get()) { + if (lastCheckpointId >= 1 && emittedBetweenCheckpoint > 0 && !failed.get().get()) { failed.get().set(true); throw new RuntimeException("Planned exception."); } // Delay execution to ensure that at-least one checkpoint is triggered before finish Thread.sleep(50); - emittedBetweenCheckpoint.incrementAndGet(); + emittedBetweenCheckpoint++; return value; } @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - LOG.info("notifyCheckpointComplete {} @ {}", checkpointedRecord, checkpointId); + public void notifyCheckpointComplete(long checkpointId) { lastCheckpointId = checkpointId; - emittedBetweenCheckpoint.set(0); - lastCheckpointedRecord.get().set(checkpointedRecord); - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - LOG.info("snapshotState {} @ {}", lastSeenRecord, context.getCheckpointId()); - checkpointedRecord = lastSeenRecord; + emittedBetweenCheckpoint = 0; } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception {} } /** @@ -604,31 +684,23 @@ public void initializeState(FunctionInitializationContext context) throws Except * receiving the checkpoint completed event. */ private static final class InfiniteIntegerSource - implements SourceFunction, CheckpointListener, CheckpointedFunction { - - private final SharedReference emittedRecordsCount; - private final SharedReference emittedRecordsWithCheckpoint; + implements SourceFunction, CheckpointListener { private volatile boolean running = true; - private volatile long temp; - private Object lock; + private final AtomicInteger nextRecord = new AtomicInteger(); - InfiniteIntegerSource( - SharedReference emittedRecordsCount, - SharedReference emittedRecordsWithCheckpoint) { - this.emittedRecordsCount = emittedRecordsCount; - this.emittedRecordsWithCheckpoint = emittedRecordsWithCheckpoint; - } + InfiniteIntegerSource() {} @Override public void run(SourceContext ctx) throws Exception { - lock = ctx.getCheckpointLock(); + Object lock = ctx.getCheckpointLock(); while (running) { synchronized (lock) { - ctx.collect(emittedRecordsCount.get().addAndGet(1)); + ctx.collect((long) nextRecord.getAndIncrement()); Thread.sleep(1); } } + LOG.info("last emitted record {}", nextRecord.get() - 1); } @Override @@ -637,22 +709,9 @@ public void cancel() { } @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - emittedRecordsWithCheckpoint.get().set(temp); + public void notifyCheckpointComplete(long checkpointId) { running = false; LOG.info("notifyCheckpointCompleted {}", checkpointId); } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - temp = emittedRecordsCount.get().get(); - LOG.info( - "snapshotState, {}, {}", - context.getCheckpointId(), - emittedRecordsCount.get().get()); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception {} } } From ee3d713795c9974dfc7d989a7c77071d84561f7c Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Fri, 7 Feb 2025 22:44:21 +0100 Subject: [PATCH 312/322] [FLINK-37281] Improve extensibility in KafkaWriterTestBase To test recovery in future PRs, it's important to decompose the #createWriter methods into common cases and advanced cases that may require some additional setup. --- .../connector/kafka/sink/KafkaSinkITCase.java | 65 ++++++++++------ .../sink/KafkaWriterFaultToleranceITCase.java | 30 ++++---- .../kafka/sink/KafkaWriterITCase.java | 77 +++++++++---------- .../kafka/sink/KafkaWriterTestBase.java | 65 ++++++++-------- 4 files changed, 122 insertions(+), 115 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java index 3ddc4b154..ba264764c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java @@ -99,15 +99,15 @@ import java.io.File; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; import java.util.UUID; -import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -256,8 +256,8 @@ public void testAbortTransactionsOfPendingCheckpointsAfterFailure( CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); config.set(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS, 2); JobID firstJobId = null; - SharedReference> checkpointedRecords = - sharedObjects.add(new ConcurrentLinkedDeque<>()); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentSkipListSet<>()); try { firstJobId = executeWithMapper( @@ -297,8 +297,8 @@ public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint( boolean chained, @InjectClusterClient ClusterClient clusterClient) throws Exception { // Run a first job opening 5 transactions one per subtask and fail in async checkpoint phase try { - SharedReference> checkpointedRecords = - sharedObjects.add(new ConcurrentLinkedDeque<>()); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentSkipListSet<>()); Configuration config = createConfiguration(5); executeWithMapper( new FailAsyncCheckpointMapper(0), @@ -315,8 +315,8 @@ public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint( // Second job aborts all transactions from previous runs with higher parallelism SharedReference failed = sharedObjects.add(new AtomicBoolean(true)); - SharedReference> checkpointedRecords = - sharedObjects.add(new ConcurrentLinkedDeque<>()); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentSkipListSet<>()); Configuration config = createConfiguration(1); executeWithMapper( new FailingCheckpointMapper(failed), @@ -338,7 +338,7 @@ private static Configuration createConfiguration(int parallelism) { private JobID executeWithMapper( MapFunction mapper, - SharedReference> checkpointedRecords, + SharedReference> checkpointedRecords, Configuration config, boolean chained, @Nullable String transactionalIdPrefix, @@ -353,7 +353,7 @@ private JobID executeWithMapper( env.setRestartStrategy(RestartStrategies.noRestart()); final DataStreamSource source = env.fromSequence(1, 10); final DataStream stream = - source.map(mapper).map(new RecordFetcher(checkpointedRecords)); + source.map(mapper).map(new RecordFetcher(checkpointedRecords)).uid("fetcher"); final KafkaSinkBuilder builder = new KafkaSinkBuilder() .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) @@ -390,8 +390,8 @@ private void testRecoveryWithAssertion( env.enableCheckpointing(300L); env.getCheckpointConfig().setMaxConcurrentCheckpoints(maxConcurrentCheckpoints); DataStreamSource source = env.fromSequence(1, 10); - SharedReference> checkpointedRecords = - sharedObjects.add(new ConcurrentLinkedDeque<>()); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentSkipListSet<>()); DataStream stream = source.map(new FailingCheckpointMapper(sharedObjects.add(new AtomicBoolean(false)))) .map(new RecordFetcher(checkpointedRecords)); @@ -435,8 +435,8 @@ private void writeRecordsToKafka(DeliveryGuarantee deliveryGuarantee, boolean ch } env.enableCheckpointing(100L); final DataStream source = env.addSource(new InfiniteIntegerSource()); - SharedReference> checkpointedRecords = - sharedObjects.add(new ConcurrentLinkedDeque<>()); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentSkipListSet<>()); source.map(new RecordFetcher(checkpointedRecords)) .sinkTo( new KafkaSinkBuilder() @@ -513,36 +513,55 @@ public byte[] serialize(Long element) { } } + /** + * Fetches records that have been successfully checkpointed. It relies on final checkpoints and + * subsumption to ultimately, emit all records that have been checkpointed. + * + *

      Note that the current implementation only works by operating on a set because on failure, + * we may up with duplicate records being added to the {@link #checkpointedRecords}. + * + *

      The fetcher uses three states to manage the records: + * + *

        + *
      1. {@link #recordsSinceLastCheckpoint} is used to buffer records between checkpoints. + *
      2. {@link #snapshottedRecords} is used to store the records that have been checkpointed. + *
      3. {@link #checkpointedRecords} is used to store snapshottedRecords where the checkpoint + * has been acknowledged. + *
      + * + *

      Records are promoted from data structure to the next (e.g. removed from the lower level). + */ private static class RecordFetcher implements MapFunction, CheckpointedFunction, CheckpointListener { - private final SharedReference> checkpointedRecords; - private List seenRecords = new ArrayList<>(); + private final SharedReference> checkpointedRecords; + private final List recordsSinceLastCheckpoint = new ArrayList<>(); private static final ListStateDescriptor STATE_DESCRIPTOR = new ListStateDescriptor<>("committed-records", BasicTypeInfo.LONG_TYPE_INFO); private ListState snapshottedRecords; - private RecordFetcher(SharedReference> checkpointedRecords) { + private RecordFetcher(SharedReference> checkpointedRecords) { this.checkpointedRecords = checkpointedRecords; } @Override public Long map(Long value) { - seenRecords.add(value); + recordsSinceLastCheckpoint.add(value); return value; } @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { - // sync with shared object, this guaranteed to sync because of final checkpoint - checkpointedRecords.get().clear(); + // sync with shared object, this is guaranteed to sync eventually because of final + // checkpoint checkpointedRecords.get().addAll(Lists.newArrayList(snapshottedRecords.get())); } @Override public void snapshotState(FunctionSnapshotContext context) throws Exception { - LOG.info("snapshotState {} @ {}", seenRecords, context.getCheckpointId()); - snapshottedRecords.addAll(seenRecords); - seenRecords.clear(); + LOG.info( + "snapshotState {} @ {}", recordsSinceLastCheckpoint, context.getCheckpointId()); + snapshottedRecords.addAll(recordsSinceLastCheckpoint); + recordsSinceLastCheckpoint.clear(); } @Override diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java index 1cf1b5c07..20af1b575 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java @@ -59,13 +59,12 @@ public void setUp(TestInfo testInfo) { @Test void testWriteExceptionWhenKafkaUnavailable() throws Exception { - Properties properties = getPropertiesForSendingFaultTolerance(); - final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); try (KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup)) { + createWriter( + DeliveryGuarantee.AT_LEAST_ONCE, + new SinkInitContext(metricGroup, timeService, null))) { writer.write(1, SINK_WRITER_CONTEXT); @@ -83,13 +82,12 @@ void testWriteExceptionWhenKafkaUnavailable() throws Exception { @Test void testFlushExceptionWhenKafkaUnavailable() throws Exception { - Properties properties = getPropertiesForSendingFaultTolerance(); - final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); try (KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup)) { + createWriter( + DeliveryGuarantee.AT_LEAST_ONCE, + new SinkInitContext(metricGroup, timeService, null))) { writer.write(1, SINK_WRITER_CONTEXT); KAFKA_CONTAINER.stop(); @@ -104,13 +102,12 @@ void testFlushExceptionWhenKafkaUnavailable() throws Exception { @Test void testCloseExceptionWhenKafkaUnavailable() throws Exception { - Properties properties = getPropertiesForSendingFaultTolerance(); - final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.AT_LEAST_ONCE, metricGroup); + createWriter( + DeliveryGuarantee.AT_LEAST_ONCE, + new SinkInitContext(metricGroup, timeService, null)); writer.write(1, SINK_WRITER_CONTEXT); @@ -131,13 +128,11 @@ void testCloseExceptionWhenKafkaUnavailable() throws Exception { @Test void testMailboxExceptionWhenKafkaUnavailable() throws Exception { - Properties properties = getPropertiesForSendingFaultTolerance(); SinkInitContext sinkInitContext = new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); try (KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.AT_LEAST_ONCE, sinkInitContext)) { + createWriter(DeliveryGuarantee.AT_LEAST_ONCE, sinkInitContext)) { KAFKA_CONTAINER.stop(); @@ -159,8 +154,9 @@ void testMailboxExceptionWhenKafkaUnavailable() throws Exception { } } - private Properties getPropertiesForSendingFaultTolerance() { - Properties properties = getKafkaClientConfiguration(); + @Override + protected Properties getKafkaClientConfiguration() { + Properties properties = super.getKafkaClientConfiguration(); // reduce the default vault for test case properties.setProperty("retries", INIT_KAFKA_RETRIES); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index a559e3a54..fbc5d0e60 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -69,8 +69,7 @@ public void setUp(TestInfo testInfo) { @ParameterizedTest @EnumSource(DeliveryGuarantee.class) public void testRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { - try (final KafkaWriter ignored = - createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) { + try (final KafkaWriter ignored = createWriter(guarantee)) { assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME).isPresent()).isTrue(); } } @@ -87,10 +86,9 @@ public void testIncreasingRecordBasedCounters() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), + createWriter( DeliveryGuarantee.AT_LEAST_ONCE, - metricGroup)) { + new SinkInitContext(metricGroup, timeService, null))) { final Counter numBytesOut = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); final Counter numRecordsOut = metricGroup.getIOMetricGroup().getNumRecordsOutCounter(); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); @@ -127,9 +125,7 @@ public void testIncreasingRecordBasedCounters() throws Exception { @Test public void testCurrentSendTimeMetric() throws Exception { - try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.AT_LEAST_ONCE)) { + try (final KafkaWriter writer = createWriter(DeliveryGuarantee.AT_LEAST_ONCE)) { final Optional> currentSendTime = metricListener.getGauge("currentSendTime"); assertThat(currentSendTime.isPresent()).isTrue(); @@ -148,6 +144,7 @@ public void testCurrentSendTimeMetric() throws Exception { } }); assertThat(currentSendTime.get().getValue()).isGreaterThan(0L); + writer.flush(true); } } @@ -158,8 +155,9 @@ void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); final KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + createWriter( + DeliveryGuarantee.EXACTLY_ONCE, + new SinkInitContext(metricGroup, timeService, null)); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); @@ -186,8 +184,9 @@ void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); final KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + createWriter( + DeliveryGuarantee.EXACTLY_ONCE, + new SinkInitContext(metricGroup, timeService, null)); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); @@ -216,8 +215,7 @@ void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); final KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + createWriter(DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); final Counter numRecordsOutErrors = sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); @@ -252,8 +250,9 @@ void testCloseAsyncErrorPropagationAndErrorCounter() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); final KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + createWriter( + DeliveryGuarantee.EXACTLY_ONCE, + new SinkInitContext(metricGroup, timeService, null)); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); @@ -286,28 +285,28 @@ private void triggerProducerException(KafkaWriter writer, Properties pr @Test public void testMetadataPublisher() throws Exception { List metadataList = new ArrayList<>(); + SinkWriterMetricGroup sinkWriterMetricGroup = createSinkWriterMetricGroup(); try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), + createWriter( DeliveryGuarantee.AT_LEAST_ONCE, - createSinkWriterMetricGroup(), - meta -> metadataList.add(meta.toString()))) { + new SinkInitContext( + sinkWriterMetricGroup, + timeService, + meta -> metadataList.add(meta.topic() + "@" + meta.offset())))) { List expected = new ArrayList<>(); for (int i = 0; i < 100; i++) { writer.write(1, SINK_WRITER_CONTEXT); - expected.add("testMetadataPublisher-0@" + i); + expected.add("testMetadataPublisher@" + i); } writer.flush(false); - assertThat(metadataList).usingRecursiveComparison().isEqualTo(expected); + assertThat(metadataList).containsExactlyInAnyOrderElementsOf(expected); } } /** Test that producer is not accidentally recreated or pool is used. */ @Test void testLingeringTransaction() throws Exception { - final KafkaWriter failedWriter = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE); + final KafkaWriter failedWriter = createWriter(DeliveryGuarantee.EXACTLY_ONCE); // create two lingering transactions failedWriter.flush(false); @@ -318,8 +317,7 @@ void testLingeringTransaction() throws Exception { failedWriter.snapshotState(2); try (final KafkaWriter recoveredWriter = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { + createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { recoveredWriter.write(1, SINK_WRITER_CONTEXT); recoveredWriter.flush(false); @@ -346,8 +344,7 @@ void testLingeringTransaction() throws Exception { names = "EXACTLY_ONCE", mode = EnumSource.Mode.EXCLUDE) void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exception { - try (final KafkaWriter writer = - createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) { + try (final KafkaWriter writer = createWriter(guarantee)) { assertThat(writer.getProducerPool()).hasSize(0); FlinkKafkaInternalProducer firstProducer = writer.getCurrentProducer(); @@ -365,10 +362,8 @@ void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exce /** Test that producers are reused when committed. */ @Test - void usePoolForTransactional() throws Exception { - try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { + void usePooledProducerForTransactional() throws Exception { + try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { assertThat(writer.getProducerPool()).hasSize(0); writer.write(1, SINK_WRITER_CONTEXT); @@ -412,9 +407,7 @@ void usePoolForTransactional() throws Exception { */ @Test void prepareCommitForEmptyTransaction() throws Exception { - try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { assertThat(writer.getProducerPool()).hasSize(0); // no data written to current transaction @@ -436,14 +429,12 @@ void prepareCommitForEmptyTransaction() throws Exception { @Test void testAbortOnClose() throws Exception { Properties properties = getKafkaClientConfiguration(); - try (final KafkaWriter writer = - createWriterWithConfiguration(properties, DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { writer.write(1, SINK_WRITER_CONTEXT); assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(0); } - try (final KafkaWriter writer = - createWriterWithConfiguration(properties, DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { writer.write(2, SINK_WRITER_CONTEXT); writer.flush(false); Collection committables = writer.prepareCommit(); @@ -468,7 +459,11 @@ private void assertKafkaMetricNotPresent( final Properties config = getKafkaClientConfiguration(); config.put(configKey, configValue); try (final KafkaWriter ignored = - createWriterWithConfiguration(config, guarantee)) { + createWriter( + builder -> + builder.setKafkaProducerConfig(config) + .setDeliveryGuarantee(guarantee), + createInitContext())) { assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME)).isNotPresent(); } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java index 479d6ab8f..46f62eab3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java @@ -33,6 +33,9 @@ import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.UserCodeClassLoader; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.serialization.ByteArraySerializer; @@ -48,7 +51,9 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.Comparator; +import java.util.Map; import java.util.Optional; import java.util.OptionalLong; import java.util.PriorityQueue; @@ -58,6 +63,7 @@ import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; +import static org.apache.kafka.clients.admin.AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG; /** Test base for KafkaWriter. */ public abstract class KafkaWriterTestBase { @@ -68,6 +74,7 @@ public abstract class KafkaWriterTestBase { protected static final String KAFKA_METRIC_WITH_GROUP_NAME = "KafkaProducer.incoming-byte-total"; protected static final SinkWriter.Context SINK_WRITER_CONTEXT = new DummySinkWriterContext(); + public static final String TEST_PREFIX = "test-prefix"; protected static String topic; protected MetricListener metricListener; @@ -84,6 +91,11 @@ public void setUp(TestInfo testInfo) { metricListener = new MetricListener(); timeService = new TriggerTimeService(); topic = testInfo.getDisplayName().replaceAll("\\W", ""); + Map properties = new java.util.HashMap<>(); + properties.put(BOOTSTRAP_SERVERS_CONFIG, KAFKA_CONTAINER.getBootstrapServers()); + try (Admin admin = AdminClient.create(properties)) { + admin.createTopics(Collections.singleton(new NewTopic(topic, 10, (short) 1))); + } } @AfterEach @@ -91,48 +103,33 @@ public void check() { checkProducerLeak(); } - protected KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee) throws IOException { - return createWriterWithConfiguration(config, guarantee, createSinkWriterMetricGroup()); + KafkaWriter createWriter(DeliveryGuarantee guarantee) throws IOException { + return createWriter(guarantee, createInitContext()); } - protected KafkaWriter createWriterWithConfiguration( - Properties config, - DeliveryGuarantee guarantee, - SinkWriterMetricGroup sinkWriterMetricGroup) + KafkaWriter createWriter(DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) throws IOException { - return createWriterWithConfiguration(config, guarantee, sinkWriterMetricGroup, null); + return createWriter(builder -> builder.setDeliveryGuarantee(guarantee), sinkInitContext); } - protected KafkaWriter createWriterWithConfiguration( - Properties config, - DeliveryGuarantee guarantee, - SinkWriterMetricGroup sinkWriterMetricGroup, - @Nullable Consumer metadataConsumer) + KafkaWriter createWriter( + Consumer> sinkBuilderAdjuster, SinkInitContext sinkInitContext) throws IOException { - KafkaSink kafkaSink = - KafkaSink.builder() - .setKafkaProducerConfig(config) - .setDeliveryGuarantee(guarantee) - .setTransactionalIdPrefix("test-prefix") - .setRecordSerializer(new DummyRecordSerializer()) - .build(); - return (KafkaWriter) - kafkaSink.createWriter( - new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer)); + return (KafkaWriter) createSink(sinkBuilderAdjuster).createWriter(sinkInitContext); } - protected KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) - throws IOException { - KafkaSink kafkaSink = + KafkaSink createSink(Consumer> sinkBuilderAdjuster) { + KafkaSinkBuilder builder = KafkaSink.builder() - .setKafkaProducerConfig(config) - .setDeliveryGuarantee(guarantee) - .setTransactionalIdPrefix("test-prefix") - .setRecordSerializer(new DummyRecordSerializer()) - .build(); - return (KafkaWriter) kafkaSink.createWriter(sinkInitContext); + .setKafkaProducerConfig(getKafkaClientConfiguration()) + .setTransactionalIdPrefix(TEST_PREFIX) + .setRecordSerializer(new DummyRecordSerializer()); + sinkBuilderAdjuster.accept(builder); + return builder.build(); + } + + SinkInitContext createInitContext() { + return new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); } protected SinkWriterMetricGroup createSinkWriterMetricGroup() { @@ -141,7 +138,7 @@ protected SinkWriterMetricGroup createSinkWriterMetricGroup() { return InternalSinkWriterMetricGroup.wrap(operatorMetricGroup); } - protected static Properties getKafkaClientConfiguration() { + protected Properties getKafkaClientConfiguration() { final Properties standardProps = new Properties(); standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers()); standardProps.put("group.id", "kafkaWriter-tests"); From 707ec4c194cdea26a34e5f0c8540621c37175c61 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 11 Feb 2025 08:30:35 +0100 Subject: [PATCH 313/322] [FLINK-37281] Fix KafkaUtil#checkProducerLeak Since Java 20, Thread.stop fails, so we just need to remember old leaks to avoid failing subsequent tests. --- .../flink/connector/kafka/testutils/KafkaUtil.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java index 7bf7bb007..78b344b64 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java @@ -38,6 +38,7 @@ import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.ConcurrentSkipListSet; import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.fail; @@ -193,12 +194,17 @@ private static Set getAllPartitions( .collect(Collectors.toSet()); } + private static final Set KNOWN_LEAKS = new ConcurrentSkipListSet<>(); + public static void checkProducerLeak() { List> leaks = null; for (int tries = 0; tries < 10; tries++) { leaks = Thread.getAllStackTraces().entrySet().stream() .filter(KafkaUtil::findAliveKafkaThread) + .filter( + threadEntry -> + !KNOWN_LEAKS.contains(threadEntry.getKey().getId())) .collect(Collectors.toList()); if (leaks.isEmpty()) { return; @@ -210,10 +216,10 @@ public static void checkProducerLeak() { } for (Map.Entry leak : leaks) { - leak.getKey().stop(); + KNOWN_LEAKS.add(leak.getKey().getId()); } fail( - "Detected producer leaks:\n" + "Detected new producer leaks:\n" + leaks.stream() .map(KafkaUtil::format) .collect(Collectors.joining("\n\n"))); From 5947f3c74a9f32cb3893891a210eb988a64b9b57 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Fri, 7 Feb 2025 22:55:53 +0100 Subject: [PATCH 314/322] [FLINK-37282] Introduce internal package to sink Move FlinkKafkaInternalProducer and TransactionalIdFactory to internal. All other classes are potentially leaked through the generics and signatures of the KafkaSink(Builder). --- .../86dfd459-67a9-4b26-9b5c-0b0bbf22681a | 105 +---------------- .../984f05c0-ec82-405e-9bcc-d202dbe7202e | 10 +- .../c0d94764-76a0-4c50-b617-70b1754c4612 | 110 +----------------- .../d853eb69-8c04-4246-9a5e-4f5911286b1d | 1 - .../kafka/sink/KafkaCommittable.java | 2 + .../connector/kafka/sink/KafkaCommitter.java | 1 + .../connector/kafka/sink/KafkaWriter.java | 2 + .../kafka/sink/TransactionAborter.java | 3 + .../FlinkKafkaInternalProducer.java | 22 ++-- .../TransactionalIdFactory.java | 23 ++-- .../FlinkKafkaInternalProducerITCase.java | 1 + .../kafka/sink/KafkaCommitterTest.java | 1 + .../kafka/sink/KafkaWriterITCase.java | 1 + .../kafka/sink/TransactionIdFactoryTest.java | 1 + 14 files changed, 46 insertions(+), 237 deletions(-) rename flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/{ => internal}/FlinkKafkaInternalProducer.java (95%) rename flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/{ => internal}/TransactionalIdFactory.java (62%) diff --git a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a index 07efd19a9..f4c3aa9f2 100644 --- a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a +++ b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a @@ -1,107 +1,5 @@ org.apache.flink.connector.kafka.sink.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.KafkaSinkITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.KafkaTransactionLogITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.KafkaWriterFaultToleranceITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match: -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.source.KafkaSourceITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.source.KafkaSourceLegacyITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.kafka.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.kafka.KafkaITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.kafka.KafkaProducerAtLeastOnceITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.kafka.KafkaProducerExactlyOnceITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.kafka.shuffle.KafkaShuffleExactlyOnceITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.kafka.shuffle.KafkaShuffleITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ -* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.KafkaSinkITCase does not satisfy: only one of the following predicates match:\ -* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ @@ -112,14 +10,13 @@ org.apache.flink.connector.kafka.sink.KafkaTransactionLogITCase does not satisfy * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match:\ org.apache.flink.connector.kafka.sink.KafkaWriterFaultToleranceITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.KafkaWriterFaultToleranceITCase does not satisfy: only one of the following predicates match:\ +org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ diff --git a/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e b/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e index 28b74b8c8..48b26ecd6 100644 --- a/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e +++ b/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e @@ -125,11 +125,11 @@ Field has type in (FlinkKafkaShuffleProducer.java:0) Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) -Method calls method in (FlinkKafkaInternalProducer.java:100) -Method calls method in (FlinkKafkaInternalProducer.java:109) -Method calls method in (FlinkKafkaInternalProducer.java:296) -Method calls method in (FlinkKafkaInternalProducer.java:297) -Method calls method in (FlinkKafkaInternalProducer.java:174) +Method calls method in (FlinkKafkaInternalProducer.java:100) +Method calls method in (FlinkKafkaInternalProducer.java:109) +Method calls method in (FlinkKafkaInternalProducer.java:296) +Method calls method in (FlinkKafkaInternalProducer.java:297) +Method calls method in (FlinkKafkaInternalProducer.java:174) Method calls method in (KafkaRecordSerializationSchemaBuilder.java:254) Method calls method in (KafkaRecordSerializationSchemaBuilder.java:255) Method calls method in (KafkaRecordSerializationSchemaBuilder.java:269) diff --git a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 index 20326f5ec..756b92768 100644 --- a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 +++ b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 @@ -24,6 +24,7 @@ Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) Constructor (org.apache.flink.api.common.typeutils.TypeSerializer)> is annotated with in (KafkaShuffleFetcher.java:0) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) Field has generic type > with type argument depending on in (KafkaClusterMetricGroupManager.java:0) Field has type in (DynamicKafkaSourceReader.java:0) Field has type in (AbstractFetcher.java:0) @@ -99,112 +100,3 @@ Method calls method in (KafkaDynamicSink.java:386) Method has return type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (KafkaDynamicSink.java:0) Method calls method in (KafkaDynamicSource.java:566) -Constructor (java.lang.String, org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext, java.lang.Runnable)> calls constructor (java.lang.String)> in (StoppableKafkaEnumContextProxy.java:95) -Constructor (java.util.function.Function, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.kafka.sink.HeaderProvider)> calls method in (KafkaRecordSerializationSchemaBuilder.java:322) -Constructor (java.util.function.Function, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.kafka.sink.HeaderProvider)> calls method in (KafkaRecordSerializationSchemaBuilder.java:323) -Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:51) -Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:53) -Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:54) -Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:134) -Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:135) -Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:136) -Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:137) -Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:138) -Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:173) -Constructor (java.lang.String)> calls method in (KafkaWriterState.java:28) -Constructor (java.lang.Object, java.util.function.Consumer)> calls method in (Recyclable.java:31) -Constructor (java.lang.Object, java.util.function.Consumer)> calls method in (Recyclable.java:32) -Constructor (int, int, java.util.function.Function, java.util.function.Consumer)> calls method in (TransactionAborter.java:60) -Constructor (java.util.Set)> calls method in (FlinkKafkaProducer.java:1591) -Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> calls method in (AbstractFetcher.java:593) -Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> calls method in (AbstractFetcher.java:595) -Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaConsumerThread, org.apache.flink.streaming.connectors.kafka.internals.KafkaCommitCallback)> calls method in (KafkaConsumerThread.java:540) -Constructor (int, org.apache.flink.api.common.serialization.DeserializationSchema, [I, org.apache.flink.api.common.serialization.DeserializationSchema, [I, boolean, [Lorg.apache.flink.streaming.connectors.kafka.table.DynamicKafkaDeserializationSchema$MetadataConverter;, org.apache.flink.api.common.typeinfo.TypeInformation, boolean)> calls method in (DynamicKafkaDeserializationSchema.java:72) -Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:71) -Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:75) -Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:87) -Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:181) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:161) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:163) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:166) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:167) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:168) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:176) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:179) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:210) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:194) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:198) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:201) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:203) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:216) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:218) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:220) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:224) -Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:226) -Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:70) -Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:71) -Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:72) -Method calls method in (DynamicKafkaSourceBuilder.java:291) -Method calls method in (DynamicKafkaSourceBuilder.java:293) -Method calls method in (DynamicKafkaSourceBuilder.java:295) -Method calls method in (DynamicKafkaSourceBuilder.java:299) -Method calls method in (DynamicKafkaSourceBuilder.java:100) -Method calls method in (DynamicKafkaSourceBuilder.java:73) -Method calls method in (DynamicKafkaSourceBuilder.java:86) -Method calls method in (DynamicKafkaSourceEnumStateSerializer.java:142) -Method calls method in (DynamicKafkaSourceEnumerator.java:514) -Method calls method in (StoppableKafkaEnumContextProxy.java:237) -Method calls method in (StoppableKafkaEnumContextProxy.java:259) -Method calls method in (DynamicKafkaSourceReader.java:418) -Method calls method in (DynamicKafkaSourceReader.java:228) -Method calls method in (FlinkKafkaInternalProducer.java:100) -Method calls method in (FlinkKafkaInternalProducer.java:109) -Method calls method in (FlinkKafkaInternalProducer.java:296) -Method calls method in (FlinkKafkaInternalProducer.java:297) -Method calls method in (FlinkKafkaInternalProducer.java:174) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:268) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:269) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:283) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:279) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:218) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:154) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:112) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:99) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:124) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:123) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:139) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:137) -Method calls method in (KafkaRecordSerializationSchemaBuilder.java:204) -Method calls method in (KafkaSerializerWrapper.java:71) -Method calls method in (KafkaSerializerWrapper.java:88) -Method calls method in (KafkaSinkBuilder.java:194) -Method calls method in (KafkaSinkBuilder.java:202) -Method calls method in (KafkaSinkBuilder.java:198) -Method calls method in (KafkaSinkBuilder.java:111) -Method calls method in (KafkaSinkBuilder.java:97) -Method calls method in (KafkaSinkBuilder.java:123) -Method calls method in (KafkaSinkBuilder.java:133) -Method calls method in (KafkaSinkBuilder.java:151) -Method calls method in (KafkaSinkBuilder.java:175) -Method calls method in (KafkaSinkBuilder.java:176) -Method calls method in (KafkaWriter.java:244) -Method calls method in (KafkaWriter.java:245) -Method calls method in (KafkaWriter.java:246) -Method calls method in (KafkaWriter.java:311) -Method calls method in (Recyclable.java:36) -Method calls method in (KafkaSourceBuilder.java:513) -Method calls method in (KafkaSourceBuilder.java:518) -Method calls method in (KafkaSourceBuilder.java:522) -Method calls method in (KafkaSourceBuilder.java:524) -Method calls method in (KafkaSourceBuilder.java:203) -Method calls method in (ReaderHandledOffsetsInitializer.java:75) -Method calls method in (SpecifiedOffsetsInitializer.java:105) -Method calls method in (KafkaPartitionSplitReader.java:540) -Method calls method in (KafkaPartitionSplitReader.java:359) -Method calls method in (KafkaValueOnlyDeserializerWrapper.java:65) -Method calls method in (KafkaShuffleFetcher.java:280) -Method calls method in (KafkaConnectorOptionsUtil.java:500) -Method calls method in (KafkaConnectorOptionsUtil.java:567) -Method calls method in (KafkaConnectorOptionsUtil.java:480) -Method calls method in (ReducingUpsertWriter.java:177) diff --git a/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d b/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d index dfb76a0c5..fddc6df33 100644 --- a/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d +++ b/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d @@ -2,4 +2,3 @@ org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema.open(org.ap org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema.serialize(java.lang.Object, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext, java.lang.Long): Argument leaf type org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer.getPartitionOffsets(java.util.Collection, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer$PartitionOffsetsRetriever): Argument leaf type org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer$PartitionOffsetsRetriever does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.invoke(org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$KafkaTransactionState, java.lang.Object, org.apache.flink.streaming.api.functions.sink.SinkFunction$Context): Argument leaf type org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$KafkaTransactionState does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated -org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition.dropLeaderData(java.util.List): Argument leaf type org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java index 72a3281e6..54ffdb99d 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java @@ -17,6 +17,8 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; + import javax.annotation.Nullable; import java.util.Objects; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java index 4dbeaf9e7..4c4458851 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.kafka.sink; import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.errors.InvalidTxnStateException; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java index 0f4e8755e..0ef37854d 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java @@ -24,6 +24,8 @@ import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.MetricUtil; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.TransactionalIdFactory; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java index cae6ca648..85a139be9 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java @@ -17,6 +17,9 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.TransactionalIdFactory; + import javax.annotation.Nullable; import java.io.Closeable; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java similarity index 95% rename from flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java rename to flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java index e514054d7..4c482acb5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -15,7 +16,9 @@ * limitations under the License. */ -package org.apache.flink.connector.kafka.sink; +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.annotation.Internal; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; @@ -43,7 +46,8 @@ /** * A {@link KafkaProducer} that exposes private fields to allow resume producing from a given state. */ -class FlinkKafkaInternalProducer extends KafkaProducer { +@Internal +public class FlinkKafkaInternalProducer extends KafkaProducer { private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaInternalProducer.class); private static final String TRANSACTION_MANAGER_FIELD_NAME = "transactionManager"; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionalIdFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/TransactionalIdFactory.java similarity index 62% rename from flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionalIdFactory.java rename to flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/TransactionalIdFactory.java index eda4c0180..1c0c6263f 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionalIdFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/TransactionalIdFactory.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -15,9 +16,13 @@ * limitations under the License. */ -package org.apache.flink.connector.kafka.sink; +package org.apache.flink.connector.kafka.sink.internal; -class TransactionalIdFactory { +import org.apache.flink.annotation.Internal; + +/** Utility class for constructing transactionalIds for Kafka transactions. */ +@Internal +public class TransactionalIdFactory { private static final String TRANSACTIONAL_ID_DELIMITER = "-"; /** diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java index 55224c052..f1b61d421 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java @@ -17,6 +17,7 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; import org.apache.flink.util.TestLoggerExtension; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java index a768bfc7a..610a21e50 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.kafka.sink; import org.apache.flink.api.connector.sink2.mocks.MockCommitRequest; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; import org.apache.flink.util.TestLoggerExtension; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index fbc5d0e60..1494c84f3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.kafka.sink; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java index 982d58e33..73adc42c2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java @@ -17,6 +17,7 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.connector.kafka.sink.internal.TransactionalIdFactory; import org.apache.flink.util.TestLogger; import org.junit.Test; From 2e652a92e48df67d77e04f7aebcf8e97c511b1cb Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Sun, 16 Feb 2025 21:01:32 +0100 Subject: [PATCH 315/322] [hotfix] Fix leaks in FlinkKafkaProducerTest --- .../kafka/FlinkKafkaProducerTest.java | 34 ++++++++++++------- 1 file changed, 22 insertions(+), 12 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java index 6fedcc43c..5c2d38038 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.kafka.clients.producer.ProducerRecord; +import org.junit.After; import org.junit.Test; import javax.annotation.Nullable; @@ -33,28 +34,35 @@ import java.util.Optional; import java.util.Properties; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link FlinkKafkaProducer}. */ public class FlinkKafkaProducerTest { + @After + public void checkLeaks() { + checkProducerLeak(); + } + @Test public void testOpenSerializationSchemaProducer() throws Exception { OpenTestingSerializationSchema schema = new OpenTestingSerializationSchema(); FlinkKafkaProducer kafkaProducer = new FlinkKafkaProducer<>("localhost:9092", "test-topic", schema); - OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>( new StreamSink<>(kafkaProducer), 1, 1, 0, IntSerializer.INSTANCE, - new OperatorID(1, 1)); + new OperatorID(1, 1))) { - testHarness.open(); + testHarness.open(); - assertThat(schema.openCalled).isTrue(); + assertThat(schema.openCalled).isTrue(); + } } @Test @@ -69,18 +77,19 @@ public void testOpenKafkaSerializationSchemaProducer() throws Exception { properties, FlinkKafkaProducer.Semantic.AT_LEAST_ONCE); - OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>( new StreamSink<>(kafkaProducer), 1, 1, 0, IntSerializer.INSTANCE, - new OperatorID(1, 1)); + new OperatorID(1, 1))) { - testHarness.open(); + testHarness.open(); - assertThat(schema.openCalled).isTrue(); + assertThat(schema.openCalled).isTrue(); + } } @Test @@ -95,18 +104,19 @@ public void testOpenKafkaCustomPartitioner() throws Exception { properties, Optional.of(partitioner)); - OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>( new StreamSink<>(kafkaProducer), 1, 1, 0, IntSerializer.INSTANCE, - new OperatorID(1, 1)); + new OperatorID(1, 1))) { - testHarness.open(); + testHarness.open(); - assertThat(partitioner.openCalled).isTrue(); + assertThat(partitioner.openCalled).isTrue(); + } } @Test(expected = NullPointerException.class) From df353e9d2543eb5babb325a4f9895fe89623a53f Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Mon, 10 Feb 2025 10:52:46 +0100 Subject: [PATCH 316/322] [FLINK-37282] Split KafkaWriter into EOS/non-EOS Split the easy case of non-transactional writer from the transactional writer to simplify reasoning about the state (e.g. which fields are used when). --- .../27a0a5e4-29c2-4069-b381-952746c90862 | 2 +- .../86dfd459-67a9-4b26-9b5c-0b0bbf22681a | 6 + .../c0d94764-76a0-4c50-b617-70b1754c4612 | 70 ++-- .../kafka/sink/ExactlyOnceKafkaWriter.java | 242 ++++++++++++++ .../flink/connector/kafka/sink/KafkaSink.java | 41 ++- .../connector/kafka/sink/KafkaWriter.java | 175 +--------- .../internal/FlinkKafkaInternalProducer.java | 45 ++- .../sink/ExactlyOnceKafkaWriterITCase.java | 309 ++++++++++++++++++ .../sink/KafkaWriterFaultToleranceITCase.java | 19 -- .../kafka/sink/KafkaWriterITCase.java | 290 ---------------- .../kafka/sink/KafkaWriterTestBase.java | 17 +- tools/releasing/shared | 2 +- 12 files changed, 665 insertions(+), 553 deletions(-) create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java diff --git a/flink-connector-kafka/archunit-violations/27a0a5e4-29c2-4069-b381-952746c90862 b/flink-connector-kafka/archunit-violations/27a0a5e4-29c2-4069-b381-952746c90862 index c7abba76e..109ecf836 100644 --- a/flink-connector-kafka/archunit-violations/27a0a5e4-29c2-4069-b381-952746c90862 +++ b/flink-connector-kafka/archunit-violations/27a0a5e4-29c2-4069-b381-952746c90862 @@ -1 +1 @@ -Method calls method in (FlinkKafkaProducer.java:1327) \ No newline at end of file +Method calls method in (FlinkKafkaProducer.java:1320) diff --git a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a index f4c3aa9f2..0feda7c9a 100644 --- a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a +++ b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a @@ -1,3 +1,9 @@ +org.apache.flink.connector.kafka.sink.ExactlyOnceKafkaWriterITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.sink.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ diff --git a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 index 756b92768..8dacea360 100644 --- a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 +++ b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 @@ -14,13 +14,13 @@ Class is annotated with in (KafkaShuffleFetcher.java:0) Constructor (org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber, org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, java.util.Properties, org.apache.flink.api.connector.source.Boundedness, org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState, org.apache.flink.connector.kafka.dynamic.source.enumerator.StoppableKafkaEnumContextProxy$StoppableKafkaEnumContextProxyFactory)> is annotated with in (DynamicKafkaSourceEnumerator.java:0) Constructor (org.apache.flink.api.connector.source.SourceReaderContext, org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema, java.util.Properties)> calls constructor (int)> in (DynamicKafkaSourceReader.java:114) -Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1733) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1710) Constructor (java.util.Set)> is annotated with in (FlinkKafkaProducer.java:0) Constructor (java.lang.String, long, short, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) Constructor (org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) -Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1879) -Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1630) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1848) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1618) Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) Constructor (org.apache.flink.api.common.typeutils.TypeSerializer)> is annotated with in (KafkaShuffleFetcher.java:0) @@ -46,10 +46,10 @@ Method calls method in (DynamicKafkaSourceReader.java:383) Method calls method in (DynamicKafkaSourceReader.java:381) Method calls method in (DynamicKafkaSourceReader.java:496) +Method is annotated with in (ExactlyOnceKafkaWriter.java:0) Method is annotated with in (KafkaSink.java:0) Method calls method in (KafkaSinkBuilder.java:152) Method is annotated with in (KafkaWriter.java:0) -Method is annotated with in (KafkaWriter.java:0) Method is annotated with in (KafkaSource.java:0) Method is annotated with in (KafkaSource.java:0) Method is annotated with in (KafkaSource.java:0) @@ -62,41 +62,41 @@ Method is annotated with in (KafkaPartitionSplitReader.java:0) Method is annotated with in (KafkaSourceReader.java:0) Method is annotated with in (KafkaSourceReader.java:0) -Method calls method in (AbstractFetcher.java:604) -Method calls method in (AbstractFetcher.java:608) -Method calls method in (AbstractFetcher.java:608) -Method calls method in (AbstractFetcher.java:593) -Method calls method in (AbstractFetcher.java:593) -Method calls constructor ([B)> in (KafkaShuffleFetcher.java:240) -Method calls method in (KafkaShuffleFetcher.java:244) +Method calls method in (AbstractFetcher.java:613) +Method calls method in (AbstractFetcher.java:617) +Method calls method in (AbstractFetcher.java:617) +Method calls method in (AbstractFetcher.java:602) +Method calls method in (AbstractFetcher.java:602) +Method calls constructor ([B)> in (KafkaShuffleFetcher.java:241) Method calls method in (KafkaShuffleFetcher.java:245) -Method calls method in (KafkaShuffleFetcher.java:254) -Method calls method in (KafkaShuffleFetcher.java:251) -Method calls method in (KafkaShuffleFetcher.java:255) -Method calls method in (KafkaShuffleFetcher.java:238) -Method gets field in (KafkaShuffleFetcher.java:244) +Method calls method in (KafkaShuffleFetcher.java:246) +Method calls method in (KafkaShuffleFetcher.java:255) +Method calls method in (KafkaShuffleFetcher.java:252) +Method calls method in (KafkaShuffleFetcher.java:256) +Method calls method in (KafkaShuffleFetcher.java:239) Method gets field in (KafkaShuffleFetcher.java:245) -Method gets field in (KafkaShuffleFetcher.java:253) -Method gets field in (KafkaShuffleFetcher.java:250) -Method gets field in (KafkaShuffleFetcher.java:254) +Method gets field in (KafkaShuffleFetcher.java:246) +Method gets field in (KafkaShuffleFetcher.java:254) +Method gets field in (KafkaShuffleFetcher.java:251) +Method gets field in (KafkaShuffleFetcher.java:255) Method is annotated with in (KafkaShuffleFetcher.java:0) -Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:186) -Method calls method in (FlinkKafkaShuffleProducer.java:205) -Method calls method in (FlinkKafkaShuffleProducer.java:204) -Method calls method in (FlinkKafkaShuffleProducer.java:190) -Method calls method in (FlinkKafkaShuffleProducer.java:193) -Method calls method in (FlinkKafkaShuffleProducer.java:195) -Method calls method in (FlinkKafkaShuffleProducer.java:196) -Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:212) -Method calls method in (FlinkKafkaShuffleProducer.java:225) -Method calls method in (FlinkKafkaShuffleProducer.java:224) -Method calls method in (FlinkKafkaShuffleProducer.java:216) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:187) +Method calls method in (FlinkKafkaShuffleProducer.java:206) +Method calls method in (FlinkKafkaShuffleProducer.java:205) +Method calls method in (FlinkKafkaShuffleProducer.java:191) +Method calls method in (FlinkKafkaShuffleProducer.java:194) +Method calls method in (FlinkKafkaShuffleProducer.java:196) +Method calls method in (FlinkKafkaShuffleProducer.java:197) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:213) +Method calls method in (FlinkKafkaShuffleProducer.java:226) +Method calls method in (FlinkKafkaShuffleProducer.java:225) Method calls method in (FlinkKafkaShuffleProducer.java:217) -Method calls method in (FlinkKafkaShuffleProducer.java:218) -Method calls method in (FlinkKafkaShuffleProducer.java:219) +Method calls method in (FlinkKafkaShuffleProducer.java:218) +Method calls method in (FlinkKafkaShuffleProducer.java:219) +Method calls method in (FlinkKafkaShuffleProducer.java:220) Method has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) -Method calls method in (KafkaConnectorOptionsUtil.java:543) -Method calls method in (KafkaConnectorOptionsUtil.java:587) -Method calls method in (KafkaDynamicSink.java:386) +Method calls method in (KafkaConnectorOptionsUtil.java:525) +Method calls method in (KafkaConnectorOptionsUtil.java:569) +Method calls method in (KafkaDynamicSink.java:401) Method has return type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (KafkaDynamicSink.java:0) Method calls method in (KafkaDynamicSource.java:566) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java new file mode 100644 index 000000000..9fbf7ce9c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.TransactionalIdFactory; +import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.ProducerFencedException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Deque; +import java.util.List; +import java.util.Optional; +import java.util.Properties; + +import static org.apache.flink.util.IOUtils.closeAll; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Exactly-once Kafka writer that writes records to Kafka in transactions. + * + * @param The type of the input elements. + */ +class ExactlyOnceKafkaWriter extends KafkaWriter { + private static final Logger LOG = LoggerFactory.getLogger(ExactlyOnceKafkaWriter.class); + private final String transactionalIdPrefix; + + private final KafkaWriterState kafkaWriterState; + // producer pool only used for exactly once + private final Deque> producerPool = + new ArrayDeque<>(); + private final Collection recoveredStates; + private long lastCheckpointId; + + private final Deque producerCloseables = new ArrayDeque<>(); + + /** + * Constructor creating a kafka writer. + * + *

      It will throw a {@link RuntimeException} if {@link + * KafkaRecordSerializationSchema#open(SerializationSchema.InitializationContext, + * KafkaRecordSerializationSchema.KafkaSinkContext)} fails. + * + * @param deliveryGuarantee the Sink's delivery guarantee + * @param kafkaProducerConfig the properties to configure the {@link FlinkKafkaInternalProducer} + * @param transactionalIdPrefix used to create the transactionalIds + * @param sinkInitContext context to provide information about the runtime environment + * @param recordSerializer serialize to transform the incoming records to {@link ProducerRecord} + * @param schemaContext context used to initialize the {@link KafkaRecordSerializationSchema} + * @param recoveredStates state from an previous execution which was covered + */ + ExactlyOnceKafkaWriter( + DeliveryGuarantee deliveryGuarantee, + Properties kafkaProducerConfig, + String transactionalIdPrefix, + Sink.InitContext sinkInitContext, + KafkaRecordSerializationSchema recordSerializer, + SerializationSchema.InitializationContext schemaContext, + Collection recoveredStates) { + super( + deliveryGuarantee, + kafkaProducerConfig, + sinkInitContext, + recordSerializer, + schemaContext); + this.transactionalIdPrefix = + checkNotNull(transactionalIdPrefix, "transactionalIdPrefix must not be null"); + + try { + recordSerializer.open(schemaContext, kafkaSinkContext); + } catch (Exception e) { + throw new FlinkRuntimeException("Cannot initialize schema.", e); + } + + this.kafkaWriterState = new KafkaWriterState(transactionalIdPrefix); + this.lastCheckpointId = + sinkInitContext + .getRestoredCheckpointId() + .orElse(CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1); + + this.recoveredStates = checkNotNull(recoveredStates, "recoveredStates"); + initFlinkMetrics(); + } + + @Override + public void initialize() { + abortLingeringTransactions(recoveredStates, lastCheckpointId + 1); + this.currentProducer = getTransactionalProducer(lastCheckpointId + 1); + this.currentProducer.beginTransaction(); + } + + @Override + public Collection prepareCommit() { + // only return a KafkaCommittable if the current transaction has been written some data + if (currentProducer.hasRecordsInTransaction()) { + final List committables = + Collections.singletonList( + KafkaCommittable.of(currentProducer, producerPool::add)); + LOG.debug("Committing {} committables.", committables); + return committables; + } + + // otherwise, we commit the empty transaction as is (no-op) and just recycle the producer + currentProducer.commitTransaction(); + producerPool.add(currentProducer); + return Collections.emptyList(); + } + + @Override + public List snapshotState(long checkpointId) throws IOException { + currentProducer = getTransactionalProducer(checkpointId + 1); + currentProducer.beginTransaction(); + return Collections.singletonList(kafkaWriterState); + } + + @Override + public void close() throws Exception { + closeAll( + this::abortCurrentProducer, + () -> closeAll(producerPool), + () -> closeAll(producerCloseables), + super::close); + } + + private void abortCurrentProducer() { + // only abort if the transaction is known to the broker (needs to have at least one record + // sent) + if (currentProducer.isInTransaction() && currentProducer.hasRecordsInTransaction()) { + try { + currentProducer.abortTransaction(); + } catch (ProducerFencedException e) { + LOG.debug( + "Producer {} fenced while aborting", currentProducer.getTransactionalId()); + } + } + } + + @VisibleForTesting + Collection> getProducerPool() { + return producerPool; + } + + private void abortLingeringTransactions( + Collection recoveredStates, long startCheckpointId) { + List prefixesToAbort = new ArrayList<>(); + prefixesToAbort.add(transactionalIdPrefix); + + final Optional lastStateOpt = recoveredStates.stream().findFirst(); + if (lastStateOpt.isPresent()) { + KafkaWriterState lastState = lastStateOpt.get(); + if (!lastState.getTransactionalIdPrefix().equals(transactionalIdPrefix)) { + prefixesToAbort.add(lastState.getTransactionalIdPrefix()); + LOG.warn( + "Transactional id prefix from previous execution {} has changed to {}.", + lastState.getTransactionalIdPrefix(), + transactionalIdPrefix); + } + } + + try (TransactionAborter transactionAborter = + new TransactionAborter( + kafkaSinkContext.getParallelInstanceId(), + kafkaSinkContext.getNumberOfParallelInstances(), + this::getOrCreateTransactionalProducer, + producerPool::add)) { + transactionAborter.abortLingeringTransactions(prefixesToAbort, startCheckpointId); + } + } + + /** + * For each checkpoint we create new {@link FlinkKafkaInternalProducer} so that new transactions + * will not clash with transactions created during previous checkpoints ({@code + * producer.initTransactions()} assures that we obtain new producerId and epoch counters). + * + *

      Ensures that all transaction ids in between lastCheckpointId and checkpointId are + * initialized. + */ + private FlinkKafkaInternalProducer getTransactionalProducer(long checkpointId) { + checkState( + checkpointId > lastCheckpointId, + "Expected %s > %s", + checkpointId, + lastCheckpointId); + FlinkKafkaInternalProducer producer = null; + // in case checkpoints have been aborted, Flink would create non-consecutive transaction ids + // this loop ensures that all gaps are filled with initialized (empty) transactions + for (long id = lastCheckpointId + 1; id <= checkpointId; id++) { + String transactionalId = + TransactionalIdFactory.buildTransactionalId( + transactionalIdPrefix, kafkaSinkContext.getParallelInstanceId(), id); + producer = getOrCreateTransactionalProducer(transactionalId); + } + this.lastCheckpointId = checkpointId; + assert producer != null; + LOG.info("Created new transactional producer {}", producer.getTransactionalId()); + return producer; + } + + private FlinkKafkaInternalProducer getOrCreateTransactionalProducer( + String transactionalId) { + FlinkKafkaInternalProducer producer = producerPool.poll(); + if (producer == null) { + producer = new FlinkKafkaInternalProducer<>(kafkaProducerConfig, transactionalId); + producerCloseables.add(producer); + producer.initTransactions(); + initKafkaMetrics(producer); + } else { + producer.initTransactionId(transactionalId); + } + return producer; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java index d3d3c89df..1fb98e6c0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java @@ -110,28 +110,35 @@ public SimpleVersionedSerializer getCommittableSerializer() { @Internal @Override public KafkaWriter createWriter(InitContext context) throws IOException { - return new KafkaWriter( - deliveryGuarantee, - kafkaProducerConfig, - transactionalIdPrefix, - context, - recordSerializer, - context.asSerializationSchemaInitializationContext(), - Collections.emptyList()); + return restoreWriter(context, Collections.emptyList()); } @Internal @Override public KafkaWriter restoreWriter( - InitContext context, Collection recoveredState) throws IOException { - return new KafkaWriter<>( - deliveryGuarantee, - kafkaProducerConfig, - transactionalIdPrefix, - context, - recordSerializer, - context.asSerializationSchemaInitializationContext(), - recoveredState); + InitContext context, Collection recoveredState) { + KafkaWriter writer; + if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) { + writer = + new ExactlyOnceKafkaWriter<>( + deliveryGuarantee, + kafkaProducerConfig, + transactionalIdPrefix, + context, + recordSerializer, + context.asSerializationSchemaInitializationContext(), + recoveredState); + } else { + writer = + new KafkaWriter<>( + deliveryGuarantee, + kafkaProducerConfig, + context, + recordSerializer, + context.asSerializationSchemaInitializationContext()); + } + writer.initialize(); + return writer; } @Internal diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java index 0ef37854d..ec8b7a223 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java @@ -25,11 +25,9 @@ import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.MetricUtil; import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; -import org.apache.flink.connector.kafka.sink.internal.TransactionalIdFactory; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; -import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricMutableWrapper; import org.apache.flink.util.FlinkRuntimeException; @@ -38,7 +36,6 @@ import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.UnknownProducerIdException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,15 +43,11 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.ArrayDeque; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Deque; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Properties; import java.util.function.Consumer; @@ -63,8 +56,7 @@ import static org.apache.flink.util.Preconditions.checkState; /** - * This class is responsible to write records in a Kafka topic and to handle the different delivery - * {@link DeliveryGuarantee}s. + * This class is responsible to write records in a Kafka topic without transactions. * * @param The type of the input elements. */ @@ -81,11 +73,10 @@ class KafkaWriter private static final String KAFKA_PRODUCER_METRICS = "producer-metrics"; private final DeliveryGuarantee deliveryGuarantee; - private final Properties kafkaProducerConfig; - private final String transactionalIdPrefix; - private final KafkaRecordSerializationSchema recordSerializer; - private final Callback deliveryCallback; - private final KafkaRecordSerializationSchema.KafkaSinkContext kafkaSinkContext; + protected final Properties kafkaProducerConfig; + protected final KafkaRecordSerializationSchema recordSerializer; + protected final Callback deliveryCallback; + protected final KafkaRecordSerializationSchema.KafkaSinkContext kafkaSinkContext; private volatile Exception asyncProducerException; private final Map previouslyCreatedMetrics = new HashMap<>(); private final SinkWriterMetricGroup metricGroup; @@ -98,14 +89,7 @@ class KafkaWriter // Number of outgoing bytes at the latest metric sync private long latestOutgoingByteTotal; private Metric byteOutMetric; - private FlinkKafkaInternalProducer currentProducer; - private final KafkaWriterState kafkaWriterState; - // producer pool only used for exactly once - private final Deque> producerPool = - new ArrayDeque<>(); - private long lastCheckpointId; - - private final Deque producerCloseables = new ArrayDeque<>(); + protected FlinkKafkaInternalProducer currentProducer; private boolean closed = false; private long lastSync = System.currentTimeMillis(); @@ -119,23 +103,18 @@ class KafkaWriter * * @param deliveryGuarantee the Sink's delivery guarantee * @param kafkaProducerConfig the properties to configure the {@link FlinkKafkaInternalProducer} - * @param transactionalIdPrefix used to create the transactionalIds * @param sinkInitContext context to provide information about the runtime environment * @param recordSerializer serialize to transform the incoming records to {@link ProducerRecord} * @param schemaContext context used to initialize the {@link KafkaRecordSerializationSchema} - * @param recoveredStates state from an previous execution which was covered */ KafkaWriter( DeliveryGuarantee deliveryGuarantee, Properties kafkaProducerConfig, - String transactionalIdPrefix, Sink.InitContext sinkInitContext, KafkaRecordSerializationSchema recordSerializer, - SerializationSchema.InitializationContext schemaContext, - Collection recoveredStates) { + SerializationSchema.InitializationContext schemaContext) { this.deliveryGuarantee = checkNotNull(deliveryGuarantee, "deliveryGuarantee"); this.kafkaProducerConfig = checkNotNull(kafkaProducerConfig, "kafkaProducerConfig"); - this.transactionalIdPrefix = checkNotNull(transactionalIdPrefix, "transactionalIdPrefix"); this.recordSerializer = checkNotNull(recordSerializer, "recordSerializer"); checkNotNull(sinkInitContext, "sinkInitContext"); this.deliveryCallback = @@ -165,29 +144,14 @@ class KafkaWriter throw new FlinkRuntimeException("Cannot initialize schema.", e); } - this.kafkaWriterState = new KafkaWriterState(transactionalIdPrefix); - this.lastCheckpointId = - sinkInitContext - .getRestoredCheckpointId() - .orElse(CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1); - if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) { - abortLingeringTransactions( - checkNotNull(recoveredStates, "recoveredStates"), lastCheckpointId + 1); - this.currentProducer = getTransactionalProducer(lastCheckpointId + 1); - this.currentProducer.beginTransaction(); - } else if (deliveryGuarantee == DeliveryGuarantee.AT_LEAST_ONCE - || deliveryGuarantee == DeliveryGuarantee.NONE) { - this.currentProducer = new FlinkKafkaInternalProducer<>(this.kafkaProducerConfig, null); - producerCloseables.add(this.currentProducer); - initKafkaMetrics(this.currentProducer); - } else { - throw new UnsupportedOperationException( - "Unsupported Kafka writer semantic " + this.deliveryGuarantee); - } - initFlinkMetrics(); } + public void initialize() { + this.currentProducer = new FlinkKafkaInternalProducer<>(this.kafkaProducerConfig); + initKafkaMetrics(this.currentProducer); + } + @Override public void write(@Nullable IN element, Context context) throws IOException { checkAsyncException(); @@ -211,40 +175,19 @@ public void flush(boolean endOfInput) throws IOException, InterruptedException { @Override public Collection prepareCommit() { - if (deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE) { - return Collections.emptyList(); - } - - // only return a KafkaCommittable if the current transaction has been written some data - if (currentProducer.hasRecordsInTransaction()) { - final List committables = - Collections.singletonList( - KafkaCommittable.of(currentProducer, producerPool::add)); - LOG.debug("Committing {} committables.", committables); - return committables; - } - - // otherwise, we commit the empty transaction as is (no-op) and just recycle the producer - currentProducer.commitTransaction(); - producerPool.add(currentProducer); return Collections.emptyList(); } @Override public List snapshotState(long checkpointId) throws IOException { - if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) { - currentProducer = getTransactionalProducer(checkpointId + 1); - currentProducer.beginTransaction(); - } - return Collections.singletonList(kafkaWriterState); + return Collections.emptyList(); } @Override public void close() throws Exception { closed = true; LOG.debug("Closing writer with {}", currentProducer); - closeAll(this::abortCurrentProducer, producerPool::clear); - closeAll(producerCloseables); + closeAll(currentProducer); checkState( currentProducer.isClosed(), "Could not close current producer " + currentProducer); currentProducer = null; @@ -253,103 +196,17 @@ public void close() throws Exception { checkAsyncException(); } - private void abortCurrentProducer() { - if (currentProducer.isInTransaction()) { - try { - currentProducer.abortTransaction(); - } catch (ProducerFencedException e) { - LOG.debug( - "Producer {} fenced while aborting", currentProducer.getTransactionalId()); - } - } - } - - @VisibleForTesting - Deque> getProducerPool() { - return producerPool; - } - @VisibleForTesting FlinkKafkaInternalProducer getCurrentProducer() { return currentProducer; } - void abortLingeringTransactions( - Collection recoveredStates, long startCheckpointId) { - List prefixesToAbort = new ArrayList<>(); - prefixesToAbort.add(transactionalIdPrefix); - - final Optional lastStateOpt = recoveredStates.stream().findFirst(); - if (lastStateOpt.isPresent()) { - KafkaWriterState lastState = lastStateOpt.get(); - if (!lastState.getTransactionalIdPrefix().equals(transactionalIdPrefix)) { - prefixesToAbort.add(lastState.getTransactionalIdPrefix()); - LOG.warn( - "Transactional id prefix from previous execution {} has changed to {}.", - lastState.getTransactionalIdPrefix(), - transactionalIdPrefix); - } - } - - try (TransactionAborter transactionAborter = - new TransactionAborter( - kafkaSinkContext.getParallelInstanceId(), - kafkaSinkContext.getNumberOfParallelInstances(), - this::getOrCreateTransactionalProducer, - producerPool::add)) { - transactionAborter.abortLingeringTransactions(prefixesToAbort, startCheckpointId); - } - } - - /** - * For each checkpoint we create new {@link FlinkKafkaInternalProducer} so that new transactions - * will not clash with transactions created during previous checkpoints ({@code - * producer.initTransactions()} assures that we obtain new producerId and epoch counters). - * - *

      Ensures that all transaction ids in between lastCheckpointId and checkpointId are - * initialized. - */ - private FlinkKafkaInternalProducer getTransactionalProducer(long checkpointId) { - checkState( - checkpointId > lastCheckpointId, - "Expected %s > %s", - checkpointId, - lastCheckpointId); - FlinkKafkaInternalProducer producer = null; - // in case checkpoints have been aborted, Flink would create non-consecutive transaction ids - // this loop ensures that all gaps are filled with initialized (empty) transactions - for (long id = lastCheckpointId + 1; id <= checkpointId; id++) { - String transactionalId = - TransactionalIdFactory.buildTransactionalId( - transactionalIdPrefix, kafkaSinkContext.getParallelInstanceId(), id); - producer = getOrCreateTransactionalProducer(transactionalId); - } - this.lastCheckpointId = checkpointId; - assert producer != null; - LOG.info("Created new transactional producer {}", producer.getTransactionalId()); - return producer; - } - - private FlinkKafkaInternalProducer getOrCreateTransactionalProducer( - String transactionalId) { - FlinkKafkaInternalProducer producer = producerPool.poll(); - if (producer == null) { - producer = new FlinkKafkaInternalProducer<>(kafkaProducerConfig, transactionalId); - producerCloseables.add(producer); - producer.initTransactions(); - initKafkaMetrics(producer); - } else { - producer.initTransactionId(transactionalId); - } - return producer; - } - - private void initFlinkMetrics() { + protected void initFlinkMetrics() { metricGroup.setCurrentSendTimeGauge(this::computeSendTime); registerMetricSync(); } - private void initKafkaMetrics(FlinkKafkaInternalProducer producer) { + protected void initKafkaMetrics(FlinkKafkaInternalProducer producer) { byteOutMetric = MetricUtil.getKafkaMetric( producer.metrics(), KAFKA_PRODUCER_METRICS, "outgoing-byte-total"); diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java index 4c482acb5..4edf6651c 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java @@ -60,16 +60,18 @@ public class FlinkKafkaInternalProducer extends KafkaProducer { private volatile boolean hasRecordsInTransaction; private volatile boolean closed; - public FlinkKafkaInternalProducer(Properties properties, @Nullable String transactionalId) { + public FlinkKafkaInternalProducer(Properties properties) { + super(properties); + LOG.debug("Created non-transactional {}", this); + } + + public FlinkKafkaInternalProducer(Properties properties, String transactionalId) { super(withTransactionalId(properties, transactionalId)); this.transactionalId = transactionalId; + LOG.debug("Created transactional {}", this); } - private static Properties withTransactionalId( - Properties properties, @Nullable String transactionalId) { - if (transactionalId == null) { - return properties; - } + private static Properties withTransactionalId(Properties properties, String transactionalId) { Properties props = new Properties(); props.putAll(properties); props.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId); @@ -95,6 +97,7 @@ public void flush() { @Override public void beginTransaction() throws ProducerFencedException { super.beginTransaction(); + LOG.debug("beginTransaction {}", transactionalId); inTransaction = true; } @@ -126,23 +129,20 @@ public boolean hasRecordsInTransaction() { @Override public void close() { - closed = true; - if (inTransaction) { - // This is state is most likely reached in case of a failure. - // If this producer is still in transaction, it should be committing. - // However, at this point, we cannot decide that and we shouldn't prolong cancellation. - // So hard kill this producer with all resources. + if (!closed) { + LOG.debug("Closing immediately {}", this); super.close(Duration.ZERO); - } else { - // If this is outside of a transaction, we should be able to cleanly shutdown. - super.close(Duration.ofHours(1)); + closed = true; } } @Override public void close(Duration timeout) { - closed = true; - super.close(timeout); + if (!closed) { + LOG.debug("Closing with {} timeout {}", timeout, this); + super.close(timeout); + closed = true; + } } public boolean isClosed() { @@ -395,13 +395,8 @@ private static void transitionTransactionManagerStateTo( @Override public String toString() { - return "FlinkKafkaInternalProducer{" - + "transactionalId='" - + transactionalId - + "', inTransaction=" - + inTransaction - + ", closed=" - + closed - + '}'; + return String.format( + "FlinkKafkaInternalProducer@%d{transactionalId='%s', inTransaction=%s, closed=%s}", + System.identityHashCode(this), transactionalId, inTransaction, closed); } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java new file mode 100644 index 000000000..69448caff --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java @@ -0,0 +1,309 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.TestLoggerExtension; + +import com.google.common.collect.Iterables; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.ProducerFencedException; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Properties; + +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode; + +/** Tests for the standalone KafkaWriter. */ +@ExtendWith(TestLoggerExtension.class) +public class ExactlyOnceKafkaWriterITCase extends KafkaWriterTestBase { + + @Test + void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); + + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + final KafkaWriter writer = + createWriter( + DeliveryGuarantee.EXACTLY_ONCE, + new SinkInitContext(metricGroup, timeService, null)); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + + // test flush + assertThatCode(() -> writer.flush(false)) + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .as("the exception is not thrown again") + .doesNotThrowAnyException(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + // async exception is checked and thrown on close + assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); + } + + @Test + void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); + + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + final KafkaWriter writer = + createWriter( + DeliveryGuarantee.EXACTLY_ONCE, + new SinkInitContext(metricGroup, timeService, null)); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + // to ensure that the exceptional send request has completed + writer.getCurrentProducer().flush(); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .as("the exception is not thrown again") + .doesNotThrowAnyException(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + // async exception is checked and thrown on close + assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); + } + + @Test + void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); + + SinkInitContext sinkInitContext = + new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); + + final KafkaWriter writer = + createWriter(DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = + sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + // to ensure that the exceptional send request has completed + writer.getCurrentProducer().flush(); + + assertThatCode( + () -> { + while (sinkInitContext.getMailboxExecutor().tryYield()) { + // execute all mails + } + }) + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .as("the exception is not thrown again") + .doesNotThrowAnyException(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + // async exception is checked and thrown on close + assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); + } + + @Test + void testCloseAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); + + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + final KafkaWriter writer = + createWriter( + DeliveryGuarantee.EXACTLY_ONCE, + new SinkInitContext(metricGroup, timeService, null)); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + // to ensure that the exceptional send request has completed + writer.getCurrentProducer().flush(); + + // test flush + assertThatCode(writer::close) + .as("flush should throw the exception from the WriterCallback") + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + } + + private void triggerProducerException(KafkaWriter writer, Properties properties) + throws IOException { + final String transactionalId = writer.getCurrentProducer().getTransactionalId(); + + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(properties, transactionalId)) { + producer.initTransactions(); + producer.beginTransaction(); + producer.send(new ProducerRecord<>(topic, "1".getBytes())); + producer.commitTransaction(); + } + + writer.write(1, SINK_WRITER_CONTEXT); + } + + /** Test that producer is not accidentally recreated or pool is used. */ + @Test + void testLingeringTransaction() throws Exception { + final KafkaWriter failedWriter = createWriter(DeliveryGuarantee.EXACTLY_ONCE); + + // create two lingering transactions + failedWriter.flush(false); + failedWriter.prepareCommit(); + failedWriter.snapshotState(1); + failedWriter.flush(false); + failedWriter.prepareCommit(); + failedWriter.snapshotState(2); + + try (final KafkaWriter recoveredWriter = + createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + recoveredWriter.write(1, SINK_WRITER_CONTEXT); + + recoveredWriter.flush(false); + Collection committables = recoveredWriter.prepareCommit(); + recoveredWriter.snapshotState(1); + assertThat(committables).hasSize(1); + final KafkaCommittable committable = committables.stream().findFirst().get(); + assertThat(committable.getProducer().isPresent()).isTrue(); + + committable.getProducer().get().getObject().commitTransaction(); + + List> records = + drainAllRecordsFromTopic(topic, getKafkaClientConfiguration(), true); + assertThat(records).hasSize(1); + } + + failedWriter.close(); + } + + /** Test that producers are reused when committed. */ + @Test + void usePooledProducerForTransactional() throws Exception { + try (final ExactlyOnceKafkaWriter writer = + createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + assertThat(writer.getProducerPool()).hasSize(0); + + writer.write(1, SINK_WRITER_CONTEXT); + writer.flush(false); + Collection committables0 = writer.prepareCommit(); + writer.snapshotState(1); + assertThat(committables0).hasSize(1); + final KafkaCommittable committable = committables0.stream().findFirst().get(); + assertThat(committable.getProducer().isPresent()).isTrue(); + + FlinkKafkaInternalProducer firstProducer = + committable.getProducer().get().getObject(); + assertThat(firstProducer != writer.getCurrentProducer()) + .as("Expected different producer") + .isTrue(); + + // recycle first producer, KafkaCommitter would commit it and then return it + assertThat(writer.getProducerPool()).hasSize(0); + firstProducer.commitTransaction(); + committable.getProducer().get().close(); + assertThat(writer.getProducerPool()).hasSize(1); + + writer.write(1, SINK_WRITER_CONTEXT); + writer.flush(false); + Collection committables1 = writer.prepareCommit(); + writer.snapshotState(2); + assertThat(committables1).hasSize(1); + final KafkaCommittable committable1 = committables1.stream().findFirst().get(); + assertThat(committable1.getProducer().isPresent()).isTrue(); + + assertThat(firstProducer == writer.getCurrentProducer()) + .as("Expected recycled producer") + .isTrue(); + } + } + + /** + * Tests that if a pre-commit attempt occurs on an empty transaction, the writer should not emit + * a KafkaCommittable, and instead immediately commit the empty transaction and recycle the + * producer. + */ + @Test + void prepareCommitForEmptyTransaction() throws Exception { + try (final ExactlyOnceKafkaWriter writer = + createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + assertThat(writer.getProducerPool()).hasSize(0); + + // no data written to current transaction + writer.flush(false); + Collection emptyCommittables = writer.prepareCommit(); + + assertThat(emptyCommittables).hasSize(0); + assertThat(writer.getProducerPool()).hasSize(1); + final FlinkKafkaInternalProducer recycledProducer = + Iterables.getFirst(writer.getProducerPool(), null); + assertThat(recycledProducer.isInTransaction()).isFalse(); + } + } + + /** + * Tests that open transactions are automatically aborted on close such that successive writes + * succeed. + */ + @Test + void testAbortOnClose() throws Exception { + Properties properties = getKafkaClientConfiguration(); + try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + writer.write(1, SINK_WRITER_CONTEXT); + assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(0); + } + + try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + writer.write(2, SINK_WRITER_CONTEXT); + writer.flush(false); + Collection committables = writer.prepareCommit(); + writer.snapshotState(1L); + + // manually commit here, which would only succeed if the first transaction was aborted + assertThat(committables).hasSize(1); + final KafkaCommittable committable = committables.stream().findFirst().get(); + String transactionalId = committable.getTransactionalId(); + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(properties, transactionalId)) { + producer.resumeTransaction(committable.getProducerId(), committable.getEpoch()); + producer.commitTransaction(); + } + + assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(1); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java index 20af1b575..9ca558f33 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java @@ -23,11 +23,7 @@ import org.apache.kafka.common.errors.NetworkException; import org.apache.kafka.common.errors.TimeoutException; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import java.util.Properties; @@ -42,21 +38,6 @@ public class KafkaWriterFaultToleranceITCase extends KafkaWriterTestBase { private static final String INIT_KAFKA_MAX_BLOCK_MS = "1000"; private static final String INIT_KAFKA_DELIVERY_TIMEOUT_MS = "1000"; - @BeforeAll - public static void beforeAll() { - KAFKA_CONTAINER.start(); - } - - @AfterAll - public static void afterAll() { - KAFKA_CONTAINER.stop(); - } - - @BeforeEach - public void setUp(TestInfo testInfo) { - super.setUp(testInfo); - } - @Test void testWriteExceptionWhenKafkaUnavailable() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index 1494c84f3..805690afa 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -24,14 +24,7 @@ import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.util.TestLoggerExtension; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.errors.ProducerFencedException; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -44,29 +37,11 @@ import java.util.Properties; import java.util.stream.IntStream; -import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode; /** Tests for the standalone KafkaWriter. */ @ExtendWith(TestLoggerExtension.class) public class KafkaWriterITCase extends KafkaWriterTestBase { - - @BeforeAll - public static void beforeAll() { - KAFKA_CONTAINER.start(); - } - - @AfterAll - public static void afterAll() { - KAFKA_CONTAINER.stop(); - } - - @BeforeEach - public void setUp(TestInfo testInfo) { - super.setUp(testInfo); - } - @ParameterizedTest @EnumSource(DeliveryGuarantee.class) public void testRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { @@ -149,140 +124,6 @@ public void testCurrentSendTimeMetric() throws Exception { } } - @Test - void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { - Properties properties = getKafkaClientConfiguration(); - - final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - - final KafkaWriter writer = - createWriter( - DeliveryGuarantee.EXACTLY_ONCE, - new SinkInitContext(metricGroup, timeService, null)); - final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); - - triggerProducerException(writer, properties); - - // test flush - assertThatCode(() -> writer.flush(false)) - .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); - - assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) - .as("the exception is not thrown again") - .doesNotThrowAnyException(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); - - // async exception is checked and thrown on close - assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); - } - - @Test - void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { - Properties properties = getKafkaClientConfiguration(); - - final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - - final KafkaWriter writer = - createWriter( - DeliveryGuarantee.EXACTLY_ONCE, - new SinkInitContext(metricGroup, timeService, null)); - final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); - - triggerProducerException(writer, properties); - // to ensure that the exceptional send request has completed - writer.getCurrentProducer().flush(); - - assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) - .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); - - assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) - .as("the exception is not thrown again") - .doesNotThrowAnyException(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); - - // async exception is checked and thrown on close - assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); - } - - @Test - void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { - Properties properties = getKafkaClientConfiguration(); - - SinkInitContext sinkInitContext = - new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); - - final KafkaWriter writer = - createWriter(DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); - final Counter numRecordsOutErrors = - sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); - - triggerProducerException(writer, properties); - // to ensure that the exceptional send request has completed - writer.getCurrentProducer().flush(); - - assertThatCode( - () -> { - while (sinkInitContext.getMailboxExecutor().tryYield()) { - // execute all mails - } - }) - .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); - - assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); - - assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) - .as("the exception is not thrown again") - .doesNotThrowAnyException(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); - - // async exception is checked and thrown on close - assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); - } - - @Test - void testCloseAsyncErrorPropagationAndErrorCounter() throws Exception { - Properties properties = getKafkaClientConfiguration(); - - final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - - final KafkaWriter writer = - createWriter( - DeliveryGuarantee.EXACTLY_ONCE, - new SinkInitContext(metricGroup, timeService, null)); - final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); - - triggerProducerException(writer, properties); - // to ensure that the exceptional send request has completed - writer.getCurrentProducer().flush(); - - // test flush - assertThatCode(writer::close) - .as("flush should throw the exception from the WriterCallback") - .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); - } - - private void triggerProducerException(KafkaWriter writer, Properties properties) - throws IOException { - final String transactionalId = writer.getCurrentProducer().getTransactionalId(); - - try (FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, transactionalId)) { - producer.initTransactions(); - producer.beginTransaction(); - producer.send(new ProducerRecord(topic, "1".getBytes())); - producer.commitTransaction(); - } - - writer.write(1, SINK_WRITER_CONTEXT); - } - @Test public void testMetadataPublisher() throws Exception { List metadataList = new ArrayList<>(); @@ -304,40 +145,6 @@ public void testMetadataPublisher() throws Exception { } } - /** Test that producer is not accidentally recreated or pool is used. */ - @Test - void testLingeringTransaction() throws Exception { - final KafkaWriter failedWriter = createWriter(DeliveryGuarantee.EXACTLY_ONCE); - - // create two lingering transactions - failedWriter.flush(false); - failedWriter.prepareCommit(); - failedWriter.snapshotState(1); - failedWriter.flush(false); - failedWriter.prepareCommit(); - failedWriter.snapshotState(2); - - try (final KafkaWriter recoveredWriter = - createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { - recoveredWriter.write(1, SINK_WRITER_CONTEXT); - - recoveredWriter.flush(false); - Collection committables = recoveredWriter.prepareCommit(); - recoveredWriter.snapshotState(1); - assertThat(committables).hasSize(1); - final KafkaCommittable committable = committables.stream().findFirst().get(); - assertThat(committable.getProducer().isPresent()).isTrue(); - - committable.getProducer().get().getObject().commitTransaction(); - - List> records = - drainAllRecordsFromTopic(topic, getKafkaClientConfiguration(), true); - assertThat(records).hasSize(1); - } - - failedWriter.close(); - } - /** Test that producer is not accidentally recreated or pool is used. */ @ParameterizedTest @EnumSource( @@ -346,8 +153,6 @@ void testLingeringTransaction() throws Exception { mode = EnumSource.Mode.EXCLUDE) void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exception { try (final KafkaWriter writer = createWriter(guarantee)) { - assertThat(writer.getProducerPool()).hasSize(0); - FlinkKafkaInternalProducer firstProducer = writer.getCurrentProducer(); writer.flush(false); Collection committables = writer.prepareCommit(); @@ -357,101 +162,6 @@ void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exce assertThat(writer.getCurrentProducer() == firstProducer) .as("Expected same producer") .isTrue(); - assertThat(writer.getProducerPool()).hasSize(0); - } - } - - /** Test that producers are reused when committed. */ - @Test - void usePooledProducerForTransactional() throws Exception { - try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { - assertThat(writer.getProducerPool()).hasSize(0); - - writer.write(1, SINK_WRITER_CONTEXT); - writer.flush(false); - Collection committables0 = writer.prepareCommit(); - writer.snapshotState(1); - assertThat(committables0).hasSize(1); - final KafkaCommittable committable = committables0.stream().findFirst().get(); - assertThat(committable.getProducer().isPresent()).isTrue(); - - FlinkKafkaInternalProducer firstProducer = - committable.getProducer().get().getObject(); - assertThat(firstProducer != writer.getCurrentProducer()) - .as("Expected different producer") - .isTrue(); - - // recycle first producer, KafkaCommitter would commit it and then return it - assertThat(writer.getProducerPool()).hasSize(0); - firstProducer.commitTransaction(); - committable.getProducer().get().close(); - assertThat(writer.getProducerPool()).hasSize(1); - - writer.write(1, SINK_WRITER_CONTEXT); - writer.flush(false); - Collection committables1 = writer.prepareCommit(); - writer.snapshotState(2); - assertThat(committables1).hasSize(1); - final KafkaCommittable committable1 = committables1.stream().findFirst().get(); - assertThat(committable1.getProducer().isPresent()).isTrue(); - - assertThat(firstProducer == writer.getCurrentProducer()) - .as("Expected recycled producer") - .isTrue(); - } - } - - /** - * Tests that if a pre-commit attempt occurs on an empty transaction, the writer should not emit - * a KafkaCommittable, and instead immediately commit the empty transaction and recycle the - * producer. - */ - @Test - void prepareCommitForEmptyTransaction() throws Exception { - try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { - assertThat(writer.getProducerPool()).hasSize(0); - - // no data written to current transaction - writer.flush(false); - Collection emptyCommittables = writer.prepareCommit(); - - assertThat(emptyCommittables).hasSize(0); - assertThat(writer.getProducerPool()).hasSize(1); - final FlinkKafkaInternalProducer recycledProducer = - writer.getProducerPool().pop(); - assertThat(recycledProducer.isInTransaction()).isFalse(); - } - } - - /** - * Tests that open transactions are automatically aborted on close such that successive writes - * succeed. - */ - @Test - void testAbortOnClose() throws Exception { - Properties properties = getKafkaClientConfiguration(); - try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { - writer.write(1, SINK_WRITER_CONTEXT); - assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(0); - } - - try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { - writer.write(2, SINK_WRITER_CONTEXT); - writer.flush(false); - Collection committables = writer.prepareCommit(); - writer.snapshotState(1L); - - // manually commit here, which would only succeed if the first transaction was aborted - assertThat(committables).hasSize(1); - final KafkaCommittable committable = committables.stream().findFirst().get(); - String transactionalId = committable.getTransactionalId(); - try (FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, transactionalId)) { - producer.resumeTransaction(committable.getProducerId(), committable.getEpoch()); - producer.commitTransaction(); - } - - assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(1); } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java index 46f62eab3..eaf1ac1dc 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java @@ -46,6 +46,8 @@ import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.containers.Network; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import javax.annotation.Nullable; @@ -66,6 +68,7 @@ import static org.apache.kafka.clients.admin.AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG; /** Test base for KafkaWriter. */ +@Testcontainers public abstract class KafkaWriterTestBase { protected static final Logger LOG = LoggerFactory.getLogger(KafkaWriterTestBase.class); @@ -80,7 +83,8 @@ public abstract class KafkaWriterTestBase { protected MetricListener metricListener; protected TriggerTimeService timeService; - protected static final KafkaContainer KAFKA_CONTAINER = + @Container + public static final KafkaContainer KAFKA_CONTAINER = createKafkaContainer(KafkaWriterTestBase.class) .withEmbeddedZookeeper() .withNetwork(NETWORK) @@ -103,19 +107,20 @@ public void check() { checkProducerLeak(); } - KafkaWriter createWriter(DeliveryGuarantee guarantee) throws IOException { + > T createWriter(DeliveryGuarantee guarantee) throws IOException { return createWriter(guarantee, createInitContext()); } - KafkaWriter createWriter(DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) - throws IOException { + > T createWriter( + DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) throws IOException { return createWriter(builder -> builder.setDeliveryGuarantee(guarantee), sinkInitContext); } - KafkaWriter createWriter( + @SuppressWarnings("unchecked") + > T createWriter( Consumer> sinkBuilderAdjuster, SinkInitContext sinkInitContext) throws IOException { - return (KafkaWriter) createSink(sinkBuilderAdjuster).createWriter(sinkInitContext); + return (T) createSink(sinkBuilderAdjuster).createWriter(sinkInitContext); } KafkaSink createSink(Consumer> sinkBuilderAdjuster) { diff --git a/tools/releasing/shared b/tools/releasing/shared index 45bb68f55..c41156180 160000 --- a/tools/releasing/shared +++ b/tools/releasing/shared @@ -1 +1 @@ -Subproject commit 45bb68f550874ec398b1da344dd4d82e703b4be1 +Subproject commit c4115618085ac046033368e8e3a7eee59874608f From db07aa67c483e973d049a4df566d96394b852d8a Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Mon, 10 Feb 2025 15:45:25 +0100 Subject: [PATCH 317/322] [FLINK-37282] Add Backchannel Backchannel provides a way for the committer to communicate to the writer even in (simple) non-chained cases thanks to colocation contraints. It's the same trick that is employed in statefun. A backchannel is stateless, however, because its state can be entirely derived from committer state. Thus, it's much easier to handle than a statefun backchannel. Backchannel will be used to communicate the committed transactions to the writer in future commits. --- .../kafka/sink/internal/Backchannel.java | 43 ++++ .../sink/internal/BackchannelFactory.java | 123 +++++++++++ .../kafka/sink/internal/BackchannelImpl.java | 142 ++++++++++++ .../sink/internal/ReadableBackchannel.java | 35 +++ .../sink/internal/WritableBackchannel.java | 32 +++ .../sink/internal/BackchannelImplTest.java | 206 ++++++++++++++++++ 6 files changed, 581 insertions(+) create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/Backchannel.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelFactory.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImpl.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ReadableBackchannel.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/WritableBackchannel.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImplTest.java diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/Backchannel.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/Backchannel.java new file mode 100644 index 000000000..244b76683 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/Backchannel.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import java.io.Closeable; + +/** + * A backchannel for communication between the commiter -> writer. It's used to signal that certain + * transactions have been committed and respective producers are good to be reused. + * + *

      The model closely follows the idea of statefun except that there is no need to checkpoint the + * state since the backchannel will fully recover on restart from the committer state. + * + *

      Establishing a backchannel for Kafka sink works because there is only writer and committer and + * nothing in between these two operators. In most cases, these two are chained in live inside the + * same task thread. In rare cases, committer and writer are not chained, so writer and committer + * are in different tasks and threads. However, because of colocations of tasks, we still know that + * both instances will run inside the same JVM and we can establish a backchannel between them. The + * latter case requires some synchronization in the buffer. + */ +public interface Backchannel extends Closeable { + /** Check if the backchannel is fully established. */ + boolean isEstablished(); + + @Override + void close(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelFactory.java new file mode 100644 index 000000000..65c2bdf80 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelFactory.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple3; + +import javax.annotation.concurrent.ThreadSafe; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; + +/** + * Creates and manages backchannels for the Kafka sink. The backchannels are used to communicate + * between the Kafka committer and writer. + * + *

      Each backchannel is uniquely identified by the subtask id, attempt number, and transactional + * id prefix. The subtask id prevents concurrent subtasks of the same sink from conflicting. The + * attempt number prevents conflicts for task-local recovery. The transactional id prefix prevents + * conflicts between multiple Kafka sinks. + */ +@Internal +@ThreadSafe +public class BackchannelFactory { + private static final BackchannelFactory INSTANCE = new BackchannelFactory(); + + /** Gets the singleton instance of the {@link BackchannelFactory}. */ + public static BackchannelFactory getInstance() { + return INSTANCE; + } + + private BackchannelFactory() {} + + /** + * The map of backchannels, keyed by the subtask id and transactional id prefix to uniquely + * identify the backchannel while establishing the connection. + */ + private final Map, BackchannelImpl> backchannels = + new ConcurrentHashMap<>(); + + /** + * Gets a {@link ReadableBackchannel} for the given subtask, attempt, and transactional id + * prefix. + * + *

      If this method is called twice with the same arguments, it will throw an exception as it + * indicates that the transactional id prefix is being reused for multiple Kafka sinks. + * + *

      If the corresponding {@link #getWritableBackchannel(int, int, String)} is called, the + * {@link ReadableBackchannel#isEstablished()} will return true. + */ + @SuppressWarnings("unchecked") + public ReadableBackchannel getReadableBackchannel( + int subtaskId, int attemptNumber, String transactionalIdPrefix) { + return (ReadableBackchannel) + getBackchannel( + subtaskId, + attemptNumber, + transactionalIdPrefix, + BackchannelImpl::createReadableBackchannel); + } + + /** + * Gets a {@link WritableBackchannel} for the given subtask, attempt, and transactional id + * prefix. + * + *

      If this method is called twice with the same arguments, it will throw an exception as it + * indicates that the transactional id prefix is being reused for multiple Kafka sinks. + * + *

      If the corresponding {@link #getReadableBackchannel(int, int, String)} is called, the + * {@link WritableBackchannel#isEstablished()} will return true. + */ + @SuppressWarnings("unchecked") + public WritableBackchannel getWritableBackchannel( + int subtaskId, int attemptNumber, String transactionalIdPrefix) { + return (WritableBackchannel) + getBackchannel( + subtaskId, + attemptNumber, + transactionalIdPrefix, + BackchannelImpl::createWritableBackchannel); + } + + private R getBackchannel( + int subtaskId, + int attemptNumber, + String transactionalIdPrefix, + Function, R> subchannelCreator) { + Tuple3 id = + new Tuple3<>(subtaskId, attemptNumber, transactionalIdPrefix); + BackchannelImpl backchannel = + backchannels.computeIfAbsent(id, k -> new BackchannelImpl<>(() -> unregister(id))); + try { + return subchannelCreator.apply(backchannel); + } catch (Exception e) { + throw new IllegalStateException( + String.format( + "Found duplicate transactionalIdPrefix for multiple Kafka sinks: %s. Transactional id prefixes need to be unique. You may experience memory leaks without fixing this.", + transactionalIdPrefix), + e); + } + } + + private void unregister(Tuple3 id) { + backchannels.remove(id); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImpl.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImpl.java new file mode 100644 index 000000000..a6996000c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImpl.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.annotation.Internal; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; + +import java.util.Deque; +import java.util.concurrent.ConcurrentLinkedDeque; + +/** + * A backchannel for communication between the Kafka committer -> writer. It's used to recycle + * producer and signal that certain transactions have been committed on recovery. + * + *

      A backchannel provides two views: a readable view for the committer to read messages, and a + * writable view for the writer to send messages. Both views have a separate lifecycle. The + * lifecycle of this backchannel is bound to the joint lifecycle of the readable and writable views. + */ +@Internal +@ThreadSafe +public final class BackchannelImpl { + /** + * The action to be executed when the backchannel is closed. The channel is classed if both + * readable and writable channels are closed. + */ + private final Runnable closeAction; + /** + * The messages to be sent from the writer to the committer. It's a thread safe deque in case + * committer and writer are not chained. + */ + private final Deque messages = new ConcurrentLinkedDeque<>(); + + /** The readable backchannel. */ + private volatile ReadableBackchannel readableBackchannel; + /** The writable backchannel. */ + private volatile WritableBackchannel writableBackchannel; + + BackchannelImpl(Runnable closeAction) { + this.closeAction = closeAction; + } + + /** + * True iff the backchannel is established, i.e. both readable and writable channels are + * created. + */ + private boolean isEstablished() { + return readableBackchannel != null && writableBackchannel != null; + } + + /** + * Closes the readable channel. If the writable channel is also closed, the backchannel is + * closed. + */ + private void closeReadableChannel() { + if (readableBackchannel == null) { + throw new IllegalStateException("Readable backchannel does not exist."); + } + readableBackchannel = null; + checkClosed(); + } + + /** Checks if the backchannel is considered closed. If so, executes the close action. */ + private void checkClosed() { + if (readableBackchannel == null && writableBackchannel == null) { + closeAction.run(); + } + } + + ReadableBackchannel createReadableBackchannel() { + if (readableBackchannel != null) { + throw new IllegalStateException("Readable backchannel already exists."); + } + return readableBackchannel = new Readable(); + } + + WritableBackchannel createWritableBackchannel() { + if (writableBackchannel != null) { + throw new IllegalStateException("Writable backchannel already exists."); + } + return writableBackchannel = new Writable(); + } + + private void closeWritableChannel() { + if (writableBackchannel == null) { + throw new IllegalStateException("Writable backchannel does not exist."); + } + writableBackchannel = null; + checkClosed(); + } + + private class Writable implements WritableBackchannel { + @Override + public void send(T message) { + messages.add(message); + } + + public boolean isEstablished() { + return BackchannelImpl.this.isEstablished(); + } + + @Override + public void close() { + closeWritableChannel(); + } + } + + private class Readable implements ReadableBackchannel { + @Nullable + @Override + public T poll() { + return messages.poll(); + } + + @Override + public boolean isEstablished() { + return BackchannelImpl.this.isEstablished(); + } + + @Override + public void close() { + closeReadableChannel(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ReadableBackchannel.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ReadableBackchannel.java new file mode 100644 index 000000000..0f2cbf17c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ReadableBackchannel.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import javax.annotation.Nullable; + +/** + * The readable portion of a backchannel for communication between the commiter -> writer. It's used + * to signal that certain transactions have been committed and respective producers are good to be + * reused. + */ +public interface ReadableBackchannel extends Backchannel { + /** + * Poll the next message from the backchannel. This method is non-blocking and returns {@code + * null} if there is no message available. + */ + @Nullable + T poll(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/WritableBackchannel.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/WritableBackchannel.java new file mode 100644 index 000000000..97a0b2c8a --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/WritableBackchannel.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +/** + * The writable portion of a {@link Backchannel} for communication between the commiter -> writer. + * It's used to signal that certain transactions have been committed and respective producers are + * good to be reused. + * + *

      Messages can be sent before the backchannel is established. They will be consumed once the + * backchannel is established. + */ +public interface WritableBackchannel extends Backchannel { + /** Send a message to the backchannel. */ + void send(T message); +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImplTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImplTest.java new file mode 100644 index 000000000..1fd64935d --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImplTest.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.util.function.RunnableWithException; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ForkJoinTask; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; + +class BackchannelImplTest { + + private static final String PREFIX = "PREFIX"; + private static final int ATTEMPT = 0; + private static final BackchannelFactory FACTORY = BackchannelFactory.getInstance(); + + @Test + public void testBasicSend() { + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX); + ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + String message = "Test message"; + + writable.send(message); + + assertThat(readable.poll()).isEqualTo(message); + } + } + + @Test + public void testSendBeforeEstablish() { + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(writable.isEstablished()).isFalse(); + + String message = "Test message"; + + writable.send(message); + + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.isEstablished()).isTrue(); + assertThat(readable.poll()).isEqualTo(message); + } + } + } + + @Test + public void testPollBeforeEstablish() { + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.isEstablished()).isFalse(); + assertThat(readable.poll()).isNull(); + } + } + + @Test + public void testBasicSendTwoThreads() throws InterruptedException, BrokenBarrierException { + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + CyclicBarrier beforeSend = new CyclicBarrier(2); + CyclicBarrier afterSend = new CyclicBarrier(2); + + String message = "Test message"; + ForkJoinTask task = + runInParallel( + () -> { + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + beforeSend.await(); + afterSend.await(); + assertThat(readable.poll()).isEqualTo(message); + } + }); + + beforeSend.await(); + writable.send(message); + afterSend.await(); + task.join(); + } + } + + @Test + public void testSendBeforeEstablishTwoThreads() + throws BrokenBarrierException, InterruptedException { + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + CyclicBarrier beforeEstablish = new CyclicBarrier(2); + + String message = "Test message"; + ForkJoinTask task = + runInParallel( + () -> { + beforeEstablish.await(); + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.poll()).isEqualTo(message); + } + }); + + writable.send(message); + beforeEstablish.await(); + task.join(); + } + } + + @Test + public void testPollBeforeEstablishTwoThreads() + throws BrokenBarrierException, InterruptedException { + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + CyclicBarrier beforeEstablish = new CyclicBarrier(2); + CyclicBarrier afterEstablish = new CyclicBarrier(2); + CyclicBarrier afterSend = new CyclicBarrier(2); + + String message = "Test message"; + ForkJoinTask task = + runInParallel( + () -> { + beforeEstablish.await(); + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + afterEstablish.await(); + writable.send(message); + afterSend.await(); + } + }); + + try { + assertThat(readable.isEstablished()).isFalse(); + assertThat(readable.poll()).isNull(); + beforeEstablish.await(); + afterEstablish.await(); + assertThat(readable.isEstablished()).isTrue(); + afterSend.await(); + assertThat(readable.poll()).isEqualTo(message); + } finally { + // make sure to join first before exiting the test or else cleanup did not properly + // happen + task.join(); + // writable channel cleaned up + assertThat(readable.isEstablished()).isFalse(); + } + } + } + + @Test + void testDuplicatePrefix() { + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + assertThatCode(() -> FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) + .hasMessageContaining("duplicate"); + } + } + + @Test + void testPrefixReuse() { + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.isEstablished()).isFalse(); + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.isEstablished()).isTrue(); + } + assertThat(readable.isEstablished()).isFalse(); + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.isEstablished()).isTrue(); + } + assertThat(readable.isEstablished()).isFalse(); + } + } + + private static ForkJoinTask runInParallel(RunnableWithException r) { + // convert to callable to allow exceptions + return ForkJoinPool.commonPool() + .submit( + () -> { + r.run(); + return true; + }); + } +} From 580d3ed632a70902ed219faae09f7b9a5f1a335e Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Mon, 10 Feb 2025 16:37:47 +0100 Subject: [PATCH 318/322] [FLINK-37282] Add ProducerPool Add first class producer pool that self-manages all resources and allows to recycle producers by transactional ids. --- .../86dfd459-67a9-4b26-9b5c-0b0bbf22681a | 6 + .../c0d94764-76a0-4c50-b617-70b1754c4612 | 1 + .../sink/internal/CheckpointTransaction.java | 78 ++++++ .../internal/FlinkKafkaInternalProducer.java | 57 ++-- .../kafka/sink/internal/ProducerPool.java | 43 +++ .../kafka/sink/internal/ProducerPoolImpl.java | 246 ++++++++++++++++++ .../sink/internal/ProducerPoolImplITCase.java | 113 ++++++++ 7 files changed, 525 insertions(+), 19 deletions(-) create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/CheckpointTransaction.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPool.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImplITCase.java diff --git a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a index 0feda7c9a..8d8514add 100644 --- a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a +++ b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a @@ -26,6 +26,12 @@ org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only o * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.internal.ProducerPoolImplITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.source.KafkaSourceITCase does not satisfy: only one of the following predicates match:\ diff --git a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 index 8dacea360..edf50b148 100644 --- a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 +++ b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 @@ -50,6 +50,7 @@ Method is annotated with in (KafkaSink.java:0) Method calls method in (KafkaSinkBuilder.java:152) Method is annotated with in (KafkaWriter.java:0) +Method is annotated with in (ProducerPoolImpl.java:0) Method is annotated with in (KafkaSource.java:0) Method is annotated with in (KafkaSource.java:0) Method is annotated with in (KafkaSource.java:0) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/CheckpointTransaction.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/CheckpointTransaction.java new file mode 100644 index 000000000..f4fc7080d --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/CheckpointTransaction.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.annotation.Internal; + +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An immutable class that represents a transactional id and a checkpoint id. It's used inside the + * {@link ProducerPoolImpl} to keep track of the transactions that are currently in-flight. The + * checkpoint id is used to subsume committed transactions wrt to recycling producers. + */ +@Internal +public class CheckpointTransaction { + private final String transactionalId; + private final long checkpointId; + + public CheckpointTransaction(String transactionalId, long checkpointId) { + this.transactionalId = checkNotNull(transactionalId, "transactionalId must not be null"); + this.checkpointId = checkpointId; + } + + public long getCheckpointId() { + return checkpointId; + } + + public String getTransactionalId() { + return transactionalId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CheckpointTransaction that = (CheckpointTransaction) o; + return checkpointId == that.checkpointId + && Objects.equals(transactionalId, that.transactionalId); + } + + @Override + public int hashCode() { + return Objects.hash(transactionalId, checkpointId); + } + + @Override + public String toString() { + return "CheckpointTransaction{" + + "transactionalId='" + + transactionalId + + '\'' + + ", checkpointId=" + + checkpointId + + '}'; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java index 4edf6651c..827c0bc61 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java @@ -18,8 +18,6 @@ package org.apache.flink.connector.kafka.sink.internal; -import org.apache.flink.annotation.Internal; - import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -46,9 +44,7 @@ /** * A {@link KafkaProducer} that exposes private fields to allow resume producing from a given state. */ -@Internal public class FlinkKafkaInternalProducer extends KafkaProducer { - private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaInternalProducer.class); private static final String TRANSACTION_MANAGER_FIELD_NAME = "transactionManager"; private static final String TRANSACTION_MANAGER_STATE_ENUM = @@ -173,28 +169,51 @@ public void initTransactionId(String transactionalId) { } } + /** + * Sets the transaction manager state to uninitialized. + * + *

      Can only be called if the producer is in a transaction. Its main purpose is to resolve the + * split brain scenario between writer and committer. + */ + public void transactionCompletedExternally() { + checkState(inTransaction, "Not in transactional state"); + this.inTransaction = false; + this.hasRecordsInTransaction = false; + Object transactionManager = getTransactionManager(); + synchronized (transactionManager) { + setField(transactionManager, "transactionalId", transactionalId); + setField( + transactionManager, + "currentState", + getTransactionManagerState("UNINITIALIZED")); + } + } + + /** + * Sets the transactional id and sets the transaction manager state to uninitialized. + * + *

      Can only be called if the producer is not in a transaction. + */ public void setTransactionId(String transactionalId) { - if (!transactionalId.equals(this.transactionalId)) { - checkState( - !inTransaction, - String.format("Another transaction %s is still open.", transactionalId)); - LOG.debug("Change transaction id from {} to {}", this.transactionalId, transactionalId); - Object transactionManager = getTransactionManager(); - synchronized (transactionManager) { - setField(transactionManager, "transactionalId", transactionalId); - setField( - transactionManager, - "currentState", - getTransactionManagerState("UNINITIALIZED")); - this.transactionalId = transactionalId; - } + checkState( + !inTransaction, + String.format("Another transaction %s is still open.", transactionalId)); + LOG.debug("Change transaction id from {} to {}", this.transactionalId, transactionalId); + this.transactionalId = transactionalId; + Object transactionManager = getTransactionManager(); + synchronized (transactionManager) { + setField(transactionManager, "transactionalId", transactionalId); + setField( + transactionManager, + "currentState", + getTransactionManagerState("UNINITIALIZED")); } } /** * Besides committing {@link org.apache.kafka.clients.producer.KafkaProducer#commitTransaction} * is also adding new partitions to the transaction. flushNewPartitions method is moving this - * logic to pre-commit/flush, to make resumeTransaction simpler. Otherwise resumeTransaction + * logic to pre-commit/flush, to make resumeTransaction simpler. Otherwise, resumeTransaction * would require to restore state of the not yet added/"in-flight" partitions. */ private void flushNewPartitions() { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPool.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPool.java new file mode 100644 index 000000000..bece0ddc7 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPool.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +/** A pool of producers that can be recycled. */ +public interface ProducerPool extends AutoCloseable { + /** + * Notify the pool that a transaction has finished. The producer with the given transactional id + * can be recycled. + */ + void recycleByTransactionId(String transactionalId); + + /** + * Get a producer for the given transactional id and checkpoint id. The producer is not recycled + * until it is passed to the committer, the committer commits the transaction, and {@link + * #recycleByTransactionId(String)} is called. Alternatively, the producer can be recycled by + * {@link #recycle(FlinkKafkaInternalProducer)}. + */ + FlinkKafkaInternalProducer getTransactionalProducer( + String transactionalId, long checkpointId); + + /** + * Explicitly recycle a producer. This is useful when the producer has not been passed to the + * committer. + */ + void recycle(FlinkKafkaInternalProducer producer); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java new file mode 100644 index 000000000..f38ef51e7 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.annotation.VisibleForTesting; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; + +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Comparator; +import java.util.Deque; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Properties; +import java.util.TreeMap; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import static org.apache.flink.util.IOUtils.closeAll; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Manages a pool of {@link FlinkKafkaInternalProducer} instances for reuse in the {@code + * KafkaWriter} and keeps track of the used transactional ids. + * + *

      Reusing the producers is important for performance reasons. The producer initialization + * includes a few requests to the broker (e.g., ApiVersion), which can be avoided with reuse. + * + *

      Tracking the transactional ids in use can be a tricky because the {@code KafkaCommitter} is + * ultimately finishing the transactions. There are two major cases: + * + *

        + *
      • The committer is chained to the writer (common case): The {@code KafkaCommittable} contains + * the producer (in-memory transfer) and the producer is only returned to the producer pool + * upon completion by the committer. Thus, none of the producers in the pool have active + * transactions. + *
      • The committer is not chained: The {@code KafkaCommittableSerializer} will return the + * producer to this pool, but it still has an ongoing transaction. The producer will be + * "cloned" in the committer by using producer id and epoch. In this case, we rely on {@link + * org.apache.kafka.common.errors.ProducerFencedException} to test later if a producer in the + * pool is still in the transaction or not. + *
      + * + *

      This pool is not thread-safe and is only intended to be accessed from the writer, which owns + * it. + */ +@NotThreadSafe +public class ProducerPoolImpl implements ProducerPool { + private static final Logger LOG = LoggerFactory.getLogger(ProducerPoolImpl.class); + + /** + * The configuration for the Kafka producer. This is used to create new producers when the pool + * is empty. + */ + private final Properties kafkaProducerConfig; + /** Callback to allow the writer to init metrics. */ + private final Consumer> producerInit; + /** + * The pool of producers that are available for reuse. This pool is used to avoid creating new + * producers for every transaction. + */ + private final Deque> producerPool = + new ArrayDeque<>(); + /** + * The map of ongoing transactions (id -> producer/CheckpointTransaction). This is used to keep + * track of the transactions that are ongoing and the respective producers are not in the pool. + */ + private final Map producerByTransactionalId = new TreeMap<>(); + /** + * A secondary tracking structure to quickly find transactions coming from an earlier + * checkpoints. + */ + private final NavigableMap transactionalIdsByCheckpoint = + new TreeMap<>(Comparator.comparing(CheckpointTransaction::getCheckpointId)); + + /** Creates a new {@link ProducerPoolImpl}. */ + public ProducerPoolImpl( + Properties kafkaProducerConfig, + Consumer> producerInit) { + this.kafkaProducerConfig = + checkNotNull(kafkaProducerConfig, "kafkaProducerConfig must not be null"); + this.producerInit = checkNotNull(producerInit, "producerInit must not be null"); + } + + @Override + public void recycleByTransactionId(String transactionalId) { + ProducerEntry producerEntry = producerByTransactionalId.remove(transactionalId); + LOG.debug("Transaction {} finished, producer {}", transactionalId, producerEntry); + if (producerEntry == null) { + // during recovery, the committer may finish transactions that are not yet ongoing from + // the writer's perspective + return; + } + + transactionalIdsByCheckpoint.remove(producerEntry.getCheckpointedTransaction()); + recycleProducer(producerEntry.getProducer()); + + // In rare cases (only for non-chained committer), some transactions may not be detected to + // be finished. + // For example, a transaction may be committed at the same time the writer state is + // snapshot. The writer contains the transaction as ongoing but the committer state will + // later not contain it. + // In these cases, we make use of the fact that committables are processed in order of the + // checkpoint id. + // That means a transaction state with checkpoint id C implies that all C' < C are finished. + NavigableMap earlierTransactions = + transactionalIdsByCheckpoint.headMap( + producerEntry.getCheckpointedTransaction(), false); + if (!earlierTransactions.isEmpty()) { + for (String id : earlierTransactions.values()) { + ProducerEntry entry = producerByTransactionalId.remove(id); + recycleProducer(entry.getProducer()); + } + earlierTransactions.clear(); + } + } + + @Override + public void recycle(FlinkKafkaInternalProducer producer) { + recycleProducer(producer); + ProducerEntry producerEntry = + producerByTransactionalId.remove(producer.getTransactionalId()); + transactionalIdsByCheckpoint.remove(producerEntry.getCheckpointedTransaction()); + } + + private void recycleProducer(@Nullable FlinkKafkaInternalProducer producer) { + // In case of recovery, we don't create a producer for the ongoing transactions. + // The producer is just initialized on committer side. + if (producer == null) { + return; + } + // For non-chained committer, we have a split brain scenario: + // Both the writer and the committer have a producer representing the same transaction. + // The committer producer has finished the transaction while the writer producer is still in + // transaction. In this case, we forcibly complete the transaction, such that we can + // initialize it. + if (producer.isInTransaction()) { + producer.transactionCompletedExternally(); + } + producerPool.add(producer); + + LOG.debug("Recycling {}, new pool size {}", producer, producerPool.size()); + } + + @Override + public FlinkKafkaInternalProducer getTransactionalProducer( + String transactionalId, long checkpointId) { + FlinkKafkaInternalProducer producer = producerPool.poll(); + if (producer == null) { + producer = new FlinkKafkaInternalProducer<>(kafkaProducerConfig, transactionalId); + producerInit.accept(producer); + } else if (transactionalId != null) { + producer.setTransactionId(transactionalId); + } + if (transactionalId != null) { + // first keep track of the transaction and producer because initTransaction may be + // interrupted + CheckpointTransaction checkpointedTransaction = + new CheckpointTransaction(transactionalId, checkpointId); + ProducerEntry existing = + producerByTransactionalId.put( + transactionalId, new ProducerEntry(producer, checkpointedTransaction)); + transactionalIdsByCheckpoint.put(checkpointedTransaction, transactionalId); + checkState( + existing == null, + "Transaction %s already ongoing existing producer %s; new producer %s", + transactionalId, + existing, + producer); + producer.initTransactions(); + } + LOG.debug("getProducer {}, new pool size {}", producer, producerPool.size()); + return producer; + } + + @VisibleForTesting + public Collection> getProducers() { + return producerPool; + } + + @Override + public void close() throws Exception { + LOG.debug( + "Closing used producers {} and free producers {}", + producerByTransactionalId, + producerPool); + closeAll( + () -> closeAll(producerPool), + () -> + closeAll( + producerByTransactionalId.values().stream() + .map(ProducerEntry::getProducer) + .collect(Collectors.toList())), + producerPool::clear, + producerByTransactionalId::clear); + } + + private static class ProducerEntry { + private final FlinkKafkaInternalProducer producer; + private final CheckpointTransaction checkpointedTransaction; + + private ProducerEntry( + FlinkKafkaInternalProducer producer, + CheckpointTransaction checkpointedTransaction) { + this.producer = checkNotNull(producer, "producer must not be null"); + this.checkpointedTransaction = + checkNotNull( + checkpointedTransaction, "checkpointedTransaction must not be null"); + } + + public CheckpointTransaction getCheckpointedTransaction() { + return checkpointedTransaction; + } + + public FlinkKafkaInternalProducer getProducer() { + return producer; + } + + @Override + public String toString() { + return producer.toString(); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImplITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImplITCase.java new file mode 100644 index 000000000..a831b8afe --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImplITCase.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +import java.util.Properties; +import java.util.function.Consumer; + +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; +import static org.assertj.core.api.Assertions.assertThat; + +@Testcontainers +class ProducerPoolImplITCase { + + public static final Consumer> INIT = p -> {}; + public static final String TRANSACTIONAL_ID = "test-transactional-id"; + + @Container + public static final KafkaContainer KAFKA_CONTAINER = + createKafkaContainer(ProducerPoolImplITCase.class).withEmbeddedZookeeper(); + + @AfterEach + void checkLeak() { + checkProducerLeak(); + } + + @Test + void testGetTransactionalProducer() throws Exception { + try (ProducerPoolImpl producerPool = new ProducerPoolImpl(getProducerConfig(), INIT)) { + + FlinkKafkaInternalProducer producer = + producerPool.getTransactionalProducer(TRANSACTIONAL_ID, 1L); + assertThat(producer).isNotNull(); + assertThat(producer.getTransactionalId()).isEqualTo(TRANSACTIONAL_ID); + assertThat(producer.isInTransaction()).isFalse(); + // everything prepared to being the transaction + producer.beginTransaction(); + // no explicit closing of producer - pool should also clean up the producer + } + } + + /** Tests direct recycling as used during abortion of transactions. */ + @Test + void testRecycleProducer() throws Exception { + try (ProducerPoolImpl producerPool = new ProducerPoolImpl(getProducerConfig(), INIT)) { + FlinkKafkaInternalProducer producer = + producerPool.getTransactionalProducer(TRANSACTIONAL_ID, 1L); + + assertThat(producerPool.getProducers()).isEmpty(); + producerPool.recycle(producer); + assertThat(producerPool.getProducers()).contains(producer); + + FlinkKafkaInternalProducer newProducer = + producerPool.getTransactionalProducer(TRANSACTIONAL_ID, 1L); + assertThat(newProducer).isSameAs(producer); + } + } + + /** Tests indirect recycling triggered through the backchannel. */ + @Test + void testRecycleByTransactionId() throws Exception { + try (ProducerPoolImpl producerPool = new ProducerPoolImpl(getProducerConfig(), INIT)) { + FlinkKafkaInternalProducer producer = + producerPool.getTransactionalProducer(TRANSACTIONAL_ID, 1L); + + assertThat(producerPool.getProducers()).isEmpty(); + producer.beginTransaction(); + producerPool.recycleByTransactionId(TRANSACTIONAL_ID); + assertThat(producerPool.getProducers()).contains(producer); + // forcefully reset transaction state for split brain scenarios + assertThat(producer.isInTransaction()).isFalse(); + + FlinkKafkaInternalProducer newProducer = + producerPool.getTransactionalProducer(TRANSACTIONAL_ID, 1L); + assertThat(newProducer).isSameAs(producer); + } + } + + private static Properties getProducerConfig() { + Properties kafkaProducerConfig = new Properties(); + kafkaProducerConfig.put( + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, KAFKA_CONTAINER.getBootstrapServers()); + kafkaProducerConfig.put( + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + kafkaProducerConfig.put( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + return kafkaProducerConfig; + } +} From d3c35462a0458ee1b94642134346388d994d5adc Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 18 Feb 2025 15:06:23 +0100 Subject: [PATCH 319/322] [FLINK-37282] Force colocation of kafka writer and kafka committer. --- .../c0d94764-76a0-4c50-b617-70b1754c4612 | 7 ++ .../d853eb69-8c04-4246-9a5e-4f5911286b1d | 2 + .../flink/connector/kafka/sink/KafkaSink.java | 24 +++++- .../connector/kafka/sink/KafkaSinkTest.java | 76 +++++++++++++++---- 4 files changed, 95 insertions(+), 14 deletions(-) diff --git a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 index edf50b148..3e3fa2536 100644 --- a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 +++ b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 @@ -1,3 +1,4 @@ +Class implements interface in (KafkaSink.java:0) Class is annotated with in (KafkaSourceEnumerator.java:0) Class is annotated with in (KafkaSourceEnumerator.java:0) Class extends class in (FlinkKafkaProducer.java:0) @@ -47,6 +48,12 @@ Method calls method in (DynamicKafkaSourceReader.java:381) Method calls method in (DynamicKafkaSourceReader.java:496) Method is annotated with in (ExactlyOnceKafkaWriter.java:0) +Method calls method in (KafkaSink.java:164) +Method calls method in (KafkaSink.java:167) +Method calls method in (KafkaSink.java:163) +Method calls method in (KafkaSink.java:166) +Method checks instanceof in (KafkaSink.java:163) +Method has generic parameter type >> with type argument depending on in (KafkaSink.java:0) Method is annotated with in (KafkaSink.java:0) Method calls method in (KafkaSinkBuilder.java:152) Method is annotated with in (KafkaWriter.java:0) diff --git a/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d b/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d index fddc6df33..c64b386c2 100644 --- a/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d +++ b/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d @@ -1,4 +1,6 @@ org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema.open(org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext): Argument leaf type org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema.serialize(java.lang.Object, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext, java.lang.Long): Argument leaf type org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated +org.apache.flink.connector.kafka.sink.KafkaSink.addPostCommitTopology(org.apache.flink.streaming.api.datastream.DataStream): Argument leaf type org.apache.flink.connector.kafka.sink.KafkaCommittable does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated +org.apache.flink.connector.kafka.sink.KafkaSink.addPostCommitTopology(org.apache.flink.streaming.api.datastream.DataStream): Argument leaf type org.apache.flink.streaming.api.connector.sink2.CommittableMessage does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer.getPartitionOffsets(java.util.Collection, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer$PartitionOffsetsRetriever): Argument leaf type org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer$PartitionOffsetsRetriever does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.invoke(org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$KafkaTransactionState, java.lang.Object, org.apache.flink.streaming.api.functions.sink.SinkFunction$Context): Argument leaf type org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$KafkaTransactionState does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java index 1fb98e6c0..93b5ca93f 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacet; import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider; @@ -28,6 +29,10 @@ import org.apache.flink.connector.kafka.lineage.TypeDatasetFacet; import org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.lineage.LineageVertex; import org.apache.flink.streaming.api.lineage.LineageVertexProvider; @@ -66,7 +71,8 @@ @PublicEvolving public class KafkaSink implements LineageVertexProvider, - TwoPhaseCommittingStatefulSink { + TwoPhaseCommittingStatefulSink, + SupportsPostCommitTopology { private static final Logger LOG = LoggerFactory.getLogger(KafkaSink.class); private final DeliveryGuarantee deliveryGuarantee; @@ -147,6 +153,22 @@ public SimpleVersionedSerializer getWriterStateSerializer() { return new KafkaWriterStateSerializer(); } + @Override + public void addPostCommitTopology(DataStream> committer) { + // this is a somewhat hacky way to ensure that the committer and writer are co-located + if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE && transactionalIdPrefix != null) { + Transformation transformation = committer.getTransformation(); + // all sink transformations output CommittableMessage, so we can safely traverse the + // chain; custom colocation key is set before and should be preserved + while (transformation.getOutputType() instanceof CommittableMessageTypeInfo + && transformation.getCoLocationGroupKey() == null) { + // colocate by transactionalIdPrefix, which should be unique + transformation.setCoLocationGroupKey(transactionalIdPrefix); + transformation = transformation.getInputs().get(0); + } + } + } + @VisibleForTesting protected Properties getKafkaProducerConfig() { return kafkaProducerConfig; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java index 1efb6ec7d..84c1e4293 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java @@ -9,6 +9,10 @@ import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider; import org.apache.flink.connector.kafka.lineage.TypeDatasetFacet; import org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.graph.StreamNode; import org.apache.flink.streaming.api.lineage.LineageVertex; import org.apache.kafka.clients.producer.ProducerRecord; @@ -35,10 +39,10 @@ void setup() { @Test public void testGetLineageVertexWhenSerializerNotAnKafkaDatasetFacetProvider() { - KafkaRecordSerializationSchema recordSerializer = + KafkaRecordSerializationSchema recordSerializer = new KafkaRecordSerializationSchemaWithoutKafkaDatasetProvider(); - KafkaSink sink = - new KafkaSink( + KafkaSink sink = + new KafkaSink<>( DeliveryGuarantee.EXACTLY_ONCE, new Properties(), "", recordSerializer); assertThat(sink.getLineageVertex().datasets()).isEmpty(); @@ -46,11 +50,11 @@ public void testGetLineageVertexWhenSerializerNotAnKafkaDatasetFacetProvider() { @Test public void testGetLineageVertexWhenNoKafkaDatasetFacetReturnedFromSerializer() { - KafkaRecordSerializationSchema recordSerializer = + KafkaRecordSerializationSchema recordSerializer = new KafkaRecordSerializationSchemaWithEmptyKafkaDatasetProvider(); - KafkaSink sink = - new KafkaSink( + KafkaSink sink = + new KafkaSink<>( DeliveryGuarantee.EXACTLY_ONCE, new Properties(), "", recordSerializer); assertThat(sink.getLineageVertex().datasets()).isEmpty(); @@ -58,11 +62,11 @@ public void testGetLineageVertexWhenNoKafkaDatasetFacetReturnedFromSerializer() @Test public void testGetLineageVertex() { - KafkaRecordSerializationSchema recordSerializer = + KafkaRecordSerializationSchema recordSerializer = new TestingKafkaRecordSerializationSchema(); - KafkaSink sink = - new KafkaSink( + KafkaSink sink = + new KafkaSink<>( DeliveryGuarantee.EXACTLY_ONCE, kafkaProperties, "", recordSerializer); LineageVertex lineageVertex = sink.getLineageVertex(); @@ -91,8 +95,54 @@ public void testGetLineageVertex() { .hasFieldOrPropertyWithValue("typeInformation", TypeInformation.of(String.class)); } - private static class KafkaRecordSerializationSchemaWithoutKafkaDatasetProvider - implements KafkaRecordSerializationSchema { + @Test + public void testCoLocation() { + String colocationKey = "testCoLocation"; + KafkaSink sink = + new KafkaSink<>( + DeliveryGuarantee.EXACTLY_ONCE, + kafkaProperties, + colocationKey, + new TestingKafkaRecordSerializationSchema()); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.fromData(1).sinkTo(sink); + + StreamGraph streamGraph = env.getStreamGraph(); + assertThat(streamGraph.getStreamNodes()) + .filteredOn(node -> !node.getInEdges().isEmpty()) + .hasSize(2) // writer and committer + .extracting(StreamNode::getCoLocationGroup) + .containsOnly(colocationKey); + } + + @Test + public void testPreserveCustomCoLocation() { + String colocationKey = "testPreserveCustomCoLocation"; + String customColocationKey = "customCoLocation"; + KafkaSink sink = + new KafkaSink<>( + DeliveryGuarantee.EXACTLY_ONCE, + kafkaProperties, + colocationKey, + new TestingKafkaRecordSerializationSchema()); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSink stream = env.fromData(1).sinkTo(sink); + stream.getTransformation().setCoLocationGroupKey(customColocationKey); + + StreamGraph streamGraph = env.getStreamGraph(); + assertThat(streamGraph.getStreamNodes()) + .filteredOn(node -> !node.getInEdges().isEmpty()) + .hasSize(2) // writer and committer + .extracting(StreamNode::getCoLocationGroup) + .containsOnly(customColocationKey); + } + + private static class KafkaRecordSerializationSchemaWithoutKafkaDatasetProvider + implements KafkaRecordSerializationSchema { @Nullable @Override public ProducerRecord serialize( @@ -102,7 +152,7 @@ public ProducerRecord serialize( } private static class KafkaRecordSerializationSchemaWithEmptyKafkaDatasetProvider - implements KafkaRecordSerializationSchema, KafkaDatasetFacetProvider { + implements KafkaRecordSerializationSchema, KafkaDatasetFacetProvider { @Nullable @Override public ProducerRecord serialize( @@ -117,7 +167,7 @@ public Optional getKafkaDatasetFacet() { } private static class TestingKafkaRecordSerializationSchema - implements KafkaRecordSerializationSchema, + implements KafkaRecordSerializationSchema, KafkaDatasetFacetProvider, TypeDatasetFacetProvider { From f4015d17bd204b8decaf8d5337de011107e808e4 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Sat, 8 Feb 2025 00:03:23 +0100 Subject: [PATCH 320/322] [FLINK-37282] Incorporate Backchannel and ProducerPool into EOSWriter --- .../984f05c0-ec82-405e-9bcc-d202dbe7202e | 3 - .../c0d94764-76a0-4c50-b617-70b1754c4612 | 12 +- .../d853eb69-8c04-4246-9a5e-4f5911286b1d | 1 + .../kafka/sink/ExactlyOnceKafkaWriter.java | 140 +++++++++--------- .../kafka/sink/KafkaCommittable.java | 20 +-- .../connector/kafka/sink/KafkaCommitter.java | 62 +++++--- .../flink/connector/kafka/sink/KafkaSink.java | 10 +- .../connector/kafka/sink/Recyclable.java | 49 ------ .../kafka/sink/TransactionAborter.java | 22 +-- .../internal/FlinkKafkaInternalProducer.java | 9 +- .../kafka/sink/internal/ProducerPoolImpl.java | 2 +- .../kafka/table/ReducingUpsertSink.java | 5 +- .../sink/ExactlyOnceKafkaWriterITCase.java | 27 ++-- .../FlinkKafkaInternalProducerITCase.java | 9 +- .../kafka/sink/KafkaCommitterTest.java | 107 ++++++++----- .../kafka/sink/KafkaWriterTestBase.java | 24 ++- 16 files changed, 257 insertions(+), 245 deletions(-) delete mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/Recyclable.java diff --git a/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e b/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e index 48b26ecd6..c27f7c3d4 100644 --- a/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e +++ b/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e @@ -26,8 +26,6 @@ Constructor (org.apache Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:138) Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:173) Constructor (java.lang.String)> calls method in (KafkaWriterState.java:28) -Constructor (java.lang.Object, java.util.function.Consumer)> calls method in (Recyclable.java:31) -Constructor (java.lang.Object, java.util.function.Consumer)> calls method in (Recyclable.java:32) Constructor (int, int, java.util.function.Function, java.util.function.Consumer)> calls method in (TransactionAborter.java:60) Constructor (java.lang.Class)> calls constructor ()> in (YamlFileMetadataService.java:270) Constructor (org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber, org.apache.flink.connector.kafka.source.enumerator.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, java.util.Properties, org.apache.flink.api.connector.source.Boundedness, org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState, org.apache.flink.connector.kafka.dynamic.source.enumerator.StoppableKafkaEnumContextProxy$StoppableKafkaEnumContextProxyFactory)> is annotated with in (DynamicKafkaSourceEnumerator.java:0) @@ -162,7 +160,6 @@ Method calls method Method is annotated with in (KafkaWriter.java:0) Method is annotated with in (KafkaWriter.java:0) Method calls method in (KafkaWriter.java:311) -Method calls method in (Recyclable.java:36) Method is annotated with in (DynamicKafkaSource.java:0) Method calls method in (DynamicKafkaSourceBuilder.java:290) Method calls method in (DynamicKafkaSourceBuilder.java:292) diff --git a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 index 3e3fa2536..ebea539d5 100644 --- a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 +++ b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 @@ -48,11 +48,13 @@ Method calls method in (DynamicKafkaSourceReader.java:381) Method calls method in (DynamicKafkaSourceReader.java:496) Method is annotated with in (ExactlyOnceKafkaWriter.java:0) -Method calls method in (KafkaSink.java:164) -Method calls method in (KafkaSink.java:167) -Method calls method in (KafkaSink.java:163) -Method calls method in (KafkaSink.java:166) -Method checks instanceof in (KafkaSink.java:163) +Method is annotated with in (ExactlyOnceKafkaWriter.java:0) +Method is annotated with in (KafkaCommitter.java:0) +Method calls method in (KafkaSink.java:168) +Method calls method in (KafkaSink.java:171) +Method calls method in (KafkaSink.java:167) +Method calls method in (KafkaSink.java:170) +Method checks instanceof in (KafkaSink.java:167) Method has generic parameter type >> with type argument depending on in (KafkaSink.java:0) Method is annotated with in (KafkaSink.java:0) Method calls method in (KafkaSinkBuilder.java:152) diff --git a/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d b/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d index c64b386c2..09a12f009 100644 --- a/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d +++ b/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d @@ -2,5 +2,6 @@ org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema.open(org.ap org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema.serialize(java.lang.Object, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext, java.lang.Long): Argument leaf type org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.connector.kafka.sink.KafkaSink.addPostCommitTopology(org.apache.flink.streaming.api.datastream.DataStream): Argument leaf type org.apache.flink.connector.kafka.sink.KafkaCommittable does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.connector.kafka.sink.KafkaSink.addPostCommitTopology(org.apache.flink.streaming.api.datastream.DataStream): Argument leaf type org.apache.flink.streaming.api.connector.sink2.CommittableMessage does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated +org.apache.flink.connector.kafka.sink.KafkaSink.createCommitter(org.apache.flink.api.connector.sink2.CommitterInitContext): Returned leaf type org.apache.flink.connector.kafka.sink.KafkaCommittable does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer.getPartitionOffsets(java.util.Collection, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer$PartitionOffsetsRetriever): Argument leaf type org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer$PartitionOffsetsRetriever does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.invoke(org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$KafkaTransactionState, java.lang.Object, org.apache.flink.streaming.api.functions.sink.SinkFunction$Context): Argument leaf type org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$KafkaTransactionState does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java index 9fbf7ce9c..4cf385fca 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java @@ -21,7 +21,11 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.internal.BackchannelFactory; import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.ProducerPool; +import org.apache.flink.connector.kafka.sink.internal.ProducerPoolImpl; +import org.apache.flink.connector.kafka.sink.internal.ReadableBackchannel; import org.apache.flink.connector.kafka.sink.internal.TransactionalIdFactory; import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.util.FlinkRuntimeException; @@ -31,20 +35,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Closeable; import java.io.IOException; -import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Deque; import java.util.List; import java.util.Optional; import java.util.Properties; import static org.apache.flink.util.IOUtils.closeAll; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; /** * Exactly-once Kafka writer that writes records to Kafka in transactions. @@ -56,13 +56,21 @@ class ExactlyOnceKafkaWriter extends KafkaWriter { private final String transactionalIdPrefix; private final KafkaWriterState kafkaWriterState; - // producer pool only used for exactly once - private final Deque> producerPool = - new ArrayDeque<>(); private final Collection recoveredStates; - private long lastCheckpointId; + private final long restoredCheckpointId; - private final Deque producerCloseables = new ArrayDeque<>(); + /** + * The producer pool that manages all transactional producers. It keeps track of the producers + * that have been recycled as well as producers that are currently in use (potentially forwarded + * to committer). + */ + private final ProducerPool producerPool; + /** + * Backchannel used to communicate committed transactions from the committer to this writer. + * Establishing the channel happens during recovery. Thus, it is only safe to poll in checkpoint + * related methods. + */ + private final ReadableBackchannel backchannel; /** * Constructor creating a kafka writer. @@ -103,43 +111,64 @@ class ExactlyOnceKafkaWriter extends KafkaWriter { } this.kafkaWriterState = new KafkaWriterState(transactionalIdPrefix); - this.lastCheckpointId = - sinkInitContext - .getRestoredCheckpointId() - .orElse(CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1); this.recoveredStates = checkNotNull(recoveredStates, "recoveredStates"); initFlinkMetrics(); + restoredCheckpointId = + sinkInitContext + .getRestoredCheckpointId() + .orElse(CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1); + int subtaskId = sinkInitContext.getTaskInfo().getIndexOfThisSubtask(); + this.producerPool = new ProducerPoolImpl(kafkaProducerConfig, this::initKafkaMetrics); + this.backchannel = + BackchannelFactory.getInstance() + .getReadableBackchannel( + subtaskId, + sinkInitContext.getTaskInfo().getAttemptNumber(), + transactionalIdPrefix); } @Override public void initialize() { - abortLingeringTransactions(recoveredStates, lastCheckpointId + 1); - this.currentProducer = getTransactionalProducer(lastCheckpointId + 1); - this.currentProducer.beginTransaction(); + abortLingeringTransactions( + checkNotNull(recoveredStates, "recoveredStates"), restoredCheckpointId + 1); + this.currentProducer = startTransaction(restoredCheckpointId + 1); + } + + private FlinkKafkaInternalProducer startTransaction(long checkpointId) { + FlinkKafkaInternalProducer producer = + producerPool.getTransactionalProducer( + TransactionalIdFactory.buildTransactionalId( + transactionalIdPrefix, + kafkaSinkContext.getParallelInstanceId(), + checkpointId), + checkpointId); + producer.beginTransaction(); + return producer; } @Override public Collection prepareCommit() { // only return a KafkaCommittable if the current transaction has been written some data if (currentProducer.hasRecordsInTransaction()) { - final List committables = - Collections.singletonList( - KafkaCommittable.of(currentProducer, producerPool::add)); - LOG.debug("Committing {} committables.", committables); - return committables; + KafkaCommittable committable = KafkaCommittable.of(currentProducer); + LOG.debug("Prepare {}.", committable); + return Collections.singletonList(committable); } - // otherwise, we commit the empty transaction as is (no-op) and just recycle the producer - currentProducer.commitTransaction(); - producerPool.add(currentProducer); + // otherwise, we recycle the producer (the pool will reset the transaction state) + producerPool.recycle(currentProducer); return Collections.emptyList(); } @Override public List snapshotState(long checkpointId) throws IOException { - currentProducer = getTransactionalProducer(checkpointId + 1); - currentProducer.beginTransaction(); + // recycle committed producers + String finishedTransactionalId; + while ((finishedTransactionalId = backchannel.poll()) != null) { + producerPool.recycleByTransactionId(finishedTransactionalId); + } + currentProducer = startTransaction(checkpointId + 1); return Collections.singletonList(kafkaWriterState); } @@ -148,7 +177,7 @@ public void close() throws Exception { closeAll( this::abortCurrentProducer, () -> closeAll(producerPool), - () -> closeAll(producerCloseables), + backchannel, super::close); } @@ -166,15 +195,23 @@ private void abortCurrentProducer() { } @VisibleForTesting - Collection> getProducerPool() { + ProducerPool getProducerPool() { return producerPool; } + @VisibleForTesting + public String getTransactionalIdPrefix() { + return transactionalIdPrefix; + } + private void abortLingeringTransactions( Collection recoveredStates, long startCheckpointId) { List prefixesToAbort = new ArrayList<>(); prefixesToAbort.add(transactionalIdPrefix); + LOG.info( + "Aborting lingering transactions from previous execution. Recovered states: {}.", + recoveredStates); final Optional lastStateOpt = recoveredStates.stream().findFirst(); if (lastStateOpt.isPresent()) { KafkaWriterState lastState = lastStateOpt.get(); @@ -191,52 +228,9 @@ private void abortLingeringTransactions( new TransactionAborter( kafkaSinkContext.getParallelInstanceId(), kafkaSinkContext.getNumberOfParallelInstances(), - this::getOrCreateTransactionalProducer, - producerPool::add)) { + id -> producerPool.getTransactionalProducer(id, startCheckpointId), + producerPool::recycle)) { transactionAborter.abortLingeringTransactions(prefixesToAbort, startCheckpointId); } } - - /** - * For each checkpoint we create new {@link FlinkKafkaInternalProducer} so that new transactions - * will not clash with transactions created during previous checkpoints ({@code - * producer.initTransactions()} assures that we obtain new producerId and epoch counters). - * - *

      Ensures that all transaction ids in between lastCheckpointId and checkpointId are - * initialized. - */ - private FlinkKafkaInternalProducer getTransactionalProducer(long checkpointId) { - checkState( - checkpointId > lastCheckpointId, - "Expected %s > %s", - checkpointId, - lastCheckpointId); - FlinkKafkaInternalProducer producer = null; - // in case checkpoints have been aborted, Flink would create non-consecutive transaction ids - // this loop ensures that all gaps are filled with initialized (empty) transactions - for (long id = lastCheckpointId + 1; id <= checkpointId; id++) { - String transactionalId = - TransactionalIdFactory.buildTransactionalId( - transactionalIdPrefix, kafkaSinkContext.getParallelInstanceId(), id); - producer = getOrCreateTransactionalProducer(transactionalId); - } - this.lastCheckpointId = checkpointId; - assert producer != null; - LOG.info("Created new transactional producer {}", producer.getTransactionalId()); - return producer; - } - - private FlinkKafkaInternalProducer getOrCreateTransactionalProducer( - String transactionalId) { - FlinkKafkaInternalProducer producer = producerPool.poll(); - if (producer == null) { - producer = new FlinkKafkaInternalProducer<>(kafkaProducerConfig, transactionalId); - producerCloseables.add(producer); - producer.initTransactions(); - initKafkaMetrics(producer); - } else { - producer.initTransactionId(transactionalId); - } - return producer; - } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java index 54ffdb99d..1ce3b6bb3 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java @@ -17,44 +17,43 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.annotation.Internal; import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; import javax.annotation.Nullable; import java.util.Objects; import java.util.Optional; -import java.util.function.Consumer; /** * This class holds the necessary information to construct a new {@link FlinkKafkaInternalProducer} * to commit transactions in {@link KafkaCommitter}. */ +@Internal class KafkaCommittable { private final long producerId; private final short epoch; private final String transactionalId; - @Nullable private Recyclable> producer; + @Nullable private FlinkKafkaInternalProducer producer; public KafkaCommittable( long producerId, short epoch, String transactionalId, - @Nullable Recyclable> producer) { + @Nullable FlinkKafkaInternalProducer producer) { this.producerId = producerId; this.epoch = epoch; this.transactionalId = transactionalId; this.producer = producer; } - public static KafkaCommittable of( - FlinkKafkaInternalProducer producer, - Consumer> recycler) { + public static KafkaCommittable of(FlinkKafkaInternalProducer producer) { return new KafkaCommittable( producer.getProducerId(), producer.getEpoch(), producer.getTransactionalId(), - new Recyclable<>(producer, recycler)); + producer); } public long getProducerId() { @@ -69,7 +68,7 @@ public String getTransactionalId() { return transactionalId; } - public Optional>> getProducer() { + public Optional> getProducer() { return Optional.ofNullable(producer); } @@ -80,8 +79,11 @@ public String toString() { + producerId + ", epoch=" + epoch - + ", transactionalId=" + + ", transactionalId='" + transactionalId + + '\'' + + ", producer=" + + producer + '}'; } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java index 4c4458851..318e8bec6 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java @@ -17,8 +17,13 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.connector.kafka.sink.internal.BackchannelFactory; import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.WritableBackchannel; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.IOUtils; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.errors.InvalidTxnStateException; @@ -49,11 +54,23 @@ class KafkaCommitter implements Committer, Closeable { + "To avoid data loss, the application will restart."; private final Properties kafkaProducerConfig; + private final WritableBackchannel backchannel; + @Nullable private FlinkKafkaInternalProducer committingProducer; - @Nullable private FlinkKafkaInternalProducer recoveryProducer; - - KafkaCommitter(Properties kafkaProducerConfig) { + KafkaCommitter( + Properties kafkaProducerConfig, + String transactionalIdPrefix, + int subtaskId, + int attemptNumber) { this.kafkaProducerConfig = kafkaProducerConfig; + backchannel = + BackchannelFactory.getInstance() + .getWritableBackchannel(subtaskId, attemptNumber, transactionalIdPrefix); + } + + @VisibleForTesting + public WritableBackchannel getBackchannel() { + return backchannel; } @Override @@ -63,17 +80,12 @@ public void commit(Collection> requests) final KafkaCommittable committable = request.getCommittable(); final String transactionalId = committable.getTransactionalId(); LOG.debug("Committing Kafka transaction {}", transactionalId); - Optional>> recyclable = - committable.getProducer(); + Optional> writerProducer = committable.getProducer(); FlinkKafkaInternalProducer producer; try { - producer = - recyclable - .>map(Recyclable::getObject) - .orElseGet(() -> getRecoveryProducer(committable)); + producer = writerProducer.orElseGet(() -> getProducer(committable)); producer.commitTransaction(); - producer.flush(); - recyclable.ifPresent(Recyclable::close); + backchannel.send(committable.getTransactionalId()); } catch (RetriableException e) { LOG.warn( "Encountered retriable exception while committing {}.", transactionalId, e); @@ -92,7 +104,7 @@ public void commit(Collection> requests) ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, kafkaProducerConfig.getProperty(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), e); - recyclable.ifPresent(Recyclable::close); + backchannel.send(committable.getTransactionalId()); request.signalFailedWithKnownReason(e); } catch (InvalidTxnStateException e) { // This exception only occurs when aborting after a commit or vice versa. @@ -102,30 +114,32 @@ public void commit(Collection> requests) + "Most likely the transaction has been aborted for some reason. Please check the Kafka logs for more details.", request, e); - recyclable.ifPresent(Recyclable::close); + backchannel.send(committable.getTransactionalId()); request.signalFailedWithKnownReason(e); } catch (UnknownProducerIdException e) { LOG.error( "Unable to commit transaction ({}) " + UNKNOWN_PRODUCER_ID_ERROR_MESSAGE, request, e); - recyclable.ifPresent(Recyclable::close); + backchannel.send(committable.getTransactionalId()); request.signalFailedWithKnownReason(e); } catch (Exception e) { LOG.error( "Transaction ({}) encountered error and data has been potentially lost.", request, e); - recyclable.ifPresent(Recyclable::close); + // cause failover request.signalFailedWithUnknownReason(e); } } } @Override - public void close() { - if (recoveryProducer != null) { - recoveryProducer.close(); + public void close() throws IOException { + try { + IOUtils.closeAll(backchannel, committingProducer); + } catch (Exception e) { + ExceptionUtils.rethrow(e); } } @@ -133,15 +147,15 @@ public void close() { * Creates a producer that can commit into the same transaction as the upstream producer that * was serialized into {@link KafkaCommittable}. */ - private FlinkKafkaInternalProducer getRecoveryProducer(KafkaCommittable committable) { - if (recoveryProducer == null) { - recoveryProducer = + private FlinkKafkaInternalProducer getProducer(KafkaCommittable committable) { + if (committingProducer == null) { + committingProducer = new FlinkKafkaInternalProducer<>( kafkaProducerConfig, committable.getTransactionalId()); } else { - recoveryProducer.setTransactionId(committable.getTransactionalId()); + committingProducer.setTransactionId(committable.getTransactionalId()); } - recoveryProducer.resumeTransaction(committable.getProducerId(), committable.getEpoch()); - return recoveryProducer; + committingProducer.resumeTransaction(committable.getProducerId(), committable.getEpoch()); + return committingProducer; } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java index 93b5ca93f..ae498ca00 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; import org.apache.flink.api.dag.Transformation; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacet; @@ -101,10 +102,13 @@ public static KafkaSinkBuilder builder() { return new KafkaSinkBuilder<>(); } - @Internal @Override - public Committer createCommitter() throws IOException { - return new KafkaCommitter(kafkaProducerConfig); + public Committer createCommitter(CommitterInitContext context) { + return new KafkaCommitter( + kafkaProducerConfig, + transactionalIdPrefix, + context.getTaskInfo().getIndexOfThisSubtask(), + context.getTaskInfo().getAttemptNumber()); } @Internal diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/Recyclable.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/Recyclable.java deleted file mode 100644 index 012fa992c..000000000 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/Recyclable.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.kafka.sink; - -import java.io.Closeable; -import java.util.function.Consumer; - -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; - -class Recyclable implements Closeable { - private T object; - private final Consumer recycler; - - public Recyclable(T object, Consumer recycler) { - this.object = checkNotNull(object); - this.recycler = checkNotNull(recycler); - } - - public T getObject() { - checkState(!isRecycled(), "Already recycled"); - return object; - } - - boolean isRecycled() { - return object == null; - } - - @Override - public void close() { - recycler.accept(object); - object = null; - } -} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java index 85a139be9..353cfea42 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java @@ -50,18 +50,18 @@ class TransactionAborter implements Closeable { private final int subtaskId; private final int parallelism; private final Function> producerFactory; - private final Consumer> closeAction; + private final Consumer> recycler; @Nullable FlinkKafkaInternalProducer producer = null; public TransactionAborter( int subtaskId, int parallelism, Function> producerFactory, - Consumer> closeAction) { + Consumer> recycler) { this.subtaskId = subtaskId; this.parallelism = parallelism; this.producerFactory = checkNotNull(producerFactory); - this.closeAction = closeAction; + this.recycler = recycler; } void abortLingeringTransactions(List prefixesToAbort, long startCheckpointId) { @@ -103,14 +103,12 @@ private int abortTransactionOfSubtask(String prefix, long startCheckpointId, int // initTransactions fences all old transactions with the same id by bumping the epoch String transactionalId = TransactionalIdFactory.buildTransactionalId(prefix, subtaskId, checkpointId); - if (producer == null) { - producer = producerFactory.apply(transactionalId); - } else { - producer.initTransactionId(transactionalId); - } + producer = producerFactory.apply(transactionalId); producer.flush(); // An epoch of 0 indicates that the id was unused before - if (producer.getEpoch() == 0) { + short epoch = producer.getEpoch(); + recycler.accept(producer); + if (epoch == 0) { // Note that the check works beyond transaction log timeouts and just depends on the // retention of the transaction topic (typically 7d). Any transaction that is not in // the that topic anymore is also not lingering (i.e., it will not block downstream @@ -126,9 +124,5 @@ private int abortTransactionOfSubtask(String prefix, long startCheckpointId, int return numTransactionAborted; } - public void close() { - if (producer != null) { - closeAction.accept(producer); - } - } + public void close() {} } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java index 827c0bc61..a6a443049 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java @@ -162,20 +162,13 @@ public long getProducerId() { return (long) getField(producerIdAndEpoch, "producerId"); } - public void initTransactionId(String transactionalId) { - if (!transactionalId.equals(this.transactionalId)) { - setTransactionId(transactionalId); - initTransactions(); - } - } - /** * Sets the transaction manager state to uninitialized. * *

      Can only be called if the producer is in a transaction. Its main purpose is to resolve the * split brain scenario between writer and committer. */ - public void transactionCompletedExternally() { + public void resetTransactionState() { checkState(inTransaction, "Not in transactional state"); this.inTransaction = false; this.hasRecordsInTransaction = false; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java index f38ef51e7..a3a2eb925 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java @@ -157,7 +157,7 @@ private void recycleProducer(@Nullable FlinkKafkaInternalProducer } @Override - public Committer createCommitter() throws IOException { - return wrappedSink.createCommitter(); + public Committer createCommitter(CommitterInitContext context) throws IOException { + return wrappedSink.createCommitter(context); } @Override diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java index 69448caff..45fc9aa17 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java @@ -19,6 +19,8 @@ import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.ProducerPoolImpl; +import org.apache.flink.connector.kafka.sink.internal.WritableBackchannel; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.util.TestLoggerExtension; @@ -201,7 +203,7 @@ void testLingeringTransaction() throws Exception { final KafkaCommittable committable = committables.stream().findFirst().get(); assertThat(committable.getProducer().isPresent()).isTrue(); - committable.getProducer().get().getObject().commitTransaction(); + committable.getProducer().get().commitTransaction(); List> records = drainAllRecordsFromTopic(topic, getKafkaClientConfiguration(), true); @@ -216,7 +218,7 @@ void testLingeringTransaction() throws Exception { void usePooledProducerForTransactional() throws Exception { try (final ExactlyOnceKafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { - assertThat(writer.getProducerPool()).hasSize(0); + assertThat(getProducers(writer)).hasSize(0); writer.write(1, SINK_WRITER_CONTEXT); writer.flush(false); @@ -226,17 +228,17 @@ void usePooledProducerForTransactional() throws Exception { final KafkaCommittable committable = committables0.stream().findFirst().get(); assertThat(committable.getProducer().isPresent()).isTrue(); - FlinkKafkaInternalProducer firstProducer = - committable.getProducer().get().getObject(); + FlinkKafkaInternalProducer firstProducer = committable.getProducer().get(); assertThat(firstProducer != writer.getCurrentProducer()) .as("Expected different producer") .isTrue(); // recycle first producer, KafkaCommitter would commit it and then return it - assertThat(writer.getProducerPool()).hasSize(0); + assertThat(getProducers(writer)).hasSize(0); firstProducer.commitTransaction(); - committable.getProducer().get().close(); - assertThat(writer.getProducerPool()).hasSize(1); + try (WritableBackchannel backchannel = getBackchannel(writer)) { + backchannel.send(firstProducer.getTransactionalId()); + } writer.write(1, SINK_WRITER_CONTEXT); writer.flush(false); @@ -261,16 +263,16 @@ void usePooledProducerForTransactional() throws Exception { void prepareCommitForEmptyTransaction() throws Exception { try (final ExactlyOnceKafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { - assertThat(writer.getProducerPool()).hasSize(0); + assertThat(getProducers(writer)).hasSize(0); // no data written to current transaction writer.flush(false); Collection emptyCommittables = writer.prepareCommit(); assertThat(emptyCommittables).hasSize(0); - assertThat(writer.getProducerPool()).hasSize(1); + assertThat(getProducers(writer)).hasSize(1); final FlinkKafkaInternalProducer recycledProducer = - Iterables.getFirst(writer.getProducerPool(), null); + Iterables.getFirst(getProducers(writer), null); assertThat(recycledProducer.isInTransaction()).isFalse(); } } @@ -306,4 +308,9 @@ void testAbortOnClose() throws Exception { assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(1); } } + + private static Collection> getProducers( + ExactlyOnceKafkaWriter writer) { + return ((ProducerPoolImpl) writer.getProducerPool()).getProducers(); + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java index f1b61d421..4da573fb4 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java @@ -66,14 +66,15 @@ public void check() { } @Test - void testInitTransactionId() { + void testResetTransactional() { final String topic = "test-init-transactions"; final String transactionIdPrefix = "testInitTransactionId-"; try (FlinkKafkaInternalProducer reuse = new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { int numTransactions = 20; for (int i = 1; i <= numTransactions; i++) { - reuse.initTransactionId(transactionIdPrefix + i); + reuse.setTransactionId(transactionIdPrefix + i); + reuse.initTransactions(); reuse.beginTransaction(); reuse.send(new ProducerRecord<>(topic, "test-value-" + i)); if (i % 2 == 0) { @@ -101,7 +102,7 @@ void testCommitResumedTransaction() { producer.beginTransaction(); producer.send(new ProducerRecord<>(topic, "test-value")); producer.flush(); - snapshottedCommittable = KafkaCommittable.of(producer, ignored -> {}); + snapshottedCommittable = KafkaCommittable.of(producer); } try (FlinkKafkaInternalProducer resumedProducer = @@ -122,7 +123,7 @@ void testCommitResumedEmptyTransactionShouldFail() { new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { producer.initTransactions(); producer.beginTransaction(); - snapshottedCommittable = KafkaCommittable.of(producer, ignored -> {}); + snapshottedCommittable = KafkaCommittable.of(producer); } try (FlinkKafkaInternalProducer resumedProducer = diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java index 610a21e50..cb27bf2c3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java @@ -18,19 +18,21 @@ package org.apache.flink.connector.kafka.sink; import org.apache.flink.api.connector.sink2.mocks.MockCommitRequest; +import org.apache.flink.connector.kafka.sink.internal.BackchannelFactory; import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.ReadableBackchannel; import org.apache.flink.util.TestLoggerExtension; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.serialization.StringSerializer; +import org.assertj.core.api.Condition; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import java.io.IOException; -import java.time.Duration; import java.util.Collections; import java.util.Properties; @@ -43,7 +45,9 @@ class KafkaCommitterTest { private static final int PRODUCER_ID = 0; private static final short EPOCH = 0; - private static final String TRANSACTIONAL_ID = "transactionalId"; + private static final String TRANS_ID = "transactionalId"; + public static final int ATTEMPT = 2; + public static final int SUB_ID = 1; @AfterEach public void check() { @@ -54,70 +58,88 @@ public void check() { @Test public void testRetryCommittableOnRetriableError() throws IOException, InterruptedException { Properties properties = getProperties(); - try (final KafkaCommitter committer = new KafkaCommitter(properties); + try (final KafkaCommitter committer = + new KafkaCommitter(properties, TRANS_ID, SUB_ID, ATTEMPT); FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, TRANSACTIONAL_ID); - Recyclable> recyclable = - new Recyclable<>(producer, p -> {})) { + new FlinkKafkaInternalProducer<>(properties, TRANS_ID); + ReadableBackchannel backchannel = + BackchannelFactory.getInstance() + .getReadableBackchannel(SUB_ID, ATTEMPT, TRANS_ID)) { final MockCommitRequest request = - new MockCommitRequest<>( - new KafkaCommittable(PRODUCER_ID, EPOCH, TRANSACTIONAL_ID, recyclable)); + new MockCommitRequest<>(KafkaCommittable.of(producer)); producer.resumeTransaction(PRODUCER_ID, EPOCH); committer.commit(Collections.singletonList(request)); assertThat(request.getNumberOfRetries()).isEqualTo(1); - assertThat(recyclable.isRecycled()).isFalse(); - // FLINK-25531: force the producer to close immediately, else it would take 1 hour - producer.close(Duration.ZERO); + assertThat(backchannel).doesNotHave(recycledProducer()); } } @Test public void testFailJobOnUnknownFatalError() throws IOException, InterruptedException { Properties properties = getProperties(); - try (final KafkaCommitter committer = new KafkaCommitter(properties); + try (final KafkaCommitter committer = + new KafkaCommitter(properties, TRANS_ID, SUB_ID, ATTEMPT); FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, TRANSACTIONAL_ID); - Recyclable> recyclable = - new Recyclable<>(producer, p -> {})) { + new FlinkKafkaInternalProducer<>(properties, TRANS_ID); + ReadableBackchannel backchannel = + BackchannelFactory.getInstance() + .getReadableBackchannel(SUB_ID, ATTEMPT, TRANS_ID)) { // will fail because transaction not started final MockCommitRequest request = - new MockCommitRequest<>( - new KafkaCommittable(PRODUCER_ID, EPOCH, TRANSACTIONAL_ID, recyclable)); + new MockCommitRequest<>(KafkaCommittable.of(producer)); committer.commit(Collections.singletonList(request)); assertThat(request.getFailedWithUnknownReason()) .isInstanceOf(IllegalStateException.class); assertThat(request.getFailedWithUnknownReason().getMessage()) .contains("Transaction was not started"); - assertThat(recyclable.isRecycled()).isTrue(); + assertThat(backchannel).doesNotHave(recycledProducer()); + } + } + + @Test + public void testFailJobOnKnownFatalError() throws IOException, InterruptedException { + Properties properties = getProperties(); + try (final KafkaCommitter committer = + new KafkaCommitter(properties, TRANS_ID, SUB_ID, ATTEMPT); + FlinkKafkaInternalProducer producer = + new MockProducer(properties, new ProducerFencedException("test")); + ReadableBackchannel backchannel = + BackchannelFactory.getInstance() + .getReadableBackchannel(SUB_ID, ATTEMPT, TRANS_ID)) { + // will fail because transaction not started + final MockCommitRequest request = + new MockCommitRequest<>(KafkaCommittable.of(producer)); + committer.commit(Collections.singletonList(request)); + // do not recycle if a fail-over is triggered; + // else there may be a race-condition in creating a new transaction with the same name + assertThat(backchannel).has(recycledProducer()); } } @Test - public void testKafkaCommitterClosesProducer() throws IOException, InterruptedException { + public void testKafkaCommitterRecyclesProducer() throws IOException, InterruptedException { Properties properties = getProperties(); - FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer(properties, TRANSACTIONAL_ID) { - @Override - public void commitTransaction() throws ProducerFencedException {} - - @Override - public void flush() {} - }; - Recyclable> recyclable = - new Recyclable<>(producer, FlinkKafkaInternalProducer::close); - try (final KafkaCommitter committer = new KafkaCommitter(properties)) { + try (FlinkKafkaInternalProducer producer = new MockProducer(properties, null); + final KafkaCommitter committer = + new KafkaCommitter(properties, TRANS_ID, SUB_ID, ATTEMPT); + ReadableBackchannel backchannel = + BackchannelFactory.getInstance() + .getReadableBackchannel(SUB_ID, ATTEMPT, TRANS_ID)) { final MockCommitRequest request = new MockCommitRequest<>( - new KafkaCommittable(PRODUCER_ID, EPOCH, TRANSACTIONAL_ID, recyclable)); + new KafkaCommittable(PRODUCER_ID, EPOCH, TRANS_ID, producer)); committer.commit(Collections.singletonList(request)); - assertThat(recyclable.isRecycled()).isTrue(); - assertThat(producer.isClosed()).isTrue(); + assertThat(backchannel).has(recycledProducer()); } } + private Condition> recycledProducer() { + return new Condition<>(backchannel -> backchannel.poll() != null, "recycled producer"); + } + Properties getProperties() { Properties properties = new Properties(); properties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "http://localhost:1"); @@ -127,4 +149,23 @@ Properties getProperties() { properties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); return properties; } + + private static class MockProducer extends FlinkKafkaInternalProducer { + private final RuntimeException commitException; + + public MockProducer(Properties properties, RuntimeException commitException) { + super(properties, KafkaCommitterTest.TRANS_ID); + this.commitException = commitException; + } + + @Override + public void commitTransaction() throws ProducerFencedException { + if (commitException != null) { + throw commitException; + } + } + + @Override + public void flush() {} + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java index eaf1ac1dc..4f492966c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java @@ -23,6 +23,8 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.base.sink.writer.TestSinkInitContext; +import org.apache.flink.connector.kafka.sink.internal.BackchannelFactory; +import org.apache.flink.connector.kafka.sink.internal.WritableBackchannel; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.OperatorIOMetricGroup; import org.apache.flink.metrics.groups.OperatorMetricGroup; @@ -77,7 +79,10 @@ public abstract class KafkaWriterTestBase { protected static final String KAFKA_METRIC_WITH_GROUP_NAME = "KafkaProducer.incoming-byte-total"; protected static final SinkWriter.Context SINK_WRITER_CONTEXT = new DummySinkWriterContext(); - public static final String TEST_PREFIX = "test-prefix"; + private static final String TEST_PREFIX = "test-prefix"; + private int writerIndex; + private static final int SUB_ID = 0; + private static final int ATTEMPT = 0; protected static String topic; protected MetricListener metricListener; @@ -103,7 +108,7 @@ public void setUp(TestInfo testInfo) { } @AfterEach - public void check() { + public void teardown() { checkProducerLeak(); } @@ -127,7 +132,7 @@ KafkaSink createSink(Consumer> sinkBuilderAdjuster) KafkaSinkBuilder builder = KafkaSink.builder() .setKafkaProducerConfig(getKafkaClientConfiguration()) - .setTransactionalIdPrefix(TEST_PREFIX) + .setTransactionalIdPrefix(TEST_PREFIX + writerIndex++) .setRecordSerializer(new DummyRecordSerializer()); sinkBuilderAdjuster.accept(builder); return builder.build(); @@ -137,6 +142,11 @@ SinkInitContext createInitContext() { return new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); } + WritableBackchannel getBackchannel(ExactlyOnceKafkaWriter writer) { + return BackchannelFactory.getInstance() + .getWritableBackchannel(SUB_ID, ATTEMPT, writer.getTransactionalIdPrefix()); + } + protected SinkWriterMetricGroup createSinkWriterMetricGroup() { DummyOperatorMetricGroup operatorMetricGroup = new DummyOperatorMetricGroup(metricListener.getMetricGroup()); @@ -182,7 +192,7 @@ public ProcessingTimeService getProcessingTimeService() { @Override public int getSubtaskId() { - return 0; + return SUB_ID; } @Override @@ -192,7 +202,7 @@ public int getNumberOfParallelSubtasks() { @Override public int getAttemptNumber() { - return 0; + return ATTEMPT; } @Override @@ -238,7 +248,7 @@ public ProducerRecord serialize( protected static class DummySinkWriterContext implements SinkWriter.Context { @Override public long currentWatermark() { - return 0; + return ATTEMPT; } @Override @@ -274,7 +284,7 @@ protected static class TriggerTimeService implements ProcessingTimeService { @Override public long getCurrentProcessingTime() { - return 0; + return ATTEMPT; } @Override From a52f15a958e8f1a040e9df899f47552181c0398c Mon Sep 17 00:00:00 2001 From: Zhanghao Chen Date: Sat, 16 Nov 2024 15:17:56 +0800 Subject: [PATCH 321/322] [FLINK-33265] Support source parallelism setting for Kafka connector Close apache/flink-connector-kafka#134 --- .../content.zh/docs/connectors/table/kafka.md | 8 +++ .../docs/connectors/table/upsert-kafka.md | 8 +++ docs/content/docs/connectors/table/kafka.md | 8 +++ .../docs/connectors/table/upsert-kafka.md | 8 +++ .../kafka/table/KafkaConnectorOptions.java | 1 + .../kafka/table/KafkaDynamicSource.java | 22 ++++-- .../kafka/table/KafkaDynamicTableFactory.java | 14 +++- .../table/UpsertKafkaDynamicTableFactory.java | 7 +- .../table/KafkaDynamicTableFactoryTest.java | 67 ++++++++++++++++--- .../UpsertKafkaDynamicTableFactoryTest.java | 47 +++++++++++-- 10 files changed, 170 insertions(+), 20 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/kafka.md b/docs/content.zh/docs/connectors/table/kafka.md index 286a922ef..9df680df8 100644 --- a/docs/content.zh/docs/connectors/table/kafka.md +++ b/docs/content.zh/docs/connectors/table/kafka.md @@ -342,6 +342,14 @@ CREATE TABLE KafkaTable ( Duration Consumer 定期探测动态创建的 Kafka topic 和 partition 的时间间隔。需要显式地设置'scan.topic-partition-discovery.interval'为0才能关闭此功能 + +

      scan.parallelism
      + optional + no + (none) + Integer + 定义 Kafka source 算子的并行度。默认情况下会使用全局默认并行度。 +
      sink.partitioner
      可选 diff --git a/docs/content.zh/docs/connectors/table/upsert-kafka.md b/docs/content.zh/docs/connectors/table/upsert-kafka.md index 3d28ae56b..bacaae52b 100644 --- a/docs/content.zh/docs/connectors/table/upsert-kafka.md +++ b/docs/content.zh/docs/connectors/table/upsert-kafka.md @@ -180,6 +180,14 @@ of all available metadata fields. + +
      scan.parallelism
      + optional + no + (none) + Integer + 定义 upsert-kafka source 算子的并行度。默认情况下会使用全局默认并行度。 +
      sink.parallelism
      可选 diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md index 5756315bc..12b0821c3 100644 --- a/docs/content/docs/connectors/table/kafka.md +++ b/docs/content/docs/connectors/table/kafka.md @@ -369,6 +369,14 @@ Connector Options Duration Interval for consumer to discover dynamically created Kafka topics and partitions periodically. To disable this feature, you need to explicitly set the 'scan.topic-partition-discovery.interval' value to 0. + +
      scan.parallelism
      + optional + no + (none) + Integer + Defines the parallelism of the Kafka source operator. If not set, the global default parallelism is used. +
      sink.partitioner
      optional diff --git a/docs/content/docs/connectors/table/upsert-kafka.md b/docs/content/docs/connectors/table/upsert-kafka.md index e8e38aeda..db75309a2 100644 --- a/docs/content/docs/connectors/table/upsert-kafka.md +++ b/docs/content/docs/connectors/table/upsert-kafka.md @@ -192,6 +192,14 @@ Connector Options format which means that key columns appear in the data type for both the key and value format. + +
      scan.parallelism
      + optional + no + (none) + Integer + Defines the parallelism of the upsert-kafka source operator. If not set, the global default parallelism is used. +
      sink.parallelism
      optional diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java index 11d3c659f..c64ab0bef 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java @@ -106,6 +106,7 @@ public class KafkaConnectorOptions { ValueFieldsStrategy.EXCEPT_KEY)) .build()); + public static final ConfigOption SCAN_PARALLELISM = FactoryUtil.SOURCE_PARALLELISM; public static final ConfigOption SINK_PARALLELISM = FactoryUtil.SINK_PARALLELISM; // -------------------------------------------------------------------------------------------- diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java index c963da762..012068085 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java @@ -71,6 +71,7 @@ import java.util.Locale; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Properties; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -171,6 +172,9 @@ public class KafkaDynamicSource protected final String tableIdentifier; + /** Parallelism of the physical Kafka consumer. * */ + protected final @Nullable Integer parallelism; + public KafkaDynamicSource( DataType physicalDataType, @Nullable DecodingFormat> keyDecodingFormat, @@ -188,7 +192,8 @@ public KafkaDynamicSource( Map specificBoundedOffsets, long boundedTimestampMillis, boolean upsertMode, - String tableIdentifier) { + String tableIdentifier, + @Nullable Integer parallelism) { // Format attributes this.physicalDataType = Preconditions.checkNotNull( @@ -228,6 +233,7 @@ public KafkaDynamicSource( this.boundedTimestampMillis = boundedTimestampMillis; this.upsertMode = upsertMode; this.tableIdentifier = tableIdentifier; + this.parallelism = parallelism; } @Override @@ -267,6 +273,11 @@ public DataStream produceDataStream( public boolean isBounded() { return kafkaSource.getBoundedness() == Boundedness.BOUNDED; } + + @Override + public Optional getParallelism() { + return Optional.ofNullable(parallelism); + } }; } @@ -344,7 +355,8 @@ public DynamicTableSource copy() { specificBoundedOffsets, boundedTimestampMillis, upsertMode, - tableIdentifier); + tableIdentifier, + parallelism); copy.producedDataType = producedDataType; copy.metadataKeys = metadataKeys; copy.watermarkStrategy = watermarkStrategy; @@ -384,7 +396,8 @@ public boolean equals(Object o) { && boundedTimestampMillis == that.boundedTimestampMillis && Objects.equals(upsertMode, that.upsertMode) && Objects.equals(tableIdentifier, that.tableIdentifier) - && Objects.equals(watermarkStrategy, that.watermarkStrategy); + && Objects.equals(watermarkStrategy, that.watermarkStrategy) + && Objects.equals(parallelism, that.parallelism); } @Override @@ -409,7 +422,8 @@ public int hashCode() { boundedTimestampMillis, upsertMode, tableIdentifier, - watermarkStrategy); + watermarkStrategy, + parallelism); } // -------------------------------------------------------------------------------------------- diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java index 8124691a5..34f57ff15 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java @@ -74,6 +74,7 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_PARALLELISM; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_MODE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; @@ -152,6 +153,7 @@ public Set> optionalOptions() { options.add(SCAN_BOUNDED_MODE); options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS); options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS); + options.add(SCAN_PARALLELISM); return options; } @@ -166,6 +168,7 @@ public Set> forwardOptions() { SCAN_STARTUP_SPECIFIC_OFFSETS, SCAN_TOPIC_PARTITION_DISCOVERY, SCAN_STARTUP_TIMESTAMP_MILLIS, + SCAN_PARALLELISM, SINK_PARTITIONER, SINK_PARALLELISM, TRANSACTIONAL_ID_PREFIX) @@ -215,6 +218,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null); + final Integer parallelism = tableOptions.getOptional(SCAN_PARALLELISM).orElse(null); + return createKafkaTableSource( physicalDataType, keyDecodingFormat.orElse(null), @@ -231,7 +236,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { boundedOptions.boundedMode, boundedOptions.specificOffsets, boundedOptions.boundedTimestampMillis, - context.getObjectIdentifier().asSummaryString()); + context.getObjectIdentifier().asSummaryString(), + parallelism); } @Override @@ -396,7 +402,8 @@ protected KafkaDynamicSource createKafkaTableSource( BoundedMode boundedMode, Map specificEndOffsets, long endTimestampMillis, - String tableIdentifier) { + String tableIdentifier, + Integer parallelism) { return new KafkaDynamicSource( physicalDataType, keyDecodingFormat, @@ -414,7 +421,8 @@ protected KafkaDynamicSource createKafkaTableSource( specificEndOffsets, endTimestampMillis, false, - tableIdentifier); + tableIdentifier, + parallelism); } protected KafkaDynamicSink createKafkaTableSink( diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java index 78debc175..275aebd3d 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java @@ -62,6 +62,7 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_PARALLELISM; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_INTERVAL; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_MAX_ROWS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARALLELISM; @@ -115,6 +116,7 @@ public Set> optionalOptions() { options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS); options.add(DELIVERY_GUARANTEE); options.add(TRANSACTIONAL_ID_PREFIX); + options.add(SCAN_PARALLELISM); return options; } @@ -150,6 +152,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { final BoundedOptions boundedOptions = getBoundedOptions(tableOptions); + Integer parallelism = tableOptions.get(SCAN_PARALLELISM); + return new KafkaDynamicSource( context.getPhysicalRowDataType(), keyDecodingFormat, @@ -167,7 +171,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { boundedOptions.specificOffsets, boundedOptions.boundedTimestampMillis, true, - context.getObjectIdentifier().asSummaryString()); + context.getObjectIdentifier().asSummaryString(), + parallelism); } @Override diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java index c1d796d08..10bfe5939 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java @@ -101,6 +101,7 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.AVRO_CONFLUENT; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.DEBEZIUM_AVRO_CONFLUENT; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.PROPERTIES_PREFIX; +import static org.apache.flink.table.factories.FactoryUtil.SOURCE_PARALLELISM; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.assertj.core.api.Assertions.assertThat; @@ -212,7 +213,8 @@ public void testTableSource() { KAFKA_SOURCE_PROPERTIES, StartupMode.SPECIFIC_OFFSETS, specificOffsets, - 0); + 0, + null); assertThat(actualKafkaSource).isEqualTo(expectedKafkaSource); ScanTableSource.ScanRuntimeProvider provider = @@ -220,6 +222,48 @@ public void testTableSource() { assertKafkaSource(provider); } + @Test + public void testTableSourceWithParallelism() { + final Map modifiedOptions = + getModifiedOptions( + getBasicSourceOptions(), + options -> options.put(SOURCE_PARALLELISM.key(), "100")); + final DynamicTableSource actualSource = createTableSource(SCHEMA, modifiedOptions); + final KafkaDynamicSource actualKafkaSource = (KafkaDynamicSource) actualSource; + + final Map specificOffsets = new HashMap<>(); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_0), OFFSET_0); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_1), OFFSET_1); + + final DecodingFormat> valueDecodingFormat = + new DecodingFormatMock(",", true); + + // Test scan source equals + final KafkaDynamicSource expectedKafkaSource = + createExpectedScanSource( + SCHEMA_DATA_TYPE, + null, + valueDecodingFormat, + new int[0], + new int[] {0, 1, 2}, + null, + Collections.singletonList(TOPIC), + null, + KAFKA_SOURCE_PROPERTIES, + StartupMode.SPECIFIC_OFFSETS, + specificOffsets, + 0, + 100); + assertThat(actualKafkaSource).isEqualTo(expectedKafkaSource); + + ScanTableSource.ScanRuntimeProvider provider = + actualKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertThat(provider).isInstanceOf(DataStreamScanProvider.class); + final DataStreamScanProvider sourceProvider = (DataStreamScanProvider) provider; + assertThat(sourceProvider.getParallelism()).isPresent(); + assertThat(sourceProvider.getParallelism()).hasValue(100); + } + @Test public void testTableSourceWithPattern() { final Map modifiedOptions = @@ -254,7 +298,8 @@ public void testTableSourceWithPattern() { KAFKA_SOURCE_PROPERTIES, StartupMode.EARLIEST, specificOffsets, - 0); + 0, + null); final KafkaDynamicSource actualKafkaSource = (KafkaDynamicSource) actualSource; assertThat(actualKafkaSource).isEqualTo(expectedKafkaSource); @@ -295,7 +340,8 @@ public void testTableSourceWithKeyValue() { KAFKA_FINAL_SOURCE_PROPERTIES, StartupMode.GROUP_OFFSETS, Collections.emptyMap(), - 0); + 0, + null); assertThat(actualSource).isEqualTo(expectedKafkaSource); } @@ -346,7 +392,8 @@ public void testTableSourceWithKeyValueAndMetadata() { KAFKA_FINAL_SOURCE_PROPERTIES, StartupMode.GROUP_OFFSETS, Collections.emptyMap(), - 0); + 0, + null); expectedKafkaSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedKafkaSource.metadataKeys = Collections.singletonList("timestamp"); @@ -1188,7 +1235,8 @@ public void testDiscoverPartitionByDefault() { props, StartupMode.SPECIFIC_OFFSETS, specificOffsets, - 0); + 0, + null); assertThat(actualSource).isEqualTo(expectedKafkaSource); ScanTableSource.ScanRuntimeProvider provider = actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); @@ -1226,7 +1274,8 @@ public void testDisableDiscoverPartition() { props, StartupMode.SPECIFIC_OFFSETS, specificOffsets, - 0); + 0, + null); assertThat(actualSource).isEqualTo(expectedKafkaSource); ScanTableSource.ScanRuntimeProvider provider = actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); @@ -1249,7 +1298,8 @@ private static KafkaDynamicSource createExpectedScanSource( Properties properties, StartupMode startupMode, Map specificStartupOffsets, - long startupTimestampMillis) { + long startupTimestampMillis, + @Nullable Integer parallelism) { return new KafkaDynamicSource( physicalDataType, keyDecodingFormat, @@ -1267,7 +1317,8 @@ private static KafkaDynamicSource createExpectedScanSource( Collections.emptyMap(), 0, false, - FactoryMocks.IDENTIFIER.asSummaryString()); + FactoryMocks.IDENTIFIER.asSummaryString(), + parallelism); } private static KafkaDynamicSink createExpectedSink( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java index 1bcd775a1..abde43dca 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java @@ -74,6 +74,8 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import javax.annotation.Nullable; + import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -87,6 +89,7 @@ import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.AVRO_CONFLUENT; +import static org.apache.flink.table.factories.FactoryUtil.SOURCE_PARALLELISM; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.assertj.core.api.Assertions.assertThat; @@ -168,7 +171,8 @@ public void testTableSource() { SOURCE_VALUE_FIELDS, null, Collections.singletonList(SOURCE_TOPIC), - UPSERT_KAFKA_SOURCE_PROPERTIES); + UPSERT_KAFKA_SOURCE_PROPERTIES, + null); assertThat(actualSource).isEqualTo(expectedSource); final KafkaDynamicSource actualUpsertKafkaSource = (KafkaDynamicSource) actualSource; @@ -177,6 +181,38 @@ public void testTableSource() { assertKafkaSource(provider); } + @Test + public void testTableSourceWithParallelism() { + final DataType producedDataType = SOURCE_SCHEMA.toPhysicalRowDataType(); + // Construct table source using options and table source factory + final Map modifiedOptions = + getModifiedOptions( + getFullSourceOptions(), + options -> options.put(SOURCE_PARALLELISM.key(), "100")); + final DynamicTableSource actualSource = createTableSource(SOURCE_SCHEMA, modifiedOptions); + + final KafkaDynamicSource expectedSource = + createExpectedScanSource( + producedDataType, + keyDecodingFormat, + valueDecodingFormat, + SOURCE_KEY_FIELDS, + SOURCE_VALUE_FIELDS, + null, + Collections.singletonList(SOURCE_TOPIC), + UPSERT_KAFKA_SOURCE_PROPERTIES, + 100); + assertThat(actualSource).isEqualTo(expectedSource); + + final KafkaDynamicSource actualUpsertKafkaSource = (KafkaDynamicSource) actualSource; + ScanTableSource.ScanRuntimeProvider provider = + actualUpsertKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertThat(provider).isInstanceOf(DataStreamScanProvider.class); + final DataStreamScanProvider sourceProvider = (DataStreamScanProvider) provider; + assertThat(sourceProvider.getParallelism()).isPresent(); + assertThat(sourceProvider.getParallelism()).hasValue(100); + } + @Test public void testTableSourceWithTopicList() { final Map modifiedOptions = @@ -199,7 +235,8 @@ public void testTableSourceWithTopicList() { SOURCE_VALUE_FIELDS, null, Arrays.asList(SOURCE_TOPIC, SOURCE_TOPIC), - UPSERT_KAFKA_SOURCE_PROPERTIES); + UPSERT_KAFKA_SOURCE_PROPERTIES, + null); assertThat(actualSource).isEqualTo(expectedSource); final KafkaDynamicSource actualUpsertKafkaSource = (KafkaDynamicSource) actualSource; @@ -851,7 +888,8 @@ private KafkaDynamicSource createExpectedScanSource( int[] valueFields, String keyPrefix, List topic, - Properties properties) { + Properties properties, + @Nullable Integer parallelism) { return new KafkaDynamicSource( producedDataType, keyDecodingFormat, @@ -869,7 +907,8 @@ private KafkaDynamicSource createExpectedScanSource( Collections.emptyMap(), 0, true, - FactoryMocks.IDENTIFIER.asSummaryString()); + FactoryMocks.IDENTIFIER.asSummaryString(), + parallelism); } private static KafkaDynamicSink createExpectedSink( From 718621e6b9a6e520b5498856ab06e807f56bbb3c Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Thu, 27 Feb 2025 10:52:24 +0800 Subject: [PATCH 322/322] [FLINK-37380] Change TransactionalIdPrefix to a required option if Exactly once in Kafka Connector --- .../connector/kafka/sink/KafkaSinkBuilder.java | 2 +- .../kafka/sink/KafkaSinkBuilderTest.java | 18 ++++++++++++++++-- .../kafka/table/KafkaChangelogTableITCase.java | 1 + 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java index f0c20cfc0..ffd4177ba 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java @@ -70,7 +70,7 @@ public class KafkaSinkBuilder { private static final int MAXIMUM_PREFIX_BYTES = 64000; private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE; - private String transactionalIdPrefix = "kafka-sink"; + private String transactionalIdPrefix; private final Properties kafkaProducerConfig; private KafkaRecordSerializationSchema recordSerializer; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java index 3e2beb9ad..9e26cd4e1 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.kafka.sink; import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.util.TestLogger; import org.apache.kafka.clients.producer.ProducerConfig; @@ -28,6 +29,7 @@ import java.util.function.Consumer; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link KafkaSinkBuilder}. */ public class KafkaSinkBuilderTest extends TestLogger { @@ -41,7 +43,7 @@ public class KafkaSinkBuilderTest extends TestLogger { }; @Test - public void testPropertyHandling() { + void testPropertyHandling() { validateProducerConfig( getBasicBuilder(), p -> { @@ -77,7 +79,7 @@ public void testPropertyHandling() { } @Test - public void testBootstrapServerSetting() { + void testBootstrapServerSetting() { Properties testConf1 = new Properties(); testConf1.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "testServer"); @@ -86,6 +88,18 @@ public void testBootstrapServerSetting() { p -> assertThat(p).containsKeys(DEFAULT_KEYS)); } + @Test + void testTransactionIdSanityCheck() { + assertThatThrownBy( + () -> + getBasicBuilder() + .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) + .build()) + .isExactlyInstanceOf(IllegalStateException.class) + .hasMessageContaining( + "EXACTLY_ONCE delivery guarantee requires a transactionIdPrefix to be set to provide unique transaction names across multiple KafkaSinks writing to the same Kafka cluster."); + } + private void validateProducerConfig( KafkaSinkBuilder builder, Consumer validator) { validator.accept(builder.build().getKafkaProducerConfig()); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java index 632b74ac9..cd8e58a79 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java @@ -474,6 +474,7 @@ private void writeRecordsToKafka(String topic, List lines) throws Except .setPartitioner(partitioner) .build()) .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) + .setTransactionalIdPrefix("kafka-sink") .build()); env.execute("Write sequence"); }