From 683ba97998afbf675bbb983c960071e74c2a1873 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 00:00:17 +0800 Subject: [PATCH 01/71] first commit --- .../serialization/DeserializationSchema.java | 9 +- .../deserialize/AmazonSqsDeserializer.java | 3 +- .../file/source/reader/JsonReadStrategy.java | 4 +- .../file/source/reader/TextReadStrategy.java | 4 +- .../deserialize/GoogleSheetsDeserializer.java | 3 +- .../http/source/DeserializationCollector.java | 3 +- .../SimpleTextDeserializationSchema.java | 7 +- .../seatunnel/kafka/config/Config.java | 5 + .../kafka/source/ConsumerMetadata.java | 8 +- .../kafka/source/KafkaConsumerThread.java | 14 +- .../seatunnel/kafka/source/KafkaSource.java | 14 +- .../kafka/source/KafkaSourceConfig.java | 102 +++++++---- .../kafka/source/KafkaSourceFactory.java | 3 +- .../kafka/source/KafkaSourceReader.java | 35 ++-- .../kafka/source/KafkaSourceSplit.java | 19 +- .../source/KafkaSourceSplitEnumerator.java | 127 ++++++++------ .../source/format/PulsarCanalDecorator.java | 8 +- .../source/reader/PulsarSourceReader.java | 3 +- .../source/PulsarCanalDecoratorTest.java | 4 +- .../rabbitmq/source/RabbitmqSourceReader.java | 3 +- .../redis/source/RedisSourceReader.java | 8 +- .../rocketmq/source/RocketMqSourceReader.java | 5 +- .../rest/RestHttpGetCommandProcessor.java | 23 ++- .../rest/RestHttpPostCommandProcessor.java | 51 +++++- ...ibleKafkaConnectDeserializationSchema.java | 17 +- ...ibleDebeziumJsonDeserializationSchema.java | 3 +- .../json/JsonDeserializationSchema.java | 16 +- .../canal/CanalJsonDeserializationSchema.java | 163 +++++++++--------- .../DebeziumJsonDeserializationSchema.java | 22 ++- .../ogg/OggJsonDeserializationSchema.java | 3 +- .../json/JsonRowDataSerDeSchemaTest.java | 20 ++- .../json/canal/CanalJsonSerDeSchemaTest.java | 91 +--------- .../debezium/DebeziumJsonSerDeSchemaTest.java | 8 +- .../text/TextDeserializationSchema.java | 13 +- .../format/text/TextFormatSchemaTest.java | 4 +- 35 files changed, 478 insertions(+), 347 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java index d7f7abb1a51..5838ba22b46 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.api.serialization; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import java.io.IOException; @@ -29,13 +30,15 @@ public interface DeserializationSchema extends Serializable { * Deserializes the byte message. * * @param message The message, as a byte array. + * @param tablePath The Registry name. * @return The deserialized message as an SeaTunnel Row (null if the message cannot be * deserialized). */ - T deserialize(byte[] message) throws IOException; + T deserialize(byte[] message, TablePath tablePath) throws IOException; - default void deserialize(byte[] message, Collector out) throws IOException { - T deserialize = deserialize(message); + default void deserialize(byte[] message, Collector out, TablePath tablePath) + throws IOException { + T deserialize = deserialize(message, tablePath); if (deserialize != null) { out.collect(deserialize); } diff --git a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/deserialize/AmazonSqsDeserializer.java b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/deserialize/AmazonSqsDeserializer.java index 7322feda795..e5b175aa7ef 100644 --- a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/deserialize/AmazonSqsDeserializer.java +++ b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/deserialize/AmazonSqsDeserializer.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.amazonsqs.deserialize; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import java.io.IOException; @@ -33,7 +34,7 @@ public AmazonSqsDeserializer(DeserializationSchema deserialization @Override public SeaTunnelRow deserializeRow(String row) { try { - return deserializationSchema.deserialize(row.getBytes()); + return deserializationSchema.deserialize(row.getBytes(), TablePath.of("")); } catch (IOException e) { return null; } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java index d8dccd86de4..c693b2ef010 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; @@ -98,7 +99,8 @@ public void read(String path, String tableId, Collector output) line -> { try { SeaTunnelRow seaTunnelRow = - deserializationSchema.deserialize(line.getBytes()); + deserializationSchema.deserialize( + line.getBytes(), TablePath.of("")); if (isMergePartition) { int index = seaTunnelRowType.getTotalFields(); for (String value : partitionsMap.values()) { diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/TextReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/TextReadStrategy.java index 586d165e1b0..87b0c38f598 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/TextReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/TextReadStrategy.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -95,7 +96,8 @@ public void read(String path, String tableId, Collector output) line -> { try { SeaTunnelRow seaTunnelRow = - deserializationSchema.deserialize(line.getBytes()); + deserializationSchema.deserialize( + line.getBytes(), TablePath.of(tableId)); if (!readColumns.isEmpty()) { // need column projection Object[] fields; diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java index 3dc933d2f9c..44553e76be0 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; import org.apache.seatunnel.connectors.seatunnel.google.sheets.exception.GoogleSheetsConnectorException; @@ -51,7 +52,7 @@ public SeaTunnelRow deserializeRow(List row) { } } String rowStr = objectMapper.writeValueAsString(map); - return deserializationSchema.deserialize(rowStr.getBytes()); + return deserializationSchema.deserialize(rowStr.getBytes(), TablePath.of("")); } catch (IOException e) { throw new GoogleSheetsConnectorException( CommonErrorCodeDeprecated.JSON_OPERATION_FAILED, diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/DeserializationCollector.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/DeserializationCollector.java index 59743c7832e..946b7f8b00e 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/DeserializationCollector.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/DeserializationCollector.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.format.json.JsonDeserializationSchema; @@ -35,7 +36,7 @@ public void collect(byte[] message, Collector out) throws IOExcept if (deserializationSchema instanceof JsonDeserializationSchema) { ((JsonDeserializationSchema) deserializationSchema).collect(message, out); } else { - SeaTunnelRow deserialize = deserializationSchema.deserialize(message); + SeaTunnelRow deserialize = deserializationSchema.deserialize(message, TablePath.of("")); out.collect(deserialize); } } diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java index a7c73c34e01..a1b8b93d7b1 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.http.source; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -30,8 +31,10 @@ public class SimpleTextDeserializationSchema implements DeserializationSchema>> TOPIC_LIST = + Options.key("topic_list") + .type(new TypeReference>>() {}) + .noDefaultValue() + .withDescription("topic list config"); } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java index e19fbc79244..6add204d413 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java @@ -17,6 +17,9 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.source; +import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.kafka.config.StartMode; import org.apache.kafka.common.TopicPartition; @@ -33,11 +36,12 @@ public class ConsumerMetadata implements Serializable { private String topic; private boolean isPattern = false; - private String bootstrapServers; private Properties properties; - private String consumerGroup; private boolean commitOnCheckpoint = false; + private String consumerGroup; private StartMode startMode = StartMode.GROUP_OFFSETS; private Map specificStartOffsets; private Long startOffsetsTimestamp; + private DeserializationSchema deserializationSchema; + private CatalogTable catalogTable; } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java index 62e9d7b9fdf..90d46bc502e 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java @@ -37,15 +37,15 @@ public class KafkaConsumerThread implements Runnable { private final LinkedBlockingQueue>> tasks; - public KafkaConsumerThread(ConsumerMetadata metadata) { + public KafkaConsumerThread(KafkaSourceConfig kafkaSourceConfig, ConsumerMetadata metadata) { this.metadata = metadata; this.tasks = new LinkedBlockingQueue<>(); this.consumer = initConsumer( - this.metadata.getBootstrapServers(), - this.metadata.getConsumerGroup(), - this.metadata.getProperties(), - !this.metadata.isCommitOnCheckpoint()); + kafkaSourceConfig.getBootstrap(), + metadata.getConsumerGroup(), + kafkaSourceConfig.getProperties(), + kafkaSourceConfig.isCommitOnCheckpoint()); } @Override @@ -64,7 +64,9 @@ public void run() { } } finally { try { - consumer.close(); + if (consumer != null) { + consumer.close(); + } } catch (Throwable t) { throw new KafkaConnectorException(KafkaConnectorErrorCode.CONSUMER_CLOSE_FAILED, t); } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java index 6a737bb1ae9..19f67511a7c 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java @@ -29,9 +29,8 @@ import org.apache.seatunnel.common.constants.JobMode; import org.apache.seatunnel.connectors.seatunnel.kafka.state.KafkaSourceState; -import com.google.common.collect.Lists; - import java.util.List; +import java.util.stream.Collectors; public class KafkaSource implements SeaTunnelSource, @@ -59,15 +58,16 @@ public String getPluginName() { @Override public List getProducedCatalogTables() { - return Lists.newArrayList(kafkaSourceConfig.getCatalogTable()); + return kafkaSourceConfig.getMapMetadata().values().stream() + .map(ConsumerMetadata::getCatalogTable) + .collect(Collectors.toList()); } @Override public SourceReader createReader( SourceReader.Context readerContext) { return new KafkaSourceReader( - kafkaSourceConfig.getMetadata(), - kafkaSourceConfig.getDeserializationSchema(), + kafkaSourceConfig, readerContext, kafkaSourceConfig.getMessageFormatErrorHandleWay()); } @@ -76,7 +76,7 @@ public SourceReader createReader( public SourceSplitEnumerator createEnumerator( SourceSplitEnumerator.Context enumeratorContext) { return new KafkaSourceSplitEnumerator( - kafkaSourceConfig.getMetadata(), + kafkaSourceConfig, enumeratorContext, kafkaSourceConfig.getDiscoveryIntervalMillis()); } @@ -86,7 +86,7 @@ public SourceSplitEnumerator restoreEnumerat SourceSplitEnumerator.Context enumeratorContext, KafkaSourceState checkpointState) { return new KafkaSourceSplitEnumerator( - kafkaSourceConfig.getMetadata(), + kafkaSourceConfig, enumeratorContext, checkpointState, kafkaSourceConfig.getDiscoveryIntervalMillis()); diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index 45ae694d54e..a213fb90557 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -20,16 +20,18 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.schema.ReadonlyConfigParser; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.connectors.seatunnel.kafka.config.Config; import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormat; import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormatErrorHandleWay; import org.apache.seatunnel.connectors.seatunnel.kafka.config.StartMode; @@ -39,25 +41,26 @@ import org.apache.seatunnel.format.json.canal.CanalJsonDeserializationSchema; import org.apache.seatunnel.format.json.debezium.DebeziumJsonDeserializationSchema; import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; -import org.apache.seatunnel.format.json.ogg.OggJsonDeserializationSchema; import org.apache.seatunnel.format.text.TextDeserializationSchema; import org.apache.seatunnel.format.text.constant.TextFormatConstant; import org.apache.commons.collections4.MapUtils; import org.apache.kafka.common.TopicPartition; +import lombok.Data; import lombok.Getter; import java.io.Serializable; 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.stream.Collectors; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.BOOTSTRAP_SERVERS; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.COMMIT_ON_CHECKPOINT; -import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.CONNECTOR_IDENTITY; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.CONSUMER_GROUP; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.DEBEZIUM_RECORD_INCLUDE_SCHEMA; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.FIELD_DELIMITER; @@ -71,37 +74,65 @@ import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE_TIMESTAMP; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.TOPIC; +@Data public class KafkaSourceConfig implements Serializable { private static final long serialVersionUID = 1L; - @Getter private final ConsumerMetadata metadata; - - @Getter private final DeserializationSchema deserializationSchema; - - @Getter private final CatalogTable catalogTable; + @Getter private final Map mapMetadata; @Getter private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; @Getter private final long discoveryIntervalMillis; + @Getter private final String bootstrap; + @Getter private Properties properties; + @Getter private final boolean commitOnCheckpoint; + public KafkaSourceConfig(ReadonlyConfig readonlyConfig) { - this.metadata = createConsumerMetadata(readonlyConfig); + this.bootstrap = readonlyConfig.get(BOOTSTRAP_SERVERS); + this.mapMetadata = createMapConsumerMetadata(readonlyConfig); this.discoveryIntervalMillis = readonlyConfig.get(KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS); this.messageFormatErrorHandleWay = readonlyConfig.get(MESSAGE_FORMAT_ERROR_HANDLE_WAY_OPTION); - this.catalogTable = createCatalogTable(readonlyConfig); - this.deserializationSchema = createDeserializationSchema(catalogTable, readonlyConfig); + this.commitOnCheckpoint = readonlyConfig.get(COMMIT_ON_CHECKPOINT); + this.properties = createKafkaProperties(readonlyConfig); + } + + private Properties createKafkaProperties(ReadonlyConfig readonlyConfig) { + Properties resultProperties = new Properties(); + readonlyConfig + .getOptional(KAFKA_CONFIG) + .ifPresent(kafkaConfig -> resultProperties.putAll(kafkaConfig)); + return resultProperties; + } + + private Map createMapConsumerMetadata( + ReadonlyConfig readonlyConfig) { + List consumerMetadataList; + if (readonlyConfig.getOptional(Config.TOPIC_LIST).isPresent()) { + consumerMetadataList = + readonlyConfig.get(Config.TOPIC_LIST).stream() + .map(ReadonlyConfig::fromMap) + .map(config -> createConsumerMetadata(config)) + .collect(Collectors.toList()); + } else { + consumerMetadataList = + Collections.singletonList(createConsumerMetadata(readonlyConfig)); + } + Map tablePathConsumer = new HashMap<>(); + for (ConsumerMetadata consumerMetadata : consumerMetadataList) { + tablePathConsumer.put(TablePath.of(consumerMetadata.getTopic()), consumerMetadata); + } + return tablePathConsumer; } private ConsumerMetadata createConsumerMetadata(ReadonlyConfig readonlyConfig) { ConsumerMetadata consumerMetadata = new ConsumerMetadata(); consumerMetadata.setTopic(readonlyConfig.get(TOPIC)); - consumerMetadata.setBootstrapServers(readonlyConfig.get(BOOTSTRAP_SERVERS)); consumerMetadata.setPattern(readonlyConfig.get(PATTERN)); - consumerMetadata.setProperties(new Properties()); consumerMetadata.setConsumerGroup(readonlyConfig.get(CONSUMER_GROUP)); - consumerMetadata.setCommitOnCheckpoint(readonlyConfig.get(COMMIT_ON_CHECKPOINT)); + consumerMetadata.setProperties(new Properties()); // parse start mode readonlyConfig .getOptional(START_MODE) @@ -152,25 +183,25 @@ private ConsumerMetadata createConsumerMetadata(ReadonlyConfig readonlyConfig) { } }); - readonlyConfig - .getOptional(KAFKA_CONFIG) - .ifPresent( - kafkaConfig -> - kafkaConfig.forEach( - (key, value) -> - consumerMetadata.getProperties().put(key, value))); - + CatalogTable catalogTable = createCatalogTable(readonlyConfig); + consumerMetadata.setCatalogTable(catalogTable); + consumerMetadata.setDeserializationSchema( + createDeserializationSchema(catalogTable, readonlyConfig)); return consumerMetadata; } private CatalogTable createCatalogTable(ReadonlyConfig readonlyConfig) { Optional> schemaOptions = readonlyConfig.getOptional(TableSchemaOptions.SCHEMA); + if (readonlyConfig.get(TOPIC) == null) { + throw new RuntimeException("Make sure the `topic` configuration option is not empty"); + } + TablePath tablePath = TablePath.of(readonlyConfig.get(TOPIC)); + TableSchema tableSchema; if (schemaOptions.isPresent()) { - return CatalogTableUtil.buildWithConfig(readonlyConfig); + tableSchema = new ReadonlyConfigParser().parse(readonlyConfig); } else { - TableIdentifier tableIdentifier = TableIdentifier.of(CONNECTOR_IDENTITY, null, null); - TableSchema tableSchema = + tableSchema = TableSchema.builder() .column( PhysicalColumn.of( @@ -185,13 +216,13 @@ private CatalogTable createCatalogTable(ReadonlyConfig readonlyConfig) { null, null)) .build(); - return CatalogTable.of( - tableIdentifier, - tableSchema, - Collections.emptyMap(), - Collections.emptyList(), - null); } + return CatalogTable.of( + TableIdentifier.of("", tablePath), + tableSchema, + Collections.emptyMap(), + Collections.emptyList(), + null); } private DeserializationSchema createDeserializationSchema( @@ -219,10 +250,6 @@ private DeserializationSchema createDeserializationSchema( return CanalJsonDeserializationSchema.builder(seaTunnelRowType) .setIgnoreParseErrors(true) .build(); - case OGG_JSON: - return OggJsonDeserializationSchema.builder(seaTunnelRowType) - .setIgnoreParseErrors(true) - .build(); case COMPATIBLE_KAFKA_CONNECT_JSON: Boolean keySchemaEnable = readonlyConfig.get( @@ -237,8 +264,7 @@ private DeserializationSchema createDeserializationSchema( return new DebeziumJsonDeserializationSchema(seaTunnelRowType, true, includeSchema); default: throw new SeaTunnelJsonFormatException( - CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, - "Unsupported format: " + format); + CommonErrorCode.UNSUPPORTED_DATA_TYPE, "Unsupported format: " + format); } } } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java index 4a41d38d6d6..7ba63479cd7 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java @@ -42,7 +42,8 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required(Config.TOPIC, Config.BOOTSTRAP_SERVERS) + .required(Config.BOOTSTRAP_SERVERS) + .exclusive(Config.TOPIC, Config.TOPIC_LIST) .optional( Config.START_MODE, Config.PATTERN, diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java index a2d3bae2b4d..411d30934ad 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormatErrorHandleWay; import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorErrorCode; @@ -58,12 +59,13 @@ public class KafkaSourceReader implements SourceReader tablePathMetadataMap; private final Set sourceSplits; private final Map> checkpointOffsetMap; private final Map consumerThreadMap; private final ExecutorService executorService; - private final DeserializationSchema deserializationSchema; private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; private final LinkedBlockingQueue pendingPartitionsQueue; @@ -71,15 +73,14 @@ public class KafkaSourceReader implements SourceReader deserializationSchema, + KafkaSourceConfig kafkaSourceConfig, Context context, MessageFormatErrorHandleWay messageFormatErrorHandleWay) { - this.metadata = metadata; + this.kafkaSourceConfig = kafkaSourceConfig; + this.tablePathMetadataMap = kafkaSourceConfig.getMapMetadata(); this.context = context; this.messageFormatErrorHandleWay = messageFormatErrorHandleWay; this.sourceSplits = new HashSet<>(); - this.deserializationSchema = deserializationSchema; this.consumerThreadMap = new ConcurrentHashMap<>(); this.checkpointOffsetMap = new ConcurrentHashMap<>(); this.executorService = @@ -112,13 +113,21 @@ public void pollNext(Collector output) throws Exception { consumerThreadMap.computeIfAbsent( sourceSplit.getTopicPartition(), s -> { - KafkaConsumerThread thread = new KafkaConsumerThread(metadata); + ConsumerMetadata currentSplitConsumerMetaData = + tablePathMetadataMap.get(sourceSplit.getTablePath()); + KafkaConsumerThread thread = + new KafkaConsumerThread( + kafkaSourceConfig, + currentSplitConsumerMetaData); executorService.submit(thread); return thread; })); sourceSplits.forEach( sourceSplit -> { CompletableFuture completableFuture = new CompletableFuture<>(); + TablePath tablePath = sourceSplit.getTablePath(); + DeserializationSchema deserializationSchema = + tablePathMetadataMap.get(tablePath).getDeserializationSchema(); try { consumerThreadMap .get(sourceSplit.getTopicPartition()) @@ -133,7 +142,7 @@ public void pollNext(Collector output) throws Exception { new StringDeserializer(); stringDeserializer.configure( Maps.fromProperties( - this.metadata.getProperties()), + kafkaSourceConfig.getProperties()), false); consumer.assign(partitions); if (sourceSplit.getStartOffset() >= 0) { @@ -157,10 +166,13 @@ public void pollNext(Collector output) throws Exception { ((CompatibleKafkaConnectDeserializationSchema) deserializationSchema) .deserialize( - record, output); + record, output, + tablePath); } else { deserializationSchema.deserialize( - record.value(), output); + record.value(), + output, + tablePath); } } catch (IOException e) { if (this.messageFormatErrorHandleWay @@ -259,7 +271,8 @@ public void notifyCheckpointComplete(long checkpointId) { .getTasks() .put( consumer -> { - if (this.metadata.isCommitOnCheckpoint()) { + if (kafkaSourceConfig + .isCommitOnCheckpoint()) { Map offsets = new HashMap<>(); if (offset >= 0) { diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java index f894a917dce..1c7cb17678f 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.source; import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.kafka.common.TopicPartition; @@ -25,15 +26,19 @@ public class KafkaSourceSplit implements SourceSplit { + private TablePath tablePath; private TopicPartition topicPartition; private long startOffset = -1L; private long endOffset = -1L; - public KafkaSourceSplit(TopicPartition topicPartition) { + public KafkaSourceSplit(TablePath tablePath, TopicPartition topicPartition) { + this.tablePath = tablePath; this.topicPartition = topicPartition; } - public KafkaSourceSplit(TopicPartition topicPartition, long startOffset, long endOffset) { + public KafkaSourceSplit( + TablePath tablePath, TopicPartition topicPartition, long startOffset, long endOffset) { + this.tablePath = tablePath; this.topicPartition = topicPartition; this.startOffset = startOffset; this.endOffset = endOffset; @@ -63,6 +68,14 @@ public void setTopicPartition(TopicPartition topicPartition) { this.topicPartition = topicPartition; } + public TablePath getTablePath() { + return tablePath; + } + + public void setTablePath(TablePath tablePath) { + this.tablePath = tablePath; + } + @Override public String splitId() { return topicPartition.topic() + "-" + topicPartition.partition(); @@ -87,6 +100,6 @@ public int hashCode() { public KafkaSourceSplit copy() { return new KafkaSourceSplit( - this.topicPartition, this.getStartOffset(), this.getEndOffset()); + this.tablePath, this.topicPartition, this.getStartOffset(), this.getEndOffset()); } } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index 9ab9f92841d..b2d06958d40 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.source; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.common.config.Common; import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorErrorCode; import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorException; @@ -56,45 +57,49 @@ public class KafkaSourceSplitEnumerator private static final String CLIENT_ID_PREFIX = "seatunnel"; - private final ConsumerMetadata metadata; + private final Map tablePathMetadataMap; private final Context context; private long discoveryIntervalMillis; private final AdminClient adminClient; - + private final KafkaSourceConfig kafkaSourceConfig; private final Map pendingSplit; private final Map assignedSplit; private ScheduledExecutorService executor; private ScheduledFuture scheduledFuture; - KafkaSourceSplitEnumerator(ConsumerMetadata metadata, Context context) { - this.metadata = metadata; + private Map topicMappingTablePathMap = new HashMap<>(); + + KafkaSourceSplitEnumerator( + KafkaSourceConfig kafkaSourceConfig, Context context) { + this.kafkaSourceConfig = kafkaSourceConfig; + this.tablePathMetadataMap = kafkaSourceConfig.getMapMetadata(); this.context = context; this.assignedSplit = new HashMap<>(); this.pendingSplit = new HashMap<>(); - this.adminClient = initAdminClient(this.metadata.getProperties()); + this.adminClient = initAdminClient(this.kafkaSourceConfig.getProperties()); } KafkaSourceSplitEnumerator( - ConsumerMetadata metadata, + KafkaSourceConfig kafkaSourceConfig, Context context, KafkaSourceState sourceState) { - this(metadata, context); + this(kafkaSourceConfig, context); } KafkaSourceSplitEnumerator( - ConsumerMetadata metadata, + KafkaSourceConfig kafkaSourceConfig, Context context, long discoveryIntervalMillis) { - this(metadata, context); + this(kafkaSourceConfig, context); this.discoveryIntervalMillis = discoveryIntervalMillis; } KafkaSourceSplitEnumerator( - ConsumerMetadata metadata, + KafkaSourceConfig kafkaSourceConfig, Context context, KafkaSourceState sourceState, long discoveryIntervalMillis) { - this(metadata, context, sourceState); + this(kafkaSourceConfig, context, sourceState); this.discoveryIntervalMillis = discoveryIntervalMillis; } @@ -133,30 +138,41 @@ public void run() throws ExecutionException, InterruptedException { } private void setPartitionStartOffset() throws ExecutionException, InterruptedException { - Collection topicPartitions = pendingSplit.keySet(); + Collection pendingSplitopicPartitions = pendingSplit.keySet(); Map topicPartitionOffsets = null; - switch (metadata.getStartMode()) { - case EARLIEST: - topicPartitionOffsets = listOffsets(topicPartitions, OffsetSpec.earliest()); - break; - case GROUP_OFFSETS: - topicPartitionOffsets = listConsumerGroupOffsets(topicPartitions); - break; - case LATEST: - topicPartitionOffsets = listOffsets(topicPartitions, OffsetSpec.latest()); - break; - case TIMESTAMP: - topicPartitionOffsets = - listOffsets( - topicPartitions, - OffsetSpec.forTimestamp(metadata.getStartOffsetsTimestamp())); - break; - case SPECIFIC_OFFSETS: - topicPartitionOffsets = metadata.getSpecificStartOffsets(); - break; - default: - break; + Map> tablePathPartitionMap = + pendingSplitopicPartitions.stream() + .collect( + Collectors.groupingBy( + tp -> topicMappingTablePathMap.get(tp.topic()), + Collectors.toSet())); + for (TablePath tablePath : tablePathPartitionMap.keySet()) { + ConsumerMetadata metadata = tablePathMetadataMap.get(tablePath); + Set topicPartitions = tablePathPartitionMap.get(tablePath); + switch (metadata.getStartMode()) { + case EARLIEST: + topicPartitionOffsets = listOffsets(topicPartitions, OffsetSpec.earliest()); + break; + case GROUP_OFFSETS: + topicPartitionOffsets = listConsumerGroupOffsets(topicPartitions, metadata); + break; + case LATEST: + topicPartitionOffsets = listOffsets(topicPartitions, OffsetSpec.latest()); + break; + case TIMESTAMP: + topicPartitionOffsets = + listOffsets( + topicPartitions, + OffsetSpec.forTimestamp(metadata.getStartOffsetsTimestamp())); + break; + case SPECIFIC_OFFSETS: + topicPartitionOffsets = metadata.getSpecificStartOffsets(); + break; + default: + break; + } } + topicPartitionOffsets.forEach( (key, value) -> { if (pendingSplit.containsKey(key)) { @@ -237,33 +253,42 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { private AdminClient initAdminClient(Properties properties) { Properties props = new Properties(); - props.putAll(properties); + if (properties != null) { + props.putAll(properties); + } props.setProperty( - ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.metadata.getBootstrapServers()); - if (this.metadata.getProperties().get("client.id") == null) { + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSourceConfig.getBootstrap()); + if (properties.get("client.id") == null) { props.setProperty( ConsumerConfig.CLIENT_ID_CONFIG, CLIENT_ID_PREFIX + "-enumerator-admin-client-" + this.hashCode()); } else { props.setProperty( - ConsumerConfig.CLIENT_ID_CONFIG, - this.metadata.getProperties().get("client.id").toString()); + ConsumerConfig.CLIENT_ID_CONFIG, properties.get("client.id").toString()); } return AdminClient.create(props); } private Set getTopicInfo() throws ExecutionException, InterruptedException { - Collection topics; - if (this.metadata.isPattern()) { - Pattern pattern = Pattern.compile(this.metadata.getTopic()); - topics = - this.adminClient.listTopics().names().get().stream() - .filter(t -> pattern.matcher(t).matches()) - .collect(Collectors.toSet()); - } else { - topics = Arrays.asList(this.metadata.getTopic().split(",")); + Collection topics = new HashSet<>(); + for (TablePath tablePath : tablePathMetadataMap.keySet()) { + ConsumerMetadata metadata = tablePathMetadataMap.get(tablePath); + Set currentPathTopics = new HashSet<>(); + if (metadata.isPattern()) { + Pattern pattern = Pattern.compile(metadata.getTopic()); + currentPathTopics.addAll( + this.adminClient.listTopics().names().get().stream() + .filter(t -> pattern.matcher(t).matches()) + .collect(Collectors.toSet())); + } else { + currentPathTopics.addAll(Arrays.asList(metadata.getTopic().split(","))); + } + currentPathTopics.stream() + .forEach(topic -> topicMappingTablePathMap.put(topic, tablePath)); + topics.addAll(currentPathTopics); } + log.info("Discovered topics: {}", topics); Collection partitions = @@ -280,7 +305,10 @@ private Set getTopicInfo() throws ExecutionException, Interrup return partitions.stream() .map( partition -> { - KafkaSourceSplit split = new KafkaSourceSplit(partition); + KafkaSourceSplit split = + new KafkaSourceSplit( + topicMappingTablePathMap.get(partition.topic()), + partition); split.setEndOffset(latestOffsets.get(split.getTopicPartition())); return split; }) @@ -341,7 +369,8 @@ private Map listOffsets( .get(); } - public Map listConsumerGroupOffsets(Collection partitions) + public Map listConsumerGroupOffsets( + Collection partitions, ConsumerMetadata metadata) throws ExecutionException, InterruptedException { ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions().topicPartitions(new ArrayList<>(partitions)); diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java index e16d098d6c9..c3c37d7ce31 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.utils.JsonUtils; @@ -52,12 +53,13 @@ public PulsarCanalDecorator(CanalJsonDeserializationSchema canalJsonDeserializat } @Override - public SeaTunnelRow deserialize(byte[] message) throws IOException { + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { throw new UnsupportedOperationException(); } @Override - public void deserialize(byte[] message, Collector out) throws IOException { + public void deserialize(byte[] message, Collector out, TablePath tablePath) + throws IOException { JsonNode pulsarCanal = JsonUtils.parseObject(message); ArrayNode canalList = JsonUtils.parseArray(pulsarCanal.get(MESSAGE).asText()); Iterator canalIterator = canalList.elements(); @@ -66,7 +68,7 @@ public void deserialize(byte[] message, Collector out) throws IOEx // reconvert pulsar handler, reference to // https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/MessageUtils.java ObjectNode root = reconvertPulsarData((ObjectNode) next); - canalJsonDeserializationSchema.deserialize(root, out); + canalJsonDeserializationSchema.deserialize(root, out, TablePath.of("")); } } diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/reader/PulsarSourceReader.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/reader/PulsarSourceReader.java index 3475cd32d9e..a7ebbd70f1f 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/reader/PulsarSourceReader.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/reader/PulsarSourceReader.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.common.Handover; import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; import org.apache.seatunnel.connectors.seatunnel.pulsar.config.PulsarClientConfig; @@ -133,7 +134,7 @@ public void pollNext(Collector output) throws Exception { final Message message = recordWithSplitId.get().getMessage(); synchronized (output.getCheckpointLock()) { splitStates.get(splitId).setLatestConsumedId(message.getMessageId()); - deserialization.deserialize(message.getData(), output); + deserialization.deserialize(message.getData(), output, TablePath.of("")); } } if (noMoreSplitsAssignment && finishedSplits.size() == splitStates.size()) { diff --git a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java index 0076bbbdbea..e5110a65b7e 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.pulsar.source; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -62,7 +63,8 @@ void decoder() throws IOException { new PulsarCanalDecorator(canalJsonDeserializationSchema); SimpleCollector simpleCollector = new SimpleCollector(); - pulsarCanalDecorator.deserialize(json.getBytes(StandardCharsets.UTF_8), simpleCollector); + pulsarCanalDecorator.deserialize( + json.getBytes(StandardCharsets.UTF_8), simpleCollector, TablePath.of("")); Assertions.assertFalse(simpleCollector.getList().isEmpty()); for (SeaTunnelRow seaTunnelRow : simpleCollector.list) { for (Object field : seaTunnelRow.getFields()) { diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceReader.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceReader.java index df80294f147..ae40f151180 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceReader.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceReader.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.Handover; import org.apache.seatunnel.connectors.seatunnel.rabbitmq.client.RabbitmqClient; @@ -127,7 +128,7 @@ public void pollNext(Collector output) throws Exception { return; } deliveryTagsProcessedForCurrentSnapshot.add(envelope.getDeliveryTag()); - deserializationSchema.deserialize(body, output); + deserializationSchema.deserialize(body, output, TablePath.of("")); } if (Boundedness.BOUNDED.equals(context.getBoundedness())) { diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java index 7dcea693fa3..7a141e99fd6 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -86,10 +87,13 @@ public void internalPollNext(Collector output) throws Exception { deserializationSchema.getProducedType(); valuesMap.put(((SeaTunnelRowType) seaTunnelRowType).getFieldName(0), k); deserializationSchema.deserialize( - JsonUtils.toJsonString(valuesMap).getBytes(), output); + JsonUtils.toJsonString(valuesMap).getBytes(), + output, + TablePath.of("")); } } else { - deserializationSchema.deserialize(value.getBytes(), output); + deserializationSchema.deserialize( + value.getBytes(), output, TablePath.of("")); } } } diff --git a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java index 42c3788e818..896fd6e1345 100644 --- a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java +++ b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.rocketmq.exception.RocketMqConnectorErrorCode; import org.apache.seatunnel.connectors.seatunnel.rocketmq.exception.RocketMqConnectorException; @@ -158,7 +159,9 @@ record -> groupRecords.get(messageQueue); for (MessageExt record : messages) { deserializationSchema.deserialize( - record.getBody(), output); + record.getBody(), + output, + TablePath.of("")); if (Boundedness.BOUNDED.equals( context.getBoundedness()) && record.getQueueOffset() diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java index 484482322c8..6ee5b46e836 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java @@ -35,6 +35,9 @@ import com.hazelcast.cluster.Address; import com.hazelcast.cluster.Cluster; import com.hazelcast.cluster.Member; +import com.hazelcast.core.Hazelcast; +import com.hazelcast.core.HazelcastInstance; +import com.hazelcast.instance.impl.HazelcastInstanceProxy; import com.hazelcast.internal.ascii.TextCommandService; import com.hazelcast.internal.ascii.rest.HttpCommandProcessor; import com.hazelcast.internal.ascii.rest.HttpGetCommand; @@ -211,7 +214,25 @@ private Map getJobMetrics(String jobMetrics) { private SeaTunnelServer getSeaTunnelServer() { Map extensionServices = this.textCommandService.getNode().getNodeExtension().createExtensionServices(); - return (SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME); + SeaTunnelServer seaTunnelServer = + (SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME); + if (!seaTunnelServer.isMasterNode()) { + for (HazelcastInstance hazelcastInstance : Hazelcast.getAllHazelcastInstances()) { + SeaTunnelServer seaTunnelServer1 = + (SeaTunnelServer) + ((HazelcastInstanceProxy) hazelcastInstance) + .getOriginal() + .node + .getNodeExtension() + .createExtensionServices() + .get(Constant.SEATUNNEL_SERVICE_NAME); + + if (seaTunnelServer1.isMasterNode()) { + return seaTunnelServer1; + } + } + } + return seaTunnelServer; } private JsonObject convertToJson(JobInfo jobInfo, long jobId) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java index 02158bbf342..d9920eb2797 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java @@ -32,6 +32,9 @@ import org.apache.seatunnel.engine.server.log.Log4j2HttpPostCommandProcessor; import org.apache.seatunnel.engine.server.utils.RestUtil; +import com.hazelcast.core.Hazelcast; +import com.hazelcast.core.HazelcastInstance; +import com.hazelcast.instance.impl.HazelcastInstanceProxy; import com.hazelcast.internal.ascii.TextCommandService; import com.hazelcast.internal.ascii.rest.HttpCommandProcessor; import com.hazelcast.internal.ascii.rest.HttpPostCommand; @@ -92,7 +95,25 @@ public void handle(HttpPostCommand httpPostCommand) { private SeaTunnelServer getSeaTunnelServer() { Map extensionServices = this.textCommandService.getNode().getNodeExtension().createExtensionServices(); - return (SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME); + SeaTunnelServer seaTunnelServer = + (SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME); + if (!seaTunnelServer.isMasterNode()) { + for (HazelcastInstance hazelcastInstance : Hazelcast.getAllHazelcastInstances()) { + seaTunnelServer = + (SeaTunnelServer) + ((HazelcastInstanceProxy) hazelcastInstance) + .getOriginal() + .node + .getNodeExtension() + .createExtensionServices() + .get(Constant.SEATUNNEL_SERVICE_NAME); + + if (seaTunnelServer.isMasterNode()) { + return seaTunnelServer; + } + } + } + return seaTunnelServer; } private void handleSubmitJob(HttpPostCommand httpPostCommand, String uri) @@ -115,7 +136,26 @@ private void handleSubmitJob(HttpPostCommand httpPostCommand, String uri) ? Long.parseLong(requestParams.get(RestConstant.JOB_ID)) : null); JobImmutableInformation jobImmutableInformation = restJobExecutionEnvironment.build(); - CoordinatorService coordinatorService = getSeaTunnelServer().getCoordinatorService(); + SeaTunnelServer seaTunnelServer = getSeaTunnelServer(); + Long jobId = + submitJob( + seaTunnelServer, + jobImmutableInformation, + jobConfig, + restJobExecutionEnvironment); + this.prepareResponse( + httpPostCommand, + new JsonObject() + .add(RestConstant.JOB_ID, jobId) + .add(RestConstant.JOB_NAME, requestParams.get(RestConstant.JOB_NAME))); + } + + private Long submitJob( + SeaTunnelServer seaTunnelServer, + JobImmutableInformation jobImmutableInformation, + JobConfig jobConfig, + RestJobExecutionEnvironment restJobExecutionEnvironment) { + CoordinatorService coordinatorService = seaTunnelServer.getCoordinatorService(); Data data = textCommandService .getNode() @@ -127,12 +167,7 @@ private void handleSubmitJob(HttpPostCommand httpPostCommand, String uri) Long.parseLong(jobConfig.getJobContext().getJobId()), data); voidPassiveCompletableFuture.join(); - Long jobId = restJobExecutionEnvironment.getJobId(); - this.prepareResponse( - httpPostCommand, - new JsonObject() - .add(RestConstant.JOB_ID, jobId) - .add(RestConstant.JOB_NAME, requestParams.get(RestConstant.JOB_NAME))); + return restJobExecutionEnvironment.getJobId(); } private void handleStopJob(HttpPostCommand httpPostCommand, String uri) { diff --git a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java index da2f8e039c1..625f9be061e 100644 --- a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java @@ -21,11 +21,12 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.common.utils.ReflectionUtils; import org.apache.seatunnel.format.json.JsonToRowConverters; import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; @@ -87,7 +88,7 @@ public CompatibleKafkaConnectDeserializationSchema( } @Override - public SeaTunnelRow deserialize(byte[] message) throws IOException { + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { throw new UnsupportedEncodingException(); } @@ -98,7 +99,8 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { * @param out * @throws Exception */ - public void deserialize(ConsumerRecord msg, Collector out) + public void deserialize( + ConsumerRecord msg, Collector out, TablePath tablePath) throws InvocationTargetException, IllegalAccessException { tryInitConverter(); SinkRecord record = convertToSinkRecord(msg); @@ -113,11 +115,18 @@ public void deserialize(ConsumerRecord msg, Collector dataType) { } @Override - public SeaTunnelRow deserialize(byte[] message) throws IOException { + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { if (message == null) { return null; } - return convertJsonNode(convertBytes(message)); + SeaTunnelRow seaTunnelRow = convertJsonNode(convertBytes(message)); + if (seaTunnelRow != null) { + seaTunnelRow.setTableId(tablePath.toString()); + } + return seaTunnelRow; } public SeaTunnelRow deserialize(String message) throws IOException { @@ -134,7 +140,7 @@ private SeaTunnelRow convertJsonNode(JsonNode jsonNode) { return null; } throw new SeaTunnelJsonFormatException( - CommonErrorCodeDeprecated.JSON_OPERATION_FAILED, + CommonErrorCode.JSON_OPERATION_FAILED, String.format("Failed to deserialize JSON '%s'.", jsonNode), t); } @@ -156,7 +162,7 @@ private JsonNode convertBytes(byte[] message) { return NullNode.getInstance(); } throw new SeaTunnelJsonFormatException( - CommonErrorCodeDeprecated.JSON_OPERATION_FAILED, + CommonErrorCode.JSON_OPERATION_FAILED, String.format("Failed to deserialize JSON '%s'.", new String(message)), t); } @@ -170,7 +176,7 @@ private JsonNode convert(String message) { return NullNode.getInstance(); } throw new SeaTunnelJsonFormatException( - CommonErrorCodeDeprecated.JSON_OPERATION_FAILED, + CommonErrorCode.JSON_OPERATION_FAILED, String.format("Failed to deserialize JSON '%s'.", message), t); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java index 96482291ecd..f5e4ee3514e 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java @@ -24,13 +24,14 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonError; -import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; +import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.format.json.JsonDeserializationSchema; +import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; import java.io.IOException; import java.util.regex.Pattern; @@ -38,9 +39,8 @@ import static java.lang.String.format; public class CanalJsonDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; - private static final String FORMAT = "Canal"; + private static final long serialVersionUID = 1L; private static final String FIELD_OLD = "old"; @@ -105,9 +105,8 @@ public CanalJsonDeserializationSchema( } @Override - public SeaTunnelRow deserialize(byte[] message) throws IOException { - throw new UnsupportedOperationException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + throw new UnsupportedOperationException(); } @Override @@ -115,93 +114,99 @@ public SeaTunnelDataType getProducedType() { return this.physicalRowType; } - public void deserialize(ObjectNode jsonNode, Collector out) throws IOException { - try { - if (database != null - && !databasePattern.matcher(jsonNode.get(FIELD_DATABASE).asText()).matches()) { - return; - } - if (table != null - && !tablePattern.matcher(jsonNode.get(FIELD_TABLE).asText()).matches()) { + @Override + public void deserialize(byte[] message, Collector out, TablePath tablePath) { + if (message == null) { + return; + } + ObjectNode jsonNode = (ObjectNode) convertBytes(message); + assert jsonNode != null; + deserialize(jsonNode, out, tablePath); + } + + public void deserialize(ObjectNode jsonNode, Collector out, TablePath tablePath) { + if (database != null + && !databasePattern.matcher(jsonNode.get(FIELD_DATABASE).asText()).matches()) { + return; + } + if (table != null && !tablePattern.matcher(jsonNode.get(FIELD_TABLE).asText()).matches()) { + return; + } + JsonNode dataNode = jsonNode.get(FIELD_DATA); + String type = jsonNode.get(FIELD_TYPE).asText(); + // When a null value is encountered, an exception needs to be thrown for easy sensing + if (dataNode == null || dataNode.isNull()) { + // We'll skip the query or create or alter event data + if (OP_QUERY.equals(type) || OP_CREATE.equals(type) || OP_ALTER.equals(type)) { return; } - - JsonNode dataNode = jsonNode.get(FIELD_DATA); - String type = jsonNode.get(FIELD_TYPE).asText(); - // When a null value is encountered, an exception needs to be thrown for easy sensing - if (dataNode == null || dataNode.isNull()) { - // We'll skip the query or create or alter event data - if (OP_QUERY.equals(type) || OP_CREATE.equals(type) || OP_ALTER.equals(type)) { - return; + throw new SeaTunnelJsonFormatException( + CommonErrorCode.JSON_OPERATION_FAILED, + format("Null data value \"%s\" Cannot send downstream", jsonNode)); + } + if (OP_INSERT.equals(type)) { + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow row = convertJsonNode(dataNode.get(i)); + if (row != null) { + row.setTableId(tablePath.toString()); } - throw new IllegalStateException( - format("Null data value '%s' Cannot send downstream", jsonNode)); + out.collect(row); } - if (OP_INSERT.equals(type)) { - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow row = convertJsonNode(dataNode.get(i)); - out.collect(row); - } - } else if (OP_UPDATE.equals(type)) { - final ArrayNode oldNode = (ArrayNode) jsonNode.get(FIELD_OLD); - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow after = convertJsonNode(dataNode.get(i)); - SeaTunnelRow before = convertJsonNode(oldNode.get(i)); - for (int f = 0; f < fieldCount; f++) { - if (before.isNullAt(f) && oldNode.findValue(fieldNames[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)); - } + } else if (OP_UPDATE.equals(type)) { + final ArrayNode oldNode = (ArrayNode) jsonNode.get(FIELD_OLD); + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow after = convertJsonNode(dataNode.get(i)); + SeaTunnelRow before = convertJsonNode(oldNode.get(i)); + for (int f = 0; f < fieldCount; f++) { + assert before != null; + if (before.isNullAt(f) && oldNode.findValue(fieldNames[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 + assert after != null; + 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)) { - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow row = convertJsonNode(dataNode.get(i)); - row.setRowKind(RowKind.DELETE); - out.collect(row); - } - } else { - throw new IllegalStateException(format("Unknown operation type '%s'.", type)); + assert before != null; + before.setRowKind(RowKind.UPDATE_BEFORE); + assert after != null; + after.setRowKind(RowKind.UPDATE_AFTER); + after.setTableId(tablePath.toString()); + before.setTableId(tablePath.toString()); + out.collect(before); + out.collect(after); + } + } else if (OP_DELETE.equals(type)) { + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow row = convertJsonNode(dataNode.get(i)); + assert row != null; + row.setRowKind(RowKind.DELETE); + row.setTableId(tablePath.toString()); + out.collect(row); } - } catch (Throwable t) { + } else { if (!ignoreParseErrors) { - throw CommonError.jsonOperationError(FORMAT, jsonNode.toString(), t); + throw new SeaTunnelJsonFormatException( + CommonErrorCode.UNSUPPORTED_DATA_TYPE, + format( + "Unknown \"type\" value \"%s\". The Canal JSON message is '%s'", + type, jsonNode.asText())); } } } - private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException { - try { - return (ObjectNode) jsonDeserializer.deserializeToJsonNode(message); - } catch (Throwable t) { - throw CommonError.jsonOperationError(FORMAT, new String(message), t); - } - } - - @Override - public void deserialize(byte[] message, Collector out) throws IOException { - if (message == null) { - return; - } - - ObjectNode jsonNode; + private JsonNode convertBytes(byte[] message) { try { - jsonNode = convertBytes(message); - } catch (SeaTunnelRuntimeException cause) { - if (!ignoreParseErrors) { - throw cause; - } else { - return; + return jsonDeserializer.deserializeToJsonNode(message); + } catch (Exception t) { + if (ignoreParseErrors) { + return null; } + throw new SeaTunnelJsonFormatException( + CommonErrorCode.JSON_OPERATION_FAILED, + String.format("Failed to deserialize JSON '%s'.", new String(message)), + t); } - - deserialize(jsonNode, out); } private SeaTunnelRow convertJsonNode(JsonNode root) { diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java index 1dece25824a..749c9026f4d 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java @@ -21,11 +21,12 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.format.json.JsonDeserializationSchema; import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; @@ -70,13 +71,14 @@ public DebeziumJsonDeserializationSchema( } @Override - public SeaTunnelRow deserialize(byte[] message) throws IOException { + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { throw new UnsupportedOperationException( "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); } @Override - public void deserialize(byte[] message, Collector out) throws IOException { + public void deserialize(byte[] message, Collector out, TablePath tablePath) + throws IOException { if (message == null || message.length == 0) { // skip tombstone messages return; @@ -89,33 +91,37 @@ public void deserialize(byte[] message, Collector out) throws IOEx if (OP_CREATE.equals(op) || OP_READ.equals(op)) { SeaTunnelRow insert = convertJsonNode(payload.get("after")); insert.setRowKind(RowKind.INSERT); + insert.setTableId(tablePath.toString()); out.collect(insert); } else if (OP_UPDATE.equals(op)) { SeaTunnelRow before = convertJsonNode(payload.get("before")); if (before == null) { throw new SeaTunnelJsonFormatException( - CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + CommonErrorCode.UNSUPPORTED_DATA_TYPE, String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); } before.setRowKind(RowKind.UPDATE_BEFORE); + before.setTableId(tablePath.toString()); out.collect(before); SeaTunnelRow after = convertJsonNode(payload.get("after")); after.setRowKind(RowKind.UPDATE_AFTER); + after.setTableId(tablePath.toString()); out.collect(after); } else if (OP_DELETE.equals(op)) { SeaTunnelRow delete = convertJsonNode(payload.get("before")); if (delete == null) { throw new SeaTunnelJsonFormatException( - CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + CommonErrorCode.UNSUPPORTED_DATA_TYPE, String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); } delete.setRowKind(RowKind.DELETE); + delete.setTableId(tablePath.toString()); out.collect(delete); } else { if (!ignoreParseErrors) { throw new SeaTunnelJsonFormatException( - CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + CommonErrorCode.UNSUPPORTED_DATA_TYPE, String.format( "Unknown \"op\" value \"%s\". The Debezium JSON message is '%s'", op, new String(message))); @@ -125,7 +131,7 @@ public void deserialize(byte[] message, Collector out) throws IOEx // a big try catch to protect the processing. if (!ignoreParseErrors) { throw new SeaTunnelJsonFormatException( - CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + CommonErrorCode.UNSUPPORTED_DATA_TYPE, String.format("Corrupt Debezium JSON message '%s'.", new String(message)), t); } @@ -147,7 +153,7 @@ private JsonNode convertBytes(byte[] message) { return null; } throw new SeaTunnelJsonFormatException( - CommonErrorCodeDeprecated.JSON_OPERATION_FAILED, + CommonErrorCode.JSON_OPERATION_FAILED, String.format("Failed to deserialize JSON '%s'.", new String(message)), t); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java index 36e6e541c42..5e8b28b6f8d 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -102,7 +103,7 @@ public OggJsonDeserializationSchema( } @Override - public SeaTunnelRow deserialize(byte[] message) throws IOException { + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { throw new UnsupportedOperationException( "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); } diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java index d9ff06965b6..ed7d34de489 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.LocalTimeType; import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -139,7 +140,8 @@ public void testSerDe() throws Exception { expected.setField(10, multiSet); expected.setField(11, nestedMap); - SeaTunnelRow seaTunnelRow = deserializationSchema.deserialize(serializedJson); + SeaTunnelRow seaTunnelRow = + deserializationSchema.deserialize(serializedJson, TablePath.of("")); assertEquals(expected, seaTunnelRow); // test serialization @@ -186,7 +188,8 @@ public void testSerDeMultiRows() throws Exception { row.put("f1", "this is row1"); row.put("f2", 12); byte[] serializedJson = objectMapper.writeValueAsBytes(root); - SeaTunnelRow rowData = deserializationSchema.deserialize(serializedJson); + SeaTunnelRow rowData = + deserializationSchema.deserialize(serializedJson, TablePath.of("")); byte[] actual = serializationSchema.serialize(rowData); assertEquals(new String(serializedJson), new String(actual)); } @@ -206,7 +209,8 @@ public void testSerDeMultiRows() throws Exception { row.put("f1", "this is row2"); row.putNull("f2"); byte[] serializedJson = objectMapper.writeValueAsBytes(root); - SeaTunnelRow rowData = deserializationSchema.deserialize(serializedJson); + SeaTunnelRow rowData = + deserializationSchema.deserialize(serializedJson, TablePath.of("")); byte[] actual = serializationSchema.serialize(rowData); assertEquals(new String(serializedJson), new String(actual)); } @@ -247,7 +251,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { for (int i = 0; i < jsons.length; i++) { String json = jsons[i]; - SeaTunnelRow row = deserializationSchema.deserialize(json.getBytes()); + SeaTunnelRow row = deserializationSchema.deserialize(json.getBytes(), TablePath.of("")); String result = new String(serializationSchema.serialize(row)); assertEquals(expected[i], result); } @@ -270,7 +274,7 @@ public void testDeserializationMissingNode() throws Exception { JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(true, false, schema); - SeaTunnelRow rowData = deserializationSchema.deserialize("".getBytes()); + SeaTunnelRow rowData = deserializationSchema.deserialize("".getBytes(), TablePath.of("")); assertEquals(null, rowData); } @@ -291,7 +295,7 @@ public void testDeserializationMissingField() throws Exception { new JsonDeserializationSchema(false, false, schema); SeaTunnelRow expected = new SeaTunnelRow(1); - SeaTunnelRow actual = deserializationSchema.deserialize(serializedJson); + SeaTunnelRow actual = deserializationSchema.deserialize(serializedJson, TablePath.of("")); assertEquals(expected, actual); // fail on missing field @@ -300,7 +304,7 @@ public void testDeserializationMissingField() throws Exception { String errorMessage = "ErrorCode:[COMMON-02], ErrorDescription:[Json covert/parse operation failed] - Failed to deserialize JSON '{\"id\":123123123}'."; try { - deserializationSchema.deserialize(serializedJson); + deserializationSchema.deserialize(serializedJson, TablePath.of("")); fail("expecting exception message: " + errorMessage); } catch (Throwable t) { assertEquals(errorMessage, t.getMessage()); @@ -308,7 +312,7 @@ public void testDeserializationMissingField() throws Exception { // ignore on parse error deserializationSchema = new JsonDeserializationSchema(false, true, schema); - assertEquals(expected, deserializationSchema.deserialize(serializedJson)); + assertEquals(expected, deserializationSchema.deserialize(serializedJson, TablePath.of(""))); errorMessage = "ErrorCode:[COMMON-06], ErrorDescription:[Illegal argument] - JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."; diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java index 601e9c738c6..1226b71b124 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java @@ -19,11 +19,10 @@ package org.apache.seatunnel.format.json.canal; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonError; -import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.junit.jupiter.api.Test; @@ -42,10 +41,8 @@ import static org.apache.seatunnel.api.table.type.BasicType.INT_TYPE; import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; public class CanalJsonSerDeSchemaTest { - private static final String FORMAT = "Canal"; private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = new SeaTunnelRowType( @@ -69,96 +66,16 @@ public void testDeserializeNullRow() throws Exception { createCanalJsonDeserializationSchema(null, null); final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize((byte[]) null, collector); + deserializationSchema.deserialize((byte[]) null, collector, TablePath.of("")); assertEquals(0, collector.list.size()); } - @Test - public void testDeserializeNoJson() throws Exception { - final CanalJsonDeserializationSchema deserializationSchema = - createCanalJsonDeserializationSchema(null, null); - final SimpleCollector collector = new SimpleCollector(); - String noJsonMsg = "{]"; - - SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, noJsonMsg); - SeaTunnelRuntimeException cause = - assertThrows( - expected.getClass(), - () -> { - deserializationSchema.deserialize(noJsonMsg.getBytes(), collector); - }); - assertEquals(cause.getMessage(), expected.getMessage()); - } - - @Test - public void testDeserializeEmptyJson() throws Exception { - final CanalJsonDeserializationSchema deserializationSchema = - createCanalJsonDeserializationSchema(null, null); - final SimpleCollector collector = new SimpleCollector(); - String emptyMsg = "{}"; - SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, emptyMsg); - SeaTunnelRuntimeException cause = - assertThrows( - expected.getClass(), - () -> { - deserializationSchema.deserialize(emptyMsg.getBytes(), collector); - }); - assertEquals(cause.getMessage(), expected.getMessage()); - } - - @Test - public void testDeserializeNoDataJson() throws Exception { - final CanalJsonDeserializationSchema deserializationSchema = - createCanalJsonDeserializationSchema(null, null); - final SimpleCollector collector = new SimpleCollector(); - String noDataMsg = "{\"type\":\"INSERT\"}"; - SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, noDataMsg); - SeaTunnelRuntimeException cause = - assertThrows( - expected.getClass(), - () -> { - deserializationSchema.deserialize(noDataMsg.getBytes(), collector); - }); - assertEquals(cause.getMessage(), expected.getMessage()); - - Throwable noDataCause = cause.getCause(); - assertEquals(noDataCause.getClass(), IllegalStateException.class); - assertEquals( - noDataCause.getMessage(), - String.format("Null data value '%s' Cannot send downstream", noDataMsg)); - } - - @Test - public void testDeserializeUnknownTypeJson() throws Exception { - final CanalJsonDeserializationSchema deserializationSchema = - createCanalJsonDeserializationSchema(null, null); - final SimpleCollector collector = new SimpleCollector(); - String unknownType = "XX"; - String unknownOperationMsg = - "{\"data\":{\"id\":101,\"name\":\"scooter\"},\"type\":\"" + unknownType + "\"}"; - SeaTunnelRuntimeException expected = - CommonError.jsonOperationError(FORMAT, unknownOperationMsg); - SeaTunnelRuntimeException cause = - assertThrows( - expected.getClass(), - () -> { - deserializationSchema.deserialize( - unknownOperationMsg.getBytes(), collector); - }); - assertEquals(cause.getMessage(), expected.getMessage()); - - Throwable unknownTypeCause = cause.getCause(); - assertEquals(unknownTypeCause.getClass(), IllegalStateException.class); - assertEquals( - unknownTypeCause.getMessage(), - String.format("Unknown operation type '%s'.", unknownType)); - } - public void runTest(List lines, CanalJsonDeserializationSchema deserializationSchema) throws IOException { SimpleCollector collector = new SimpleCollector(); for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + deserializationSchema.deserialize( + line.getBytes(StandardCharsets.UTF_8), collector, TablePath.of("")); } List expected = diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index 20088e525bf..f244839007e 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -19,6 +19,7 @@ package org.apache.seatunnel.format.json.debezium; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -55,8 +56,8 @@ void testNullRowMessages() throws Exception { new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(null, collector); - deserializationSchema.deserialize(new byte[0], collector); + deserializationSchema.deserialize(null, collector, TablePath.of("")); + deserializationSchema.deserialize(new byte[0], collector, TablePath.of("")); assertEquals(0, collector.list.size()); } @@ -74,7 +75,8 @@ private void testSerializationDeserialization(String resourceFile, boolean schem SimpleCollector collector = new SimpleCollector(); for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + deserializationSchema.deserialize( + line.getBytes(StandardCharsets.UTF_8), collector, TablePath.of("")); } List expected = diff --git a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java index b6429eb600e..1e089d2c7b6 100644 --- a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java @@ -18,13 +18,14 @@ package org.apache.seatunnel.format.text; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.DateUtils; import org.apache.seatunnel.common.utils.TimeUtils; @@ -112,14 +113,16 @@ public TextDeserializationSchema build() { } @Override - public SeaTunnelRow deserialize(byte[] message) throws IOException { + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { String content = new String(message); Map splitsMap = splitLineBySeaTunnelRowType(content, seaTunnelRowType, 0); Object[] objects = new Object[seaTunnelRowType.getTotalFields()]; for (int i = 0; i < objects.length; i++) { objects[i] = convert(splitsMap.get(i), seaTunnelRowType.getFieldType(i), 0); } - return new SeaTunnelRow(objects); + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(objects); + seaTunnelRow.setTableId(tablePath.toString()); + return seaTunnelRow; } @Override @@ -175,7 +178,7 @@ private Object convert(String field, SeaTunnelDataType fieldType, int level) return objectArrayList.toArray(new Double[0]); default: throw new SeaTunnelTextFormatException( - CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + CommonErrorCode.UNSUPPORTED_DATA_TYPE, String.format( "SeaTunnel array not support this data type [%s]", elementType.getSqlType())); @@ -238,7 +241,7 @@ private Object convert(String field, SeaTunnelDataType fieldType, int level) return new SeaTunnelRow(objects); default: throw new SeaTunnelTextFormatException( - CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + CommonErrorCode.UNSUPPORTED_DATA_TYPE, String.format( "SeaTunnel not support this data type [%s]", fieldType.getSqlType())); diff --git a/seatunnel-formats/seatunnel-format-text/src/test/java/org/apache/seatunnel/format/text/TextFormatSchemaTest.java b/seatunnel-formats/seatunnel-format-text/src/test/java/org/apache/seatunnel/format/text/TextFormatSchemaTest.java index 57e99d49b69..7bff7cdc11b 100644 --- a/seatunnel-formats/seatunnel-format-text/src/test/java/org/apache/seatunnel/format/text/TextFormatSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-text/src/test/java/org/apache/seatunnel/format/text/TextFormatSchemaTest.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.format.text; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -136,7 +137,8 @@ public void testParse() throws IOException { .seaTunnelRowType(seaTunnelRowType) .delimiter("\u0001") .build(); - SeaTunnelRow seaTunnelRow = deserializationSchema.deserialize(content.getBytes()); + SeaTunnelRow seaTunnelRow = + deserializationSchema.deserialize(content.getBytes(), TablePath.of("")); String data = new String(serializationSchema.serialize(seaTunnelRow)); Assertions.assertEquals(((Map) (seaTunnelRow.getField(1))).get("tyrantlucifer"), 18); Assertions.assertEquals(((Map) (seaTunnelRow.getField(1))).get("Kris"), 21); From 5bbbc40fae792cf800e28408dd8c3ea579cc45c3 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 00:01:11 +0800 Subject: [PATCH 02/71] first commit --- .../connectors/seatunnel/kafka/source/KafkaSourceConfig.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index a213fb90557..dfb37a82e39 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -101,9 +101,7 @@ public KafkaSourceConfig(ReadonlyConfig readonlyConfig) { private Properties createKafkaProperties(ReadonlyConfig readonlyConfig) { Properties resultProperties = new Properties(); - readonlyConfig - .getOptional(KAFKA_CONFIG) - .ifPresent(kafkaConfig -> resultProperties.putAll(kafkaConfig)); + readonlyConfig.getOptional(KAFKA_CONFIG).ifPresent(resultProperties::putAll); return resultProperties; } From 97ffb2b41a2bef72df31061644984597cf4c7bea Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 00:07:26 +0800 Subject: [PATCH 03/71] [Feature][Kafka] Support multi-table source read #5992 --- .../kafka/source/KafkaSourceSplitEnumerator.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index b2d06958d40..f70da292c43 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -258,13 +258,13 @@ private AdminClient initAdminClient(Properties properties) { } props.setProperty( ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSourceConfig.getBootstrap()); - if (properties.get("client.id") == null) { + if (properties.get("client.id") != null) { props.setProperty( - ConsumerConfig.CLIENT_ID_CONFIG, - CLIENT_ID_PREFIX + "-enumerator-admin-client-" + this.hashCode()); + ConsumerConfig.CLIENT_ID_CONFIG, properties.get("client.id").toString()); } else { props.setProperty( - ConsumerConfig.CLIENT_ID_CONFIG, properties.get("client.id").toString()); + ConsumerConfig.CLIENT_ID_CONFIG, + CLIENT_ID_PREFIX + "-enumerator-admin-client-" + this.hashCode()); } return AdminClient.create(props); From 77eec173f6ba211f5c241bc6e2dca7f06e391830 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 10:48:12 +0800 Subject: [PATCH 04/71] [Feature][Kafka] Support multi-table source read #5992 --- .../json/JsonDeserializationSchema.java | 7 +- .../canal/CanalJsonDeserializationSchema.java | 172 +++++++++--------- .../DebeziumJsonDeserializationSchema.java | 12 +- .../ogg/OggJsonDeserializationSchema.java | 20 +- .../json/ogg/OggJsonSerDeSchemaTest.java | 27 ++- 5 files changed, 126 insertions(+), 112 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 7d06bee5609..659dc5a049e 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -33,7 +33,6 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.api.table.type.SqlType; -import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; @@ -140,7 +139,7 @@ private SeaTunnelRow convertJsonNode(JsonNode jsonNode) { return null; } throw new SeaTunnelJsonFormatException( - CommonErrorCode.JSON_OPERATION_FAILED, + CommonErrorCodeDeprecated.JSON_OPERATION_FAILED, String.format("Failed to deserialize JSON '%s'.", jsonNode), t); } @@ -162,7 +161,7 @@ private JsonNode convertBytes(byte[] message) { return NullNode.getInstance(); } throw new SeaTunnelJsonFormatException( - CommonErrorCode.JSON_OPERATION_FAILED, + CommonErrorCodeDeprecated.JSON_OPERATION_FAILED, String.format("Failed to deserialize JSON '%s'.", new String(message)), t); } @@ -176,7 +175,7 @@ private JsonNode convert(String message) { return NullNode.getInstance(); } throw new SeaTunnelJsonFormatException( - CommonErrorCode.JSON_OPERATION_FAILED, + CommonErrorCodeDeprecated.JSON_OPERATION_FAILED, String.format("Failed to deserialize JSON '%s'.", message), t); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java index f5e4ee3514e..3aaffd1d345 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java @@ -29,9 +29,9 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.common.exception.CommonError; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.format.json.JsonDeserializationSchema; -import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; import java.io.IOException; import java.util.regex.Pattern; @@ -39,9 +39,10 @@ import static java.lang.String.format; public class CanalJsonDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = 1L; + private static final String FORMAT = "Canal"; + private static final String FIELD_OLD = "old"; private static final String FIELD_DATA = "data"; @@ -64,9 +65,9 @@ public class CanalJsonDeserializationSchema implements DeserializationSchema) instead."); } @Override @@ -114,99 +116,99 @@ public SeaTunnelDataType getProducedType() { return this.physicalRowType; } - @Override - public void deserialize(byte[] message, Collector out, TablePath tablePath) { - if (message == null) { - return; - } - ObjectNode jsonNode = (ObjectNode) convertBytes(message); - assert jsonNode != null; - deserialize(jsonNode, out, tablePath); - } - - public void deserialize(ObjectNode jsonNode, Collector out, TablePath tablePath) { - if (database != null - && !databasePattern.matcher(jsonNode.get(FIELD_DATABASE).asText()).matches()) { - return; - } - if (table != null && !tablePattern.matcher(jsonNode.get(FIELD_TABLE).asText()).matches()) { - return; - } - JsonNode dataNode = jsonNode.get(FIELD_DATA); - String type = jsonNode.get(FIELD_TYPE).asText(); - // When a null value is encountered, an exception needs to be thrown for easy sensing - if (dataNode == null || dataNode.isNull()) { - // We'll skip the query or create or alter event data - if (OP_QUERY.equals(type) || OP_CREATE.equals(type) || OP_ALTER.equals(type)) { + public void deserialize(ObjectNode jsonNode, Collector out, TablePath tablePath) + throws IOException { + try { + if (database != null + && !databasePattern.matcher(jsonNode.get(FIELD_DATABASE).asText()).matches()) { return; } - throw new SeaTunnelJsonFormatException( - CommonErrorCode.JSON_OPERATION_FAILED, - format("Null data value \"%s\" Cannot send downstream", jsonNode)); - } - if (OP_INSERT.equals(type)) { - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow row = convertJsonNode(dataNode.get(i)); - if (row != null) { - row.setTableId(tablePath.toString()); + if (table != null + && !tablePattern.matcher(jsonNode.get(FIELD_TABLE).asText()).matches()) { + return; + } + + JsonNode dataNode = jsonNode.get(FIELD_DATA); + String type = jsonNode.get(FIELD_TYPE).asText(); + // When a null value is encountered, an exception needs to be thrown for easy sensing + if (dataNode == null || dataNode.isNull()) { + // We'll skip the query or create or alter event data + if (OP_QUERY.equals(type) || OP_CREATE.equals(type) || OP_ALTER.equals(type)) { + return; } - out.collect(row); + throw new IllegalStateException( + format("Null data value '%s' Cannot send downstream", jsonNode)); } - } else if (OP_UPDATE.equals(type)) { - final ArrayNode oldNode = (ArrayNode) jsonNode.get(FIELD_OLD); - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow after = convertJsonNode(dataNode.get(i)); - SeaTunnelRow before = convertJsonNode(oldNode.get(i)); - for (int f = 0; f < fieldCount; f++) { - assert before != null; - if (before.isNullAt(f) && oldNode.findValue(fieldNames[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 - assert after != null; - before.setField(f, after.getField(f)); + if (OP_INSERT.equals(type)) { + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow row = convertJsonNode(dataNode.get(i)); + row.setTableId(tablePath.toString()); + out.collect(row); + } + } else if (OP_UPDATE.equals(type)) { + final ArrayNode oldNode = (ArrayNode) jsonNode.get(FIELD_OLD); + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow after = convertJsonNode(dataNode.get(i)); + SeaTunnelRow before = convertJsonNode(oldNode.get(i)); + for (int f = 0; f < fieldCount; f++) { + if (before.isNullAt(f) && oldNode.findValue(fieldNames[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); + before.setTableId(tablePath.toString()); + after.setRowKind(RowKind.UPDATE_AFTER); + after.setTableId(tablePath.toString()); + out.collect(before); + out.collect(after); } - assert before != null; - before.setRowKind(RowKind.UPDATE_BEFORE); - assert after != null; - after.setRowKind(RowKind.UPDATE_AFTER); - after.setTableId(tablePath.toString()); - before.setTableId(tablePath.toString()); - out.collect(before); - out.collect(after); - } - } else if (OP_DELETE.equals(type)) { - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow row = convertJsonNode(dataNode.get(i)); - assert row != null; - row.setRowKind(RowKind.DELETE); - row.setTableId(tablePath.toString()); - out.collect(row); + } else if (OP_DELETE.equals(type)) { + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow row = convertJsonNode(dataNode.get(i)); + row.setRowKind(RowKind.DELETE); + row.setTableId(tablePath.toString()); + out.collect(row); + } + } else { + throw new IllegalStateException(format("Unknown operation type '%s'.", type)); } - } else { + } catch (Throwable t) { if (!ignoreParseErrors) { - throw new SeaTunnelJsonFormatException( - CommonErrorCode.UNSUPPORTED_DATA_TYPE, - format( - "Unknown \"type\" value \"%s\". The Canal JSON message is '%s'", - type, jsonNode.asText())); + throw CommonError.jsonOperationError(FORMAT, jsonNode.toString(), t); } } } - private JsonNode convertBytes(byte[] message) { + private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException { + try { + return (ObjectNode) jsonDeserializer.deserializeToJsonNode(message); + } catch (Throwable t) { + throw CommonError.jsonOperationError(FORMAT, new String(message), t); + } + } + + @Override + public void deserialize(byte[] message, Collector out, TablePath tablePath) + throws IOException { + if (message == null) { + return; + } + + ObjectNode jsonNode; try { - return jsonDeserializer.deserializeToJsonNode(message); - } catch (Exception t) { - if (ignoreParseErrors) { - return null; + jsonNode = convertBytes(message); + } catch (SeaTunnelRuntimeException cause) { + if (!ignoreParseErrors) { + throw cause; + } else { + return; } - throw new SeaTunnelJsonFormatException( - CommonErrorCode.JSON_OPERATION_FAILED, - String.format("Failed to deserialize JSON '%s'.", new String(message)), - t); } + + deserialize(jsonNode, out, tablePath); } private SeaTunnelRow convertJsonNode(JsonNode root) { diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java index 749c9026f4d..e8fd090161b 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java @@ -26,7 +26,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; import org.apache.seatunnel.format.json.JsonDeserializationSchema; import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; @@ -97,7 +97,7 @@ public void deserialize(byte[] message, Collector out, TablePath t SeaTunnelRow before = convertJsonNode(payload.get("before")); if (before == null) { throw new SeaTunnelJsonFormatException( - CommonErrorCode.UNSUPPORTED_DATA_TYPE, + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); } before.setRowKind(RowKind.UPDATE_BEFORE); @@ -112,7 +112,7 @@ public void deserialize(byte[] message, Collector out, TablePath t SeaTunnelRow delete = convertJsonNode(payload.get("before")); if (delete == null) { throw new SeaTunnelJsonFormatException( - CommonErrorCode.UNSUPPORTED_DATA_TYPE, + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); } delete.setRowKind(RowKind.DELETE); @@ -121,7 +121,7 @@ public void deserialize(byte[] message, Collector out, TablePath t } else { if (!ignoreParseErrors) { throw new SeaTunnelJsonFormatException( - CommonErrorCode.UNSUPPORTED_DATA_TYPE, + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, String.format( "Unknown \"op\" value \"%s\". The Debezium JSON message is '%s'", op, new String(message))); @@ -131,7 +131,7 @@ public void deserialize(byte[] message, Collector out, TablePath t // a big try catch to protect the processing. if (!ignoreParseErrors) { throw new SeaTunnelJsonFormatException( - CommonErrorCode.UNSUPPORTED_DATA_TYPE, + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, String.format("Corrupt Debezium JSON message '%s'.", new String(message)), t); } @@ -153,7 +153,7 @@ private JsonNode convertBytes(byte[] message) { return null; } throw new SeaTunnelJsonFormatException( - CommonErrorCode.JSON_OPERATION_FAILED, + CommonErrorCodeDeprecated.JSON_OPERATION_FAILED, String.format("Failed to deserialize JSON '%s'.", new String(message)), t); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java index 5e8b28b6f8d..6784a378866 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java @@ -62,9 +62,9 @@ public class OggJsonDeserializationSchema implements DeserializationSchema out) throws IOException { + public void deserialize(ObjectNode jsonNode, Collector out, TablePath tablePath) + throws IOException { try { if (database != null && !databasePattern @@ -141,6 +142,7 @@ public void deserialize(ObjectNode jsonNode, Collector out) throws // Gets the data for the INSERT operation JsonNode dataAfter = jsonNode.get(DATA_AFTER); SeaTunnelRow row = convertJsonNode(dataAfter); + row.setTableId(tablePath.toString()); out.collect(row); } else if (OP_UPDATE.equals(op)) { JsonNode dataBefore = jsonNode.get(DATA_BEFORE); @@ -154,11 +156,12 @@ public void deserialize(ObjectNode jsonNode, Collector out) throws SeaTunnelRow before = convertJsonNode(dataBefore); // Gets the data for the UPDATE AFTER operation SeaTunnelRow after = convertJsonNode(dataAfter); - before.setRowKind(RowKind.UPDATE_BEFORE); + before.setTableId(tablePath.toString()); out.collect(before); after.setRowKind(RowKind.UPDATE_AFTER); + after.setTableId(tablePath.toString()); out.collect(after); } else if (OP_DELETE.equals(op)) { JsonNode dataBefore = jsonNode.get(DATA_BEFORE); @@ -173,6 +176,7 @@ public void deserialize(ObjectNode jsonNode, Collector out) throws String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); } before.setRowKind(RowKind.DELETE); + before.setTableId(tablePath.toString()); out.collect(before); } else { throw new IllegalStateException(format("Unknown operation type '%s'.", op)); @@ -184,7 +188,9 @@ public void deserialize(ObjectNode jsonNode, Collector out) throws } } - public void deserialize(byte[] message, Collector out) throws IOException { + @Override + public void deserialize(byte[] message, Collector out, TablePath tablePath) + throws IOException { if (message == null || message.length == 0) { // skip tombstone messages return; @@ -200,7 +206,7 @@ public void deserialize(byte[] message, Collector out) throws IOEx return; } } - deserialize(jsonNode, out); + deserialize(jsonNode, out, tablePath); } private SeaTunnelRow convertJsonNode(JsonNode root) { diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java index 1f44c7cfbcb..ef348a67619 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java @@ -19,6 +19,7 @@ package org.apache.seatunnel.format.json.ogg; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -47,6 +48,7 @@ public class OggJsonSerDeSchemaTest { private static final String FORMAT = "Ogg"; + private static final String RESOURCE_OGG_DATA_NAME = "ogg-data-filter-table.txt"; private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = new SeaTunnelRowType( @@ -55,7 +57,7 @@ public class OggJsonSerDeSchemaTest { @Test public void testFilteringTables() throws Exception { - List lines = readLines("ogg-data-filter-table.txt"); + List lines = readLines(); OggJsonDeserializationSchema deserializationSchema = new OggJsonDeserializationSchema.Builder(PHYSICAL_DATA_TYPE) .setDatabase("^OG.*") @@ -70,7 +72,7 @@ public void testDeserializeNullRow() throws Exception { createOggJsonDeserializationSchema(null, null); final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize((byte[]) null, collector); + deserializationSchema.deserialize((byte[]) null, collector, TablePath.of("")); assertEquals(0, collector.list.size()); } @@ -86,7 +88,8 @@ public void testDeserializeNoJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize(noJsonMsg.getBytes(), collector); + deserializationSchema.deserialize( + noJsonMsg.getBytes(), collector, TablePath.of("")); }); assertEquals(cause.getMessage(), expected.getMessage()); } @@ -102,7 +105,8 @@ public void testDeserializeEmptyJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize(emptyMsg.getBytes(), collector); + deserializationSchema.deserialize( + emptyMsg.getBytes(), collector, TablePath.of("")); }); assertEquals(cause.getMessage(), expected.getMessage()); } @@ -118,7 +122,8 @@ public void testDeserializeNoDataJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize(noDataMsg.getBytes(), collector); + deserializationSchema.deserialize( + noDataMsg.getBytes(), collector, TablePath.of("")); }); assertEquals(cause.getMessage(), expected.getMessage()); @@ -148,7 +153,7 @@ public void testDeserializeUnknownTypeJson() throws Exception { expected.getClass(), () -> { deserializationSchema.deserialize( - unknownOperationMsg.getBytes(), collector); + unknownOperationMsg.getBytes(), collector, TablePath.of("")); }); assertEquals(cause.getMessage(), expected.getMessage()); @@ -163,7 +168,8 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa throws IOException { SimpleCollector collector = new SimpleCollector(); for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + deserializationSchema.deserialize( + line.getBytes(StandardCharsets.UTF_8), collector, TablePath.of("")); } List expected = @@ -239,8 +245,9 @@ private OggJsonDeserializationSchema createOggJsonDeserializationSchema( .build(); } - private static List readLines(String resource) throws IOException { - final URL url = OggJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); + private static List readLines() throws IOException { + final URL url = + OggJsonSerDeSchemaTest.class.getClassLoader().getResource(RESOURCE_OGG_DATA_NAME); Assertions.assertNotNull(url); Path path = new File(url.getFile()).toPath(); return Files.readAllLines(path); @@ -248,7 +255,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(SeaTunnelRow record) { From 2ecbc0c50b9d31bd30265e0116697a7238376345 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 14:40:40 +0800 Subject: [PATCH 05/71] [Feature][Kafka] Support multi-table source read #5992 --- .../seatunnel/kafka/source/KafkaSourceConfig.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index dfb37a82e39..4101f71d2b8 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -30,7 +30,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; import org.apache.seatunnel.connectors.seatunnel.kafka.config.Config; import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormat; import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormatErrorHandleWay; @@ -41,6 +41,7 @@ import org.apache.seatunnel.format.json.canal.CanalJsonDeserializationSchema; import org.apache.seatunnel.format.json.debezium.DebeziumJsonDeserializationSchema; import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; +import org.apache.seatunnel.format.json.ogg.OggJsonDeserializationSchema; import org.apache.seatunnel.format.text.TextDeserializationSchema; import org.apache.seatunnel.format.text.constant.TextFormatConstant; @@ -248,6 +249,10 @@ private DeserializationSchema createDeserializationSchema( return CanalJsonDeserializationSchema.builder(seaTunnelRowType) .setIgnoreParseErrors(true) .build(); + case OGG_JSON: + return OggJsonDeserializationSchema.builder(seaTunnelRowType) + .setIgnoreParseErrors(true) + .build(); case COMPATIBLE_KAFKA_CONNECT_JSON: Boolean keySchemaEnable = readonlyConfig.get( @@ -262,7 +267,8 @@ private DeserializationSchema createDeserializationSchema( return new DebeziumJsonDeserializationSchema(seaTunnelRowType, true, includeSchema); default: throw new SeaTunnelJsonFormatException( - CommonErrorCode.UNSUPPORTED_DATA_TYPE, "Unsupported format: " + format); + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + "Unsupported format: " + format); } } } From 0c6445f7bb5cdbf863d66cca0875a3bcb7ecfe46 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 15:05:47 +0800 Subject: [PATCH 06/71] [Feature][Kafka] Support multi-table source read #5992 --- .../kafka/source/ConsumerMetadata.java | 1 - .../kafka/source/KafkaSourceConfig.java | 38 +++++++++---------- .../src/test/resources/ogg/ogg_data.txt | 2 +- 3 files changed, 19 insertions(+), 22 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java index 6add204d413..8ce9dbb870a 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java @@ -37,7 +37,6 @@ public class ConsumerMetadata implements Serializable { private String topic; private boolean isPattern = false; private Properties properties; - private boolean commitOnCheckpoint = false; private String consumerGroup; private StartMode startMode = StartMode.GROUP_OFFSETS; private Map specificStartOffsets; diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index 4101f71d2b8..8115c333a1a 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -48,7 +48,6 @@ import org.apache.commons.collections4.MapUtils; import org.apache.kafka.common.TopicPartition; -import lombok.Data; import lombok.Getter; import java.io.Serializable; @@ -75,29 +74,25 @@ import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE_TIMESTAMP; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.TOPIC; -@Data public class KafkaSourceConfig implements Serializable { private static final long serialVersionUID = 1L; - @Getter private final Map mapMetadata; - - @Getter private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; - - @Getter private final long discoveryIntervalMillis; - @Getter private final String bootstrap; - @Getter private Properties properties; + @Getter private final Map mapMetadata; @Getter private final boolean commitOnCheckpoint; + @Getter private final Properties properties; + @Getter private final long discoveryIntervalMillis; + @Getter private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; public KafkaSourceConfig(ReadonlyConfig readonlyConfig) { this.bootstrap = readonlyConfig.get(BOOTSTRAP_SERVERS); this.mapMetadata = createMapConsumerMetadata(readonlyConfig); + this.commitOnCheckpoint = readonlyConfig.get(COMMIT_ON_CHECKPOINT); + this.properties = createKafkaProperties(readonlyConfig); this.discoveryIntervalMillis = readonlyConfig.get(KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS); this.messageFormatErrorHandleWay = readonlyConfig.get(MESSAGE_FORMAT_ERROR_HANDLE_WAY_OPTION); - this.commitOnCheckpoint = readonlyConfig.get(COMMIT_ON_CHECKPOINT); - this.properties = createKafkaProperties(readonlyConfig); } private Properties createKafkaProperties(ReadonlyConfig readonlyConfig) { @@ -119,11 +114,12 @@ private Map createMapConsumerMetadata( consumerMetadataList = Collections.singletonList(createConsumerMetadata(readonlyConfig)); } - Map tablePathConsumer = new HashMap<>(); - for (ConsumerMetadata consumerMetadata : consumerMetadataList) { - tablePathConsumer.put(TablePath.of(consumerMetadata.getTopic()), consumerMetadata); - } - return tablePathConsumer; + + return consumerMetadataList.stream() + .collect( + Collectors.toMap( + consumerMetadata -> TablePath.of(consumerMetadata.getTopic()), + consumerMetadata -> consumerMetadata)); } private ConsumerMetadata createConsumerMetadata(ReadonlyConfig readonlyConfig) { @@ -132,6 +128,12 @@ private ConsumerMetadata createConsumerMetadata(ReadonlyConfig readonlyConfig) { consumerMetadata.setPattern(readonlyConfig.get(PATTERN)); consumerMetadata.setConsumerGroup(readonlyConfig.get(CONSUMER_GROUP)); consumerMetadata.setProperties(new Properties()); + // Create a catalog + CatalogTable catalogTable = createCatalogTable(readonlyConfig); + consumerMetadata.setCatalogTable(catalogTable); + consumerMetadata.setDeserializationSchema( + createDeserializationSchema(catalogTable, readonlyConfig)); + // parse start mode readonlyConfig .getOptional(START_MODE) @@ -182,10 +184,6 @@ private ConsumerMetadata createConsumerMetadata(ReadonlyConfig readonlyConfig) { } }); - CatalogTable catalogTable = createCatalogTable(readonlyConfig); - consumerMetadata.setCatalogTable(catalogTable); - consumerMetadata.setDeserializationSchema( - createDeserializationSchema(catalogTable, readonlyConfig)); return consumerMetadata; } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/ogg/ogg_data.txt b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/ogg/ogg_data.txt index 2576267a5bf..0c470fd7c45 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/ogg/ogg_data.txt +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/ogg/ogg_data.txt @@ -13,4 +13,4 @@ {"table":"OGG.OGG_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.OGG_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.OGG_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.OGG_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"} +{"table":"OGG.OGG_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"} \ No newline at end of file From aa37aa12c9a224369d0e3232dd5b4a34743cc5fd Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 15:07:21 +0800 Subject: [PATCH 07/71] [Feature][Kafka] Support multi-table source read #5992 --- .../seatunnel/format/json/JsonDeserializationSchema.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 659dc5a049e..8d5b799c38c 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -101,9 +101,7 @@ public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOEx return null; } SeaTunnelRow seaTunnelRow = convertJsonNode(convertBytes(message)); - if (seaTunnelRow != null) { - seaTunnelRow.setTableId(tablePath.toString()); - } + seaTunnelRow.setTableId(tablePath.toString()); return seaTunnelRow; } From 798533e89469decfc5f4fe80d4c8458200d637b2 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 15:22:43 +0800 Subject: [PATCH 08/71] [Feature][Kafka] Support multi-table source read #5992 --- .../seatunnel/kafka/source/KafkaSource.java | 10 ++----- .../source/KafkaSourceSplitEnumerator.java | 29 +++---------------- 2 files changed, 6 insertions(+), 33 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java index 19f67511a7c..0c8cb4d496a 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java @@ -75,10 +75,7 @@ public SourceReader createReader( @Override public SourceSplitEnumerator createEnumerator( SourceSplitEnumerator.Context enumeratorContext) { - return new KafkaSourceSplitEnumerator( - kafkaSourceConfig, - enumeratorContext, - kafkaSourceConfig.getDiscoveryIntervalMillis()); + return new KafkaSourceSplitEnumerator(kafkaSourceConfig, enumeratorContext, null); } @Override @@ -86,10 +83,7 @@ public SourceSplitEnumerator restoreEnumerat SourceSplitEnumerator.Context enumeratorContext, KafkaSourceState checkpointState) { return new KafkaSourceSplitEnumerator( - kafkaSourceConfig, - enumeratorContext, - checkpointState, - kafkaSourceConfig.getDiscoveryIntervalMillis()); + kafkaSourceConfig, enumeratorContext, checkpointState); } @Override diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index f70da292c43..f0b773ef98d 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -70,37 +70,16 @@ public class KafkaSourceSplitEnumerator private Map topicMappingTablePathMap = new HashMap<>(); KafkaSourceSplitEnumerator( - KafkaSourceConfig kafkaSourceConfig, Context context) { + KafkaSourceConfig kafkaSourceConfig, + Context context, + KafkaSourceState sourceState) { this.kafkaSourceConfig = kafkaSourceConfig; this.tablePathMetadataMap = kafkaSourceConfig.getMapMetadata(); this.context = context; this.assignedSplit = new HashMap<>(); this.pendingSplit = new HashMap<>(); this.adminClient = initAdminClient(this.kafkaSourceConfig.getProperties()); - } - - KafkaSourceSplitEnumerator( - KafkaSourceConfig kafkaSourceConfig, - Context context, - KafkaSourceState sourceState) { - this(kafkaSourceConfig, context); - } - - KafkaSourceSplitEnumerator( - KafkaSourceConfig kafkaSourceConfig, - Context context, - long discoveryIntervalMillis) { - this(kafkaSourceConfig, context); - this.discoveryIntervalMillis = discoveryIntervalMillis; - } - - KafkaSourceSplitEnumerator( - KafkaSourceConfig kafkaSourceConfig, - Context context, - KafkaSourceState sourceState, - long discoveryIntervalMillis) { - this(kafkaSourceConfig, context, sourceState); - this.discoveryIntervalMillis = discoveryIntervalMillis; + this.discoveryIntervalMillis = kafkaSourceConfig.getDiscoveryIntervalMillis(); } @Override From 3996bec2d8abe11c568450338587984423a3af90 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 15:31:06 +0800 Subject: [PATCH 09/71] [Feature][Kafka] Support multi-table source read #5992 --- .../seatunnel/kafka/source/KafkaSourceSplitEnumerator.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index f0b773ef98d..b8c8c782035 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -59,7 +59,7 @@ public class KafkaSourceSplitEnumerator private final Map tablePathMetadataMap; private final Context context; - private long discoveryIntervalMillis; + private final long discoveryIntervalMillis; private final AdminClient adminClient; private final KafkaSourceConfig kafkaSourceConfig; private final Map pendingSplit; @@ -117,10 +117,10 @@ public void run() throws ExecutionException, InterruptedException { } private void setPartitionStartOffset() throws ExecutionException, InterruptedException { - Collection pendingSplitopicPartitions = pendingSplit.keySet(); + Set pendingTopicPartitions = pendingSplit.keySet(); Map topicPartitionOffsets = null; Map> tablePathPartitionMap = - pendingSplitopicPartitions.stream() + pendingTopicPartitions.stream() .collect( Collectors.groupingBy( tp -> topicMappingTablePathMap.get(tp.topic()), From 3735d229057038e81819d946b0880894122afc4d Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 15:34:05 +0800 Subject: [PATCH 10/71] [Feature][Kafka] Support multi-table source read #5992 --- .../seatunnel/kafka/source/KafkaSourceSplitEnumerator.java | 1 + 1 file changed, 1 insertion(+) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index b8c8c782035..c3357861593 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -119,6 +119,7 @@ public void run() throws ExecutionException, InterruptedException { private void setPartitionStartOffset() throws ExecutionException, InterruptedException { Set pendingTopicPartitions = pendingSplit.keySet(); Map topicPartitionOffsets = null; + // Set kafka TopicPartition based on the topicPath granularity Map> tablePathPartitionMap = pendingTopicPartitions.stream() .collect( From b40fdf796ac2320d80921f3156d989594040198e Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 15:36:55 +0800 Subject: [PATCH 11/71] [Feature][Kafka] Support multi-table source read #5992 --- .../seatunnel/kafka/source/KafkaSourceSplitEnumerator.java | 1 + 1 file changed, 1 insertion(+) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index c3357861593..478554f0141 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -127,6 +127,7 @@ private void setPartitionStartOffset() throws ExecutionException, InterruptedExc tp -> topicMappingTablePathMap.get(tp.topic()), Collectors.toSet())); for (TablePath tablePath : tablePathPartitionMap.keySet()) { + // Supports topic list fine-grained Settings for kafka consumer configurations ConsumerMetadata metadata = tablePathMetadataMap.get(tablePath); Set topicPartitions = tablePathPartitionMap.get(tablePath); switch (metadata.getStartMode()) { From 77e376401793807aba42e09fdb114df18121127c Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 15:39:30 +0800 Subject: [PATCH 12/71] [Feature][Kafka] Support multi-table source read #5992 --- .../seatunnel/kafka/source/KafkaSourceSplitEnumerator.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index 478554f0141..fc71fdfa729 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -286,10 +286,9 @@ private Set getTopicInfo() throws ExecutionException, Interrup return partitions.stream() .map( partition -> { - KafkaSourceSplit split = - new KafkaSourceSplit( - topicMappingTablePathMap.get(partition.topic()), - partition); + // Obtain the corresponding topic TablePath from kafka topic + TablePath tablePath = topicMappingTablePathMap.get(partition.topic()); + KafkaSourceSplit split = new KafkaSourceSplit(tablePath, partition); split.setEndOffset(latestOffsets.get(split.getTopicPartition())); return split; }) From 5c9ef495efaa9937ba9de13b1d24e1840ded2865 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 17:28:45 +0800 Subject: [PATCH 13/71] fix null --- .../pulsar/source/format/PulsarCanalDecorator.java | 3 +++ .../CompatibleKafkaConnectDeserializationSchema.java | 10 ++++------ .../format/json/JsonDeserializationSchema.java | 3 ++- .../format/text/TextDeserializationSchema.java | 3 +++ 4 files changed, 12 insertions(+), 7 deletions(-) diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java index c3c37d7ce31..0644f9df7ff 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java @@ -60,6 +60,9 @@ public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOEx @Override public void deserialize(byte[] message, Collector out, TablePath tablePath) throws IOException { + if (message == null || message.length == 0) { + return; + } JsonNode pulsarCanal = JsonUtils.parseObject(message); ArrayNode canalList = JsonUtils.parseArray(pulsarCanal.get(MESSAGE).asText()); Iterator canalIterator = canalList.elements(); diff --git a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java index 625f9be061e..990fc24701e 100644 --- a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java @@ -103,6 +103,9 @@ public void deserialize( ConsumerRecord msg, Collector out, TablePath tablePath) throws InvocationTargetException, IllegalAccessException { tryInitConverter(); + if (msg == null) { + return; + } SinkRecord record = convertToSinkRecord(msg); RowKind rowKind = RowKind.INSERT; JsonNode jsonNode = @@ -115,17 +118,12 @@ public void deserialize( for (int i = 0; i < arrayNode.size(); i++) { SeaTunnelRow row = convertJsonNode(arrayNode.get(i)); row.setRowKind(rowKind); - if (row != null) { - row.setTableId(tablePath.toString()); - } + row.setTableId(tablePath.toString()); out.collect(row); } } else { SeaTunnelRow row = convertJsonNode(payload); row.setRowKind(rowKind); - if (row != null) { - row.setTableId(tablePath.toString()); - } row.setTableId(tablePath.toString()); out.collect(row); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 8d5b799c38c..c0ea4e8f82a 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -97,10 +97,11 @@ private static boolean hasDecimalType(SeaTunnelDataType dataType) { @Override public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - if (message == null) { + if (message == null || message.length == 0) { return null; } SeaTunnelRow seaTunnelRow = convertJsonNode(convertBytes(message)); + assert seaTunnelRow != null; seaTunnelRow.setTableId(tablePath.toString()); return seaTunnelRow; } diff --git a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java index 1e089d2c7b6..1a2d2a99b86 100644 --- a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java @@ -114,6 +114,9 @@ public TextDeserializationSchema build() { @Override public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + if (message == null || message.length == 0) { + return null; + } String content = new String(message); Map splitsMap = splitLineBySeaTunnelRowType(content, seaTunnelRowType, 0); Object[] objects = new Object[seaTunnelRowType.getTotalFields()]; From 8b5923846952c7e311b4eaa4cf69292bc024598c Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 19:31:42 +0800 Subject: [PATCH 14/71] add e2e & doc --- docs/en/connector-v2/source/kafka.md | 63 ++++++++++++++++- .../seatunnel/kafka/config/Config.java | 3 +- .../source/KafkaSourceSplitEnumerator.java | 18 ++--- .../e2e/connector/kafka/KafkaFormatIT.java | 11 ++- .../kafka_multi_source_json_to_assert.conf | 69 +++++++++++++++++++ 5 files changed, 151 insertions(+), 13 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_json_to_assert.conf diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index 16b9c5420b3..cef9748033d 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -35,6 +35,7 @@ They can be downloaded via install-plugin.sh or from the Maven central repositor | Name | Type | Required | Default | Description | |-------------------------------------|-----------------------------------------------------------------------------|----------|--------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | topic | String | Yes | - | 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 comma like 'topic-1,topic-2'. | +| topic_list | Map | No | - | Topic list config You can configure only one `topic_list` and one `topic` at the same time | | bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | | pattern | Boolean | No | false | If `pattern` is set to `true`,the regular expression for a pattern of topic names to read from. All topics in clients with names that match the specified regular expression will be subscribed by the consumer. | | consumer.group | String | No | SeaTunnel-Consumer-Group | `Kafka consumer group id`, used to distinguish different consumer groups. | @@ -54,7 +55,7 @@ They can be downloaded via install-plugin.sh or from the Maven central repositor ### Simple -> This example reads the data of kafka's topic_1, topic_2, topic_3 and prints it to the client.And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in Install SeaTunnel to install and deploy SeaTunnel. And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in [Install SeaTunnel](../../start-v2/locally/deployment.md) to install and deploy SeaTunnel. And then follow the instructions in [Quick Start With SeaTunnel Engine](../../start-v2/locally/quick-start-seatunnel-engine.md) to run this job. +> Currently, multiple kafka source reads are supported using the Zeta engine, but note that you can only configure one instance of `bootstrap.servers`, and only one in the `topic_list` and `topic` ``parameters ```hocon # Defining the runtime environment @@ -160,3 +161,63 @@ source { } ``` +### Multiple Kafka Source + +> This example reads the data of kafka's topic_1, topic_2, topic_3 and prints it to the client.And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in Install SeaTunnel to install and deploy SeaTunnel. And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in [Install SeaTunnel](../../start-v2/locally/deployment.md) to install and deploy SeaTunnel. And then follow the instructions in [Quick Start With SeaTunnel Engine](../../start-v2/locally/quick-start-seatunnel-engine.md) to run this job. + +```hocon +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + Kafka { + bootstrap.servers = "kafka_e2e:9092" + topic_list = [ + { + topic = "test-ogg-source" + consumer.group = "ogg_multi_group" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = ogg_json + }, { + topic = "test-cdc_mds" + consumer.group = "canal_multi_group" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = canal_json + }, { + topic = ".*debezium*." + pattern = "true" + consumer.group = "debezium_multi_group" + start_mode = earliest + } + ] + } +} + +sink { + Assert { + rules { + // The current table name is the same as the topic name in order to maintain fewer configuration parameters + table-names = ["test-cdc_mds", "test-ogg-source",".*debezium*."] + } + } +} +``` + diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java index 998ea060ff0..5d74c3d2dfe 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java @@ -181,5 +181,6 @@ public class Config { Options.key("topic_list") .type(new TypeReference>>() {}) .noDefaultValue() - .withDescription("topic list config"); + .withDescription( + "Topic list config You can configure only one `topic_list` and one `topic` at the same time"); } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index fc71fdfa729..128a999b87b 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -67,7 +67,7 @@ public class KafkaSourceSplitEnumerator private ScheduledExecutorService executor; private ScheduledFuture scheduledFuture; - private Map topicMappingTablePathMap = new HashMap<>(); + private final Map topicMappingTablePathMap = new HashMap<>(); KafkaSourceSplitEnumerator( KafkaSourceConfig kafkaSourceConfig, @@ -118,7 +118,7 @@ public void run() throws ExecutionException, InterruptedException { private void setPartitionStartOffset() throws ExecutionException, InterruptedException { Set pendingTopicPartitions = pendingSplit.keySet(); - Map topicPartitionOffsets = null; + Map topicPartitionOffsets = new HashMap<>(); // Set kafka TopicPartition based on the topicPath granularity Map> tablePathPartitionMap = pendingTopicPartitions.stream() @@ -132,22 +132,24 @@ private void setPartitionStartOffset() throws ExecutionException, InterruptedExc Set topicPartitions = tablePathPartitionMap.get(tablePath); switch (metadata.getStartMode()) { case EARLIEST: - topicPartitionOffsets = listOffsets(topicPartitions, OffsetSpec.earliest()); + topicPartitionOffsets.putAll( + listOffsets(topicPartitions, OffsetSpec.earliest())); break; case GROUP_OFFSETS: - topicPartitionOffsets = listConsumerGroupOffsets(topicPartitions, metadata); + topicPartitionOffsets.putAll( + listConsumerGroupOffsets(topicPartitions, metadata)); break; case LATEST: - topicPartitionOffsets = listOffsets(topicPartitions, OffsetSpec.latest()); + topicPartitionOffsets.putAll(listOffsets(topicPartitions, OffsetSpec.latest())); break; case TIMESTAMP: - topicPartitionOffsets = + topicPartitionOffsets.putAll( listOffsets( topicPartitions, - OffsetSpec.forTimestamp(metadata.getStartOffsetsTimestamp())); + OffsetSpec.forTimestamp(metadata.getStartOffsetsTimestamp()))); break; case SPECIFIC_OFFSETS: - topicPartitionOffsets = metadata.getSpecificStartOffsets(); + topicPartitionOffsets.putAll(metadata.getSpecificStartOffsets()); break; default: break; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java index a3c2fb52f73..a71533081fc 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java @@ -204,12 +204,17 @@ public void startUp() throws ClassNotFoundException, InterruptedException, IOExc Thread.sleep(20 * 1000); } + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "The multi-catalog does not currently support the Spark Flink engine") @TestTemplate - public void testFormatCanalCheck(TestContainer container) + public void testMultiFormatCheck(TestContainer container) throws IOException, InterruptedException { - LOG.info("====================== Check Canal======================"); + LOG.info( + "====================== Multi Source Format Canal and Ogg Check ======================"); Container.ExecResult execCanalResultKafka = - container.executeJob("/canalFormatIT/kafka_source_canal_to_kafka.conf"); + container.executeJob("/multiFormatIT/kafka_multi_source_json_to_assert.conf"); Assertions.assertEquals( 0, execCanalResultKafka.getExitCode(), execCanalResultKafka.getStderr()); Container.ExecResult execCanalResultToPgSql = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_json_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_json_to_assert.conf new file mode 100644 index 00000000000..4bea3f4d0b4 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_json_to_assert.conf @@ -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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + Kafka { + bootstrap.servers = "kafka_e2e:9092" + topic_list = [ + { + topic = "^test-ogg-sou.*" + pattern = "true" + consumer.group = "ogg_multi_group" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = ogg_json + }, { + topic = "test-cdc_mds" + consumer.group = "canal_multi_group" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = canal_json + } + ] + } +} + +sink { + Assert { + rules { + // The current table name is the same as the topic name in order to maintain fewer configuration parameters + table-names = ["^test-ogg-sou.*","test-cdc_mds"] + } + } +} From c2bbc3b272ba38926e63ca65a2920ea99795bbd4 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 19:33:16 +0800 Subject: [PATCH 15/71] add e2e & doc --- .../seatunnel/kafka/source/KafkaSourceSplitEnumerator.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index 128a999b87b..f89fd8bb798 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -267,8 +267,7 @@ private Set getTopicInfo() throws ExecutionException, Interrup } else { currentPathTopics.addAll(Arrays.asList(metadata.getTopic().split(","))); } - currentPathTopics.stream() - .forEach(topic -> topicMappingTablePathMap.put(topic, tablePath)); + currentPathTopics.forEach(topic -> topicMappingTablePathMap.put(topic, tablePath)); topics.addAll(currentPathTopics); } From 73f1705908f65585a4c32133f0238de7365ad09e Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 19:38:32 +0800 Subject: [PATCH 16/71] add e2e & doc --- docs/en/connector-v2/source/kafka.md | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index cef9748033d..efa04069a1e 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -55,7 +55,7 @@ They can be downloaded via install-plugin.sh or from the Maven central repositor ### Simple -> Currently, multiple kafka source reads are supported using the Zeta engine, but note that you can only configure one instance of `bootstrap.servers`, and only one in the `topic_list` and `topic` ``parameters +> This example reads the data of kafka's topic_1, topic_2, topic_3 and prints it to the client.And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in Install SeaTunnel to install and deploy SeaTunnel. And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in [Install SeaTunnel](../../start-v2/locally/deployment.md) to install and deploy SeaTunnel. And then follow the instructions in [Quick Start With SeaTunnel Engine](../../start-v2/locally/quick-start-seatunnel-engine.md) to run this job. ```hocon # Defining the runtime environment @@ -163,9 +163,10 @@ source { ### Multiple Kafka Source -> This example reads the data of kafka's topic_1, topic_2, topic_3 and prints it to the client.And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in Install SeaTunnel to install and deploy SeaTunnel. And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in [Install SeaTunnel](../../start-v2/locally/deployment.md) to install and deploy SeaTunnel. And then follow the instructions in [Quick Start With SeaTunnel Engine](../../start-v2/locally/quick-start-seatunnel-engine.md) to run this job. +> Currently, multiple kafka source reads are supported using the Zeta engine, but note that you can only configure one instance of `bootstrap.servers`, and only one in the `topic_list` and `topic` ``parameters ```hocon + env { execution.parallelism = 1 job.mode = "BATCH" @@ -176,7 +177,8 @@ source { bootstrap.servers = "kafka_e2e:9092" topic_list = [ { - topic = "test-ogg-source" + topic = "^test-ogg-sou.*" + pattern = "true" consumer.group = "ogg_multi_group" start_mode = earliest schema = { @@ -201,11 +203,6 @@ source { } }, format = canal_json - }, { - topic = ".*debezium*." - pattern = "true" - consumer.group = "debezium_multi_group" - start_mode = earliest } ] } @@ -215,7 +212,7 @@ sink { Assert { rules { // The current table name is the same as the topic name in order to maintain fewer configuration parameters - table-names = ["test-cdc_mds", "test-ogg-source",".*debezium*."] + table-names = ["^test-ogg-sou.*","test-cdc_mds"] } } } From 85f98b11323221b5ed3547256428c5c2c3177074 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Dec 2023 22:28:50 +0800 Subject: [PATCH 17/71] add e2e & doc --- .../seatunnel/e2e/connector/kafka/KafkaFormatIT.java | 8 +------- ...assert.conf => kafka_multi_source_to_assert.conf.conf} | 0 2 files changed, 1 insertion(+), 7 deletions(-) rename seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/{kafka_multi_source_json_to_assert.conf => kafka_multi_source_to_assert.conf.conf} (100%) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java index a71533081fc..988164f1873 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java @@ -214,15 +214,9 @@ public void testMultiFormatCheck(TestContainer container) LOG.info( "====================== Multi Source Format Canal and Ogg Check ======================"); Container.ExecResult execCanalResultKafka = - container.executeJob("/multiFormatIT/kafka_multi_source_json_to_assert.conf"); + container.executeJob("/multiFormatIT/kafka_multi_source_to_assert.conf"); Assertions.assertEquals( 0, execCanalResultKafka.getExitCode(), execCanalResultKafka.getStderr()); - Container.ExecResult execCanalResultToPgSql = - container.executeJob("/canalFormatIT/kafka_source_canal_cdc_to_pgsql.conf"); - Assertions.assertEquals( - 0, execCanalResultToPgSql.getExitCode(), execCanalResultToPgSql.getStderr()); - // Check Canal - checkCanalFormat(); } @TestTemplate diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_json_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf.conf similarity index 100% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_json_to_assert.conf rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf.conf From 2ebcf8784e113aee872381123aa9a820e5d5b504 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Wed, 13 Dec 2023 13:17:31 +0800 Subject: [PATCH 18/71] fix --- .../rest/RestHttpPostCommandProcessor.java | 52 ------------------- 1 file changed, 52 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java index 2b1d71cea72..9aa1e8c2e74 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java @@ -205,58 +205,6 @@ private JsonNode requestHandle(HttpPostCommand httpPostCommand) { return requestBodyJsonNode; } - private Long submitJob( - SeaTunnelServer seaTunnelServer, - JobImmutableInformation jobImmutableInformation, - JobConfig jobConfig, - RestJobExecutionEnvironment restJobExecutionEnvironment) { - CoordinatorService coordinatorService = seaTunnelServer.getCoordinatorService(); - Data data = - textCommandService - .getNode() - .nodeEngine - .getSerializationService() - .toData(jobImmutableInformation); - PassiveCompletableFuture voidPassiveCompletableFuture = - coordinatorService.submitJob( - Long.parseLong(jobConfig.getJobContext().getJobId()), data); - voidPassiveCompletableFuture.join(); - - return restJobExecutionEnvironment.getJobId(); - SeaTunnelServer seaTunnelServer = - (SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME); - if (!seaTunnelServer.isMasterNode()) { - for (HazelcastInstance hazelcastInstance : Hazelcast.getAllHazelcastInstances()) { - seaTunnelServer = - (SeaTunnelServer) - ((HazelcastInstanceProxy) hazelcastInstance) - .getOriginal() - .node - .getNodeExtension() - .createExtensionServices() - .get(Constant.SEATUNNEL_SERVICE_NAME); - - if (seaTunnelServer.isMasterNode()) { - return seaTunnelServer; - } - } - } - return seaTunnelServer; - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(); - Long jobId = - submitJob( - seaTunnelServer, - jobImmutableInformation, - jobConfig, - restJobExecutionEnvironment); - - this.prepareResponse( - httpPostCommand, - new JsonObject() - .add(RestConstant.JOB_ID, jobId) - .add(RestConstant.JOB_NAME, requestParams.get(RestConstant.JOB_NAME))); - } - private Long submitJob( SeaTunnelServer seaTunnelServer, JobImmutableInformation jobImmutableInformation, From fcdd7751d3cc1a8f921d48f08234a11835c23ea2 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Wed, 13 Dec 2023 15:54:12 +0800 Subject: [PATCH 19/71] add doc desc --- docs/en/connector-v2/source/kafka.md | 2 +- ...ce_to_assert.conf.conf => kafka_multi_source_to_assert.conf} | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/{kafka_multi_source_to_assert.conf.conf => kafka_multi_source_to_assert.conf} (100%) diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index efa04069a1e..d41f0893ce6 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -163,7 +163,7 @@ source { ### Multiple Kafka Source -> Currently, multiple kafka source reads are supported using the Zeta engine, but note that you can only configure one instance of `bootstrap.servers`, and only one in the `topic_list` and `topic` ``parameters +> Currently, multiple kafka source reads are supported using the Zeta engine, but note that you can only configure one instance of `bootstrap.servers`, and only one in the `topic_list` and `topic` parameters,Currently, when you use multiple topics in `topic_list`, you are still free to set parameters for each `topic` ```hocon diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf similarity index 100% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf.conf rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf From fd26df737d42279de2ad2424b9f2b51aa8bd1a90 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Wed, 13 Dec 2023 15:56:44 +0800 Subject: [PATCH 20/71] add doc desc --- docs/en/connector-v2/source/kafka.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index d41f0893ce6..b15b29db00e 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -205,6 +205,11 @@ source { format = canal_json } ] + # Each topic in the `topic_list` shares the config + kafka.config = { + client.id = client_1 + max.poll.records = 50000 + } } } From 86f9d163a273fe998ade256409f28ba108069155 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Thu, 14 Dec 2023 13:48:29 +0800 Subject: [PATCH 21/71] add doc desc --- docs/en/connector-v2/source/kafka.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index eba5c59658e..7755f666c1c 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -32,11 +32,10 @@ They can be downloaded via install-plugin.sh or from the Maven central repositor ## Source Options - | Name | Type | Required | Default | Description | |-------------------------------------|-----------------------------------------------------------------------------|----------|--------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | topic | String | Yes | - | 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 comma like 'topic-1,topic-2'. | -| topic_list | Map | No | - | Topic list config You can configure only one `topic_list` and one `topic` at the same time | +| topic_list | Map | No | - | Topic list config You can configure only one `topic_list` and one `topic` at the same time | | bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | | pattern | Boolean | No | false | If `pattern` is set to `true`,the regular expression for a pattern of topic names to read from. All topics in clients with names that match the specified regular expression will be subscribed by the consumer. | | consumer.group | String | No | SeaTunnel-Consumer-Group | `Kafka consumer group id`, used to distinguish different consumer groups. | @@ -51,6 +50,7 @@ They can be downloaded via install-plugin.sh or from the Maven central repositor | start_mode.timestamp | Long | No | - | The time required for consumption mode to be "timestamp". | | partition-discovery.interval-millis | Long | No | -1 | The interval for dynamically discovering topics and partitions. | | common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details | + ## Task Example ### Simple @@ -82,7 +82,7 @@ source { auto.offset.reset = "earliest" enable.auto.commit = "false" } - } + } } sink { Console {} From 9763a805786774495beb68f73ce678a1817d9aaa Mon Sep 17 00:00:00 2001 From: zhilinli Date: Thu, 14 Dec 2023 16:47:33 +0800 Subject: [PATCH 22/71] add doc desc --- .../e2e/connector/kafka/KafkaFormatIT.java | 110 +++++++++++++++++- .../avro/AvroDeserializationSchema.java | 3 +- .../avro/AvroSerializationSchemaTest.java | 15 +-- .../json/JsonRowDataSerDeSchemaTest.java | 26 +---- .../debezium/DebeziumJsonSerDeSchemaTest.java | 29 ++--- 5 files changed, 139 insertions(+), 44 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java index 96173d93776..a89ec079721 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java @@ -25,7 +25,10 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.psql.PostgresJdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.utils.JdbcUtils; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; @@ -63,10 +66,13 @@ import java.io.InputStreamReader; import java.math.BigDecimal; import java.sql.Connection; +import java.sql.Date; import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.sql.Time; +import java.sql.Timestamp; import java.time.Duration; import java.time.LocalDate; import java.time.LocalDateTime; @@ -76,7 +82,9 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -101,7 +109,6 @@ public class KafkaFormatIT extends TestSuiteBase implements TestResource { // ---------------------------Canal Format Parameter--------------------------------------- private static final String CANAL_KAFKA_SINK_TOPIC = "test-canal-sink"; - private static final String CANAL_MYSQL_DATABASE = "canal"; private static final String CANAL_DATA_PATH = "/canal/canal_data.txt"; private static final String CANAL_KAFKA_SOURCE_TOPIC = "test-cdc_mds"; @@ -341,6 +348,22 @@ public void testMultiFormatCheck(TestContainer container) 0, execCanalResultKafka.getExitCode(), execCanalResultKafka.getStderr()); } + @TestTemplate + public void testFormatCanalCheck(TestContainer container) + throws IOException, InterruptedException { + LOG.info("====================== Check Canal======================"); + Container.ExecResult execCanalResultKafka = + container.executeJob("/canalFormatIT/kafka_source_canal_to_kafka.conf"); + Assertions.assertEquals( + 0, execCanalResultKafka.getExitCode(), execCanalResultKafka.getStderr()); + Container.ExecResult execCanalResultToPgSql = + container.executeJob("/canalFormatIT/kafka_source_canal_cdc_to_pgsql.conf"); + Assertions.assertEquals( + 0, execCanalResultToPgSql.getExitCode(), execCanalResultToPgSql.getStderr()); + // Check Canal + checkCanalFormat(); + } + @TestTemplate public void testFormatOggCheck(TestContainer container) throws IOException, InterruptedException { @@ -837,8 +860,7 @@ private List> getPostgreSinkTableList(String tableName) { while (resultSet.next()) { SeaTunnelRow row = - postgresJdbcRowConverter.toInternal( - resultSet, sinkTableRowTypes.get(tableName)); + testPgToInternal(resultSet, sinkTableRowTypes.get(tableName)); actual.add(Arrays.asList(row.getFields())); } } @@ -853,6 +875,88 @@ private List> getPostgreSinkTableList(String tableName) { return actual; } + public SeaTunnelRow testPgToInternal(ResultSet rs, SeaTunnelRowType typeInfo) + throws SQLException { + String PG_GEOMETRY = "GEOMETRY"; + String PG_GEOGRAPHY = "GEOGRAPHY"; + + Object[] fields = new Object[typeInfo.getTotalFields()]; + for (int fieldIndex = 0; fieldIndex < typeInfo.getTotalFields(); fieldIndex++) { + SeaTunnelDataType seaTunnelDataType = typeInfo.getFieldType(fieldIndex); + int resultSetIndex = fieldIndex + 1; + String metaDataColumnType = + rs.getMetaData().getColumnTypeName(resultSetIndex).toUpperCase(Locale.ROOT); + switch (seaTunnelDataType.getSqlType()) { + case STRING: + if (metaDataColumnType.equals(PG_GEOMETRY) + || metaDataColumnType.equals(PG_GEOGRAPHY)) { + fields[fieldIndex] = + rs.getObject(resultSetIndex) == null + ? null + : rs.getObject(resultSetIndex).toString(); + } else { + fields[fieldIndex] = JdbcUtils.getString(rs, resultSetIndex); + } + break; + case BOOLEAN: + fields[fieldIndex] = JdbcUtils.getBoolean(rs, resultSetIndex); + break; + case TINYINT: + fields[fieldIndex] = JdbcUtils.getByte(rs, resultSetIndex); + break; + case SMALLINT: + fields[fieldIndex] = JdbcUtils.getShort(rs, resultSetIndex); + break; + case INT: + fields[fieldIndex] = JdbcUtils.getInt(rs, resultSetIndex); + break; + case BIGINT: + fields[fieldIndex] = JdbcUtils.getLong(rs, resultSetIndex); + break; + case FLOAT: + fields[fieldIndex] = JdbcUtils.getFloat(rs, resultSetIndex); + break; + case DOUBLE: + fields[fieldIndex] = JdbcUtils.getDouble(rs, resultSetIndex); + break; + case DECIMAL: + fields[fieldIndex] = JdbcUtils.getBigDecimal(rs, resultSetIndex); + break; + case DATE: + Date sqlDate = JdbcUtils.getDate(rs, resultSetIndex); + fields[fieldIndex] = + Optional.ofNullable(sqlDate).map(e -> e.toLocalDate()).orElse(null); + break; + case TIME: + Time sqlTime = JdbcUtils.getTime(rs, resultSetIndex); + fields[fieldIndex] = + Optional.ofNullable(sqlTime).map(e -> e.toLocalTime()).orElse(null); + break; + case TIMESTAMP: + Timestamp sqlTimestamp = JdbcUtils.getTimestamp(rs, resultSetIndex); + fields[fieldIndex] = + Optional.ofNullable(sqlTimestamp) + .map(e -> e.toLocalDateTime()) + .orElse(null); + break; + case BYTES: + fields[fieldIndex] = JdbcUtils.getBytes(rs, resultSetIndex); + break; + case NULL: + fields[fieldIndex] = null; + break; + case MAP: + case ARRAY: + case ROW: + default: + throw new JdbcConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + "Unexpected value: " + seaTunnelDataType); + } + } + return new SeaTunnelRow(fields); + } + @Override public void tearDown() { if (KAFKA_CONTAINER != null) { diff --git a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java index b682a8e6431..1e9be047e60 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.format.avro; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -41,7 +42,7 @@ public AvroDeserializationSchema(SeaTunnelRowType rowType) { } @Override - public SeaTunnelRow deserialize(byte[] message) throws IOException { + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(message, null); GenericRecord record = this.converter.getReader().read(null, decoder); return converter.converter(record, rowType); diff --git a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java index 5f505e1ba6b..72a5f97a157 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.format.avro; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -38,13 +39,14 @@ class AvroSerializationSchemaTest { - private LocalDate localDate = LocalDate.of(2023, 1, 1); - private BigDecimal bigDecimal = new BigDecimal("61592600349703735722.724745739637773662"); - private LocalDateTime localDateTime = LocalDateTime.of(2023, 1, 1, 6, 30, 40); + private static final LocalDate localDate = LocalDate.of(2023, 1, 1); + private static final BigDecimal bigDecimal = + new BigDecimal("61592600349703735722.724745739637773662"); + private static final LocalDateTime localDateTime = LocalDateTime.of(2023, 1, 1, 6, 30, 40); private SeaTunnelRow buildSeaTunnelRow() { SeaTunnelRow subSeaTunnelRow = new SeaTunnelRow(14); - Map map = new HashMap(); + Map map = new HashMap<>(); map.put("k1", "v1"); map.put("k2", "v2"); String[] strArray = new String[] {"l1", "l2"}; @@ -116,7 +118,6 @@ private SeaTunnelRowType buildSeaTunnelRowType() { new DecimalType(38, 18), LocalTimeType.LOCAL_DATE_TIME_TYPE }; - SeaTunnelRowType subRow = new SeaTunnelRowType(subField, subFieldTypes); String[] fieldNames = { "c_map", @@ -150,7 +151,7 @@ private SeaTunnelRowType buildSeaTunnelRowType() { LocalTimeType.LOCAL_DATE_TYPE, new DecimalType(38, 18), LocalTimeType.LOCAL_DATE_TIME_TYPE, - subRow + new SeaTunnelRowType(subField, subFieldTypes) }; SeaTunnelRowType rowType = new SeaTunnelRowType(fieldNames, fieldTypes); return rowType; @@ -163,7 +164,7 @@ public void testSerialization() throws IOException { AvroSerializationSchema serializationSchema = new AvroSerializationSchema(rowType); byte[] serialize = serializationSchema.serialize(seaTunnelRow); AvroDeserializationSchema deserializationSchema = new AvroDeserializationSchema(rowType); - SeaTunnelRow deserialize = deserializationSchema.deserialize(serialize); + SeaTunnelRow deserialize = deserializationSchema.deserialize(serialize, TablePath.of("")); String[] strArray1 = (String[]) seaTunnelRow.getField(1); String[] strArray2 = (String[]) deserialize.getField(1); Assertions.assertArrayEquals(strArray1, strArray2); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java index 54125110b79..c972607f5de 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java @@ -277,7 +277,7 @@ public void testDeserializationMissingNode() throws Exception { JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(true, false, schema); SeaTunnelRow rowData = deserializationSchema.deserialize("".getBytes(), TablePath.of("")); - assertEquals(null, rowData); + assertNull(rowData); } @Test @@ -296,8 +296,7 @@ public void testDeserializationPassMissingField() throws Exception { final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, false, schema); SeaTunnelRow expected = new SeaTunnelRow(1); - SeaTunnelRow actual = deserializationSchema.deserialize(serializedJson, TablePath.of("")); - SeaTunnelRow actual = deser.deserialize(serializedJson); + SeaTunnelRow actual = deser.deserialize(serializedJson, TablePath.of("")); assertEquals(expected, actual); } @@ -322,13 +321,13 @@ public void testDeserializationMissingField() throws Exception { assertThrows( SeaTunnelRuntimeException.class, () -> { - deser.deserialize(serializedJson); + deser.deserialize(serializedJson, TablePath.of("")); }, "expecting exception message: " + expected.getMessage()); assertEquals(actual.getMessage(), expected.getMessage()); SeaTunnelRuntimeException expectedCause = - CommonError.jsonOperationError("Common", "Field $.name in " + root.toString()); + CommonError.jsonOperationError("Common", "Field $.name in " + root); Throwable cause = actual.getCause(); assertEquals(cause.getClass(), expectedCause.getClass()); assertEquals(cause.getMessage(), expectedCause.getMessage()); @@ -349,25 +348,12 @@ public void testDeserializationIgnoreParseError() throws Exception { // ignore on parse error final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, true, schema); - assertEquals(expected, deser.deserialize(serializedJson)); + assertEquals(expected, deser.deserialize(serializedJson, TablePath.of(""))); } @Test public void testDeserializationFailOnMissingFieldIgnoreParseError() throws Exception { String errorMessage = - "ErrorCode:[COMMON-02], ErrorDescription:[Json covert/parse operation failed] - Failed to deserialize JSON '{\"id\":123123123}'."; - try { - deserializationSchema.deserialize(serializedJson, TablePath.of("")); - fail("expecting exception message: " + errorMessage); - } catch (Throwable t) { - assertEquals(errorMessage, t.getMessage()); - } - - // ignore on parse error - deserializationSchema = new JsonDeserializationSchema(false, true, schema); - assertEquals(expected, deserializationSchema.deserialize(serializedJson, TablePath.of(""))); - - errorMessage = "ErrorCode:[COMMON-06], ErrorDescription:[Illegal argument] - JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."; SeaTunnelJsonFormatException actual = @@ -403,7 +389,7 @@ public void testDeserializationNoJson() throws Exception { assertThrows( SeaTunnelRuntimeException.class, () -> { - deser.deserialize(noJson.getBytes()); + deser.deserialize(noJson.getBytes(), TablePath.of("")); }, "expecting exception message: " + expected.getMessage()); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index 05db7f50a23..7c5b6789d16 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -48,6 +48,7 @@ public class DebeziumJsonSerDeSchemaTest { private static final String FORMAT = "Debezium"; + private static final String FILE_DATA_NAME = "debezium-data.txt"; private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = new SeaTunnelRowType( @@ -67,11 +68,11 @@ void testNullRowMessages() throws Exception { @Test public void testSerializationAndSchemaExcludeDeserialization() throws Exception { - testSerializationDeserialization("debezium-data.txt", false); + testSerializationDeserialization(false); } @Test - public void testDeserializeNoJson() throws Exception { + public void testDeserializeNoJson() { final DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); final SimpleCollector collector = new SimpleCollector(); @@ -83,13 +84,14 @@ public void testDeserializeNoJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize(noJsonMsg.getBytes(), collector); + deserializationSchema.deserialize( + noJsonMsg.getBytes(), collector, TablePath.of("")); }); assertEquals(cause.getMessage(), expected.getMessage()); } @Test - public void testDeserializeEmptyJson() throws Exception { + public void testDeserializeEmptyJson() { final DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); final SimpleCollector collector = new SimpleCollector(); @@ -99,13 +101,14 @@ public void testDeserializeEmptyJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize(emptyMsg.getBytes(), collector); + deserializationSchema.deserialize( + emptyMsg.getBytes(), collector, TablePath.of("")); }); assertEquals(cause.getMessage(), expected.getMessage()); } @Test - public void testDeserializeNoDataJson() throws Exception { + public void testDeserializeNoDataJson() { final DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); final SimpleCollector collector = new SimpleCollector(); @@ -115,7 +118,8 @@ public void testDeserializeNoDataJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize(noDataMsg.getBytes(), collector); + deserializationSchema.deserialize( + noDataMsg.getBytes(), collector, TablePath.of("")); }); assertEquals(cause.getMessage(), expected.getMessage()); @@ -131,7 +135,7 @@ public void testDeserializeNoDataJson() throws Exception { } @Test - public void testDeserializeUnknownOperationTypeJson() throws Exception { + public void testDeserializeUnknownOperationTypeJson() { final DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); final SimpleCollector collector = new SimpleCollector(); @@ -147,7 +151,7 @@ public void testDeserializeUnknownOperationTypeJson() throws Exception { expected.getClass(), () -> { deserializationSchema.deserialize( - unknownOperationMsg.getBytes(), collector); + unknownOperationMsg.getBytes(), collector, TablePath.of("")); }); assertEquals(cause.getMessage(), expected.getMessage()); @@ -158,9 +162,8 @@ public void testDeserializeUnknownOperationTypeJson() throws Exception { String.format("Unknown operation type '%s'.", unknownType)); } - private void testSerializationDeserialization(String resourceFile, boolean schemaInclude) - throws Exception { - List lines = readLines(resourceFile); + private void testSerializationDeserialization(boolean schemaInclude) throws Exception { + List lines = readLines(FILE_DATA_NAME); DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, true, schemaInclude); @@ -242,7 +245,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(SeaTunnelRow record) { From fd9b07763b28ad5c75a0c97ef06c0e80d739acea Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 15 Dec 2023 13:30:45 +0800 Subject: [PATCH 23/71] add doc desc --- .../apache/seatunnel/format/avro/AvroToRowConverter.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroToRowConverter.java b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroToRowConverter.java index 989087ee570..edf748c2336 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroToRowConverter.java +++ b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroToRowConverter.java @@ -157,6 +157,12 @@ protected static Object convertArray(List val, SeaTunnelDataType data integers[i] = (Integer) val.get(i); } return integers; + case TINYINT: + Byte[] tinyintBytes = new Byte[length]; + for (int i = 0; i < tinyintBytes.length; i++) { + tinyintBytes[i] = (Byte) val.get(i); + } + return tinyintBytes; case BIGINT: Long[] longs = new Long[length]; for (int i = 0; i < longs.length; i++) { From 12c64179497407d2b1d470a7fba4d06f3731ae4e Mon Sep 17 00:00:00 2001 From: zhilinli Date: Sun, 17 Dec 2023 09:57:53 +0800 Subject: [PATCH 24/71] fix --- .../seatunnel/e2e/connector/kafka/KafkaFormatIT.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java index 0f51bfb75b7..d51ce35b80d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java @@ -27,10 +27,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; -import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.psql.PostgresJdbcRowConverter; -import org.apache.seatunnel.connectors.seatunnel.jdbc.utils.JdbcUtils; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; @@ -68,13 +65,10 @@ import java.io.InputStreamReader; import java.math.BigDecimal; import java.sql.Connection; -import java.sql.Date; import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; -import java.sql.Time; -import java.sql.Timestamp; import java.time.Duration; import java.time.LocalDate; import java.time.LocalDateTime; @@ -84,9 +78,7 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; -import java.util.Locale; import java.util.Map; -import java.util.Optional; import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; From 066d6ddac7f59cd6ce5610b08c9a3da8c90a4851 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Sun, 17 Dec 2023 12:19:27 +0800 Subject: [PATCH 25/71] fix --- .../apache/seatunnel/format/avro/AvroToRowConverter.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroToRowConverter.java b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroToRowConverter.java index edf748c2336..989087ee570 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroToRowConverter.java +++ b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroToRowConverter.java @@ -157,12 +157,6 @@ protected static Object convertArray(List val, SeaTunnelDataType data integers[i] = (Integer) val.get(i); } return integers; - case TINYINT: - Byte[] tinyintBytes = new Byte[length]; - for (int i = 0; i < tinyintBytes.length; i++) { - tinyintBytes[i] = (Byte) val.get(i); - } - return tinyintBytes; case BIGINT: Long[] longs = new Long[length]; for (int i = 0; i < longs.length; i++) { From 5f9026f1c34f31d1f24d5527cbdb20c7573e3eb2 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 5 Jan 2024 21:00:21 +0800 Subject: [PATCH 26/71] fix ci error --- .../google/sheets/deserialize/GoogleSheetsDeserializer.java | 3 ++- .../seatunnel/format/avro/AvroSerializationSchemaTest.java | 5 ++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java index 76cce870c72..e80e8a1018c 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.CommonError; import org.apache.seatunnel.connectors.seatunnel.google.sheets.exception.GoogleSheetsError; @@ -53,7 +54,7 @@ public SeaTunnelRow deserializeRow(List row) { try { String rowStr = objectMapper.writeValueAsString(map); - return deserializationSchema.deserialize(rowStr.getBytes()); + return deserializationSchema.deserialize(rowStr.getBytes(), TablePath.of("")); } catch (JsonProcessingException e) { throw CommonError.jsonOperationError("GoogleSheets", map.toString(), e); } catch (IOException e) { diff --git a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java index 1939e4c3883..1da662c8fda 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java @@ -153,8 +153,7 @@ private SeaTunnelRowType buildSeaTunnelRowType() { LocalTimeType.LOCAL_DATE_TIME_TYPE, new SeaTunnelRowType(subField, subFieldTypes) }; - SeaTunnelRowType rowType = new SeaTunnelRowType(fieldNames, fieldTypes); - return rowType; + return new SeaTunnelRowType(fieldNames, fieldTypes); } @Test @@ -164,7 +163,7 @@ public void testSerialization() throws IOException { AvroSerializationSchema serializationSchema = new AvroSerializationSchema(rowType); byte[] bytes = serializationSchema.serialize(seaTunnelRow); AvroDeserializationSchema deserializationSchema = new AvroDeserializationSchema(rowType); - SeaTunnelRow deserialize = deserializationSchema.deserialize(bytes); + SeaTunnelRow deserialize = deserializationSchema.deserialize(bytes, TablePath.of("")); String[] strArray1 = (String[]) seaTunnelRow.getField(1); String[] strArray2 = (String[]) deserialize.getField(1); Assertions.assertArrayEquals(strArray1, strArray2); From 59e7bb0eff151854ef447c9ec38a9c489874ff5d Mon Sep 17 00:00:00 2001 From: zhilinli Date: Sat, 6 Jan 2024 09:56:26 +0800 Subject: [PATCH 27/71] fix ci error --- .../org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index d86aa891865..44d082b6f30 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -377,7 +378,8 @@ public void testFakeSourceToKafkaAvroFormat(TestContainer container) "test_avro_topic_fake_source", value -> { try { - return avroDeserializationSchema.deserialize(value); + return avroDeserializationSchema.deserialize( + value, TablePath.of("")); } catch (IOException e) { throw new RuntimeException(e); } @@ -427,7 +429,8 @@ public void testKafkaAvroToAssert(TestContainer container) "test_avro_topic", value -> { try { - return avroDeserializationSchema.deserialize(value); + return avroDeserializationSchema.deserialize( + value, TablePath.of("")); } catch (IOException e) { throw new RuntimeException(e); } From 9ac89dbf489346fdf71111e93921a39c6241710d Mon Sep 17 00:00:00 2001 From: zhilinli Date: Sat, 3 Feb 2024 09:29:55 +0800 Subject: [PATCH 28/71] fix --- .../json/JsonRowDataSerDeSchemaTest.java | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java index 037a60d2ee1..3b8d8783399 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java @@ -24,8 +24,8 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.type.LocalTimeType; import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -152,7 +152,8 @@ public void testSerDe() throws Exception { expected.setField(10, multiSet); expected.setField(11, nestedMap); - SeaTunnelRow seaTunnelRow = deserializationSchema.deserialize(serializedJson); + SeaTunnelRow seaTunnelRow = + deserializationSchema.deserialize(serializedJson, TablePath.of("")); assertEquals(expected, seaTunnelRow); // test serialization @@ -199,7 +200,8 @@ public void testSerDeMultiRows() throws Exception { row.put("f1", "this is row1"); row.put("f2", 12); byte[] serializedJson = objectMapper.writeValueAsBytes(root); - SeaTunnelRow rowData = deserializationSchema.deserialize(serializedJson); + SeaTunnelRow rowData = + deserializationSchema.deserialize(serializedJson, TablePath.of("")); byte[] actual = serializationSchema.serialize(rowData); assertEquals(new String(serializedJson), new String(actual)); } @@ -219,7 +221,8 @@ public void testSerDeMultiRows() throws Exception { row.put("f1", "this is row2"); row.putNull("f2"); byte[] serializedJson = objectMapper.writeValueAsBytes(root); - SeaTunnelRow rowData = deserializationSchema.deserialize(serializedJson); + SeaTunnelRow rowData = + deserializationSchema.deserialize(serializedJson, TablePath.of("")); byte[] actual = serializationSchema.serialize(rowData); assertEquals(new String(serializedJson), new String(actual)); } @@ -260,7 +263,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { for (int i = 0; i < jsons.length; i++) { String json = jsons[i]; - SeaTunnelRow row = deserializationSchema.deserialize(json.getBytes()); + SeaTunnelRow row = deserializationSchema.deserialize(json.getBytes(), TablePath.of("")); String result = new String(serializationSchema.serialize(row)); assertEquals(expected[i], result); } @@ -283,7 +286,7 @@ public void testDeserializationMissingNode() throws Exception { JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(true, false, schema); - SeaTunnelRow rowData = deserializationSchema.deserialize("".getBytes()); + SeaTunnelRow rowData = deserializationSchema.deserialize("".getBytes(), TablePath.of("")); assertEquals(null, rowData); } @@ -303,7 +306,7 @@ public void testDeserializationPassMissingField() throws Exception { final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, false, schema); SeaTunnelRow expected = new SeaTunnelRow(1); - SeaTunnelRow actual = deser.deserialize(serializedJson); + SeaTunnelRow actual = deser.deserialize(serializedJson, TablePath.of("")); assertEquals(expected, actual); } @@ -328,13 +331,13 @@ public void testDeserializationMissingField() throws Exception { assertThrows( SeaTunnelRuntimeException.class, () -> { - deser.deserialize(serializedJson); + deser.deserialize(serializedJson, TablePath.of("")); }, "expecting exception message: " + expected.getMessage()); assertEquals(actual.getMessage(), expected.getMessage()); SeaTunnelRuntimeException expectedCause = - CommonError.jsonOperationError("Common", "Field $.name in " + root.toString()); + CommonError.jsonOperationError("Common", "Field $.name in " + root); Throwable cause = actual.getCause(); assertEquals(cause.getClass(), expectedCause.getClass()); assertEquals(cause.getMessage(), expectedCause.getMessage()); @@ -355,7 +358,7 @@ public void testDeserializationIgnoreParseError() throws Exception { // ignore on parse error final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, true, schema); - assertEquals(expected, deser.deserialize(serializedJson)); + assertEquals(expected, deser.deserialize(serializedJson, TablePath.of(""))); } @Test @@ -396,7 +399,7 @@ public void testDeserializationNoJson() throws Exception { assertThrows( SeaTunnelRuntimeException.class, () -> { - deser.deserialize(noJson.getBytes()); + deser.deserialize(noJson.getBytes(), TablePath.of("")); }, "expecting exception message: " + expected.getMessage()); From 7aad1067e017eb332a6d5b3e3c5e2ea69b9d11dc Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 1 Mar 2024 15:18:31 +0800 Subject: [PATCH 29/71] fix --- .../serialization/DeserializationSchema.java | 16 ++++++++++++++ .../deserialize/AmazonSqsDeserializer.java | 3 +-- .../file/source/reader/JsonReadStrategy.java | 4 +--- .../deserialize/GoogleSheetsDeserializer.java | 3 +-- .../http/source/DeserializationCollector.java | 3 +-- .../SimpleTextDeserializationSchema.java | 14 ++++++++---- .../source/format/PulsarCanalDecorator.java | 12 +++++----- .../source/reader/PulsarSourceReader.java | 3 +-- .../source/PulsarCanalDecoratorTest.java | 4 +--- .../rabbitmq/source/RabbitmqSourceReader.java | 3 +-- .../redis/source/RedisSourceReader.java | 3 +-- .../e2e/connector/kafka/KafkaIT.java | 7 ++---- .../avro/AvroDeserializationSchema.java | 9 +++++++- .../avro/AvroSerializationSchemaTest.java | 3 +-- ...ibleKafkaConnectDeserializationSchema.java | 10 +++++++-- ...ibleDebeziumJsonDeserializationSchema.java | 5 +++++ .../json/JsonDeserializationSchema.java | 12 ++++++---- .../canal/CanalJsonDeserializationSchema.java | 8 ++++++- .../DebeziumJsonDeserializationSchema.java | 8 ++++++- .../ogg/OggJsonDeserializationSchema.java | 8 ++++++- .../json/JsonRowDataSerDeSchemaTest.java | 22 ++++++++----------- .../json/canal/CanalJsonSerDeSchemaTest.java | 6 ++--- .../debezium/DebeziumJsonSerDeSchemaTest.java | 19 ++++++---------- .../json/ogg/OggJsonSerDeSchemaTest.java | 17 +++++--------- .../text/TextDeserializationSchema.java | 13 +++++++---- .../format/text/TextFormatSchemaTest.java | 4 +--- 26 files changed, 128 insertions(+), 91 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java index 5838ba22b46..6cbce2ea91a 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java @@ -26,6 +26,15 @@ public interface DeserializationSchema extends Serializable { + /** + * Deserializes the byte message. + * + * @param message The message, as a byte array. + * @return The deserialized message as an SeaTunnel Row (null if the message cannot be + * deserialized). + */ + T deserialize(byte[] message) throws IOException; + /** * Deserializes the byte message. * @@ -36,6 +45,13 @@ public interface DeserializationSchema extends Serializable { */ T deserialize(byte[] message, TablePath tablePath) throws IOException; + default void deserialize(byte[] message, Collector out) throws IOException { + T deserialize = deserialize(message); + if (deserialize != null) { + out.collect(deserialize); + } + } + default void deserialize(byte[] message, Collector out, TablePath tablePath) throws IOException { T deserialize = deserialize(message, tablePath); diff --git a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/deserialize/AmazonSqsDeserializer.java b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/deserialize/AmazonSqsDeserializer.java index e5b175aa7ef..7322feda795 100644 --- a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/deserialize/AmazonSqsDeserializer.java +++ b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/deserialize/AmazonSqsDeserializer.java @@ -18,7 +18,6 @@ package org.apache.seatunnel.connectors.seatunnel.amazonsqs.deserialize; import org.apache.seatunnel.api.serialization.DeserializationSchema; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import java.io.IOException; @@ -34,7 +33,7 @@ public AmazonSqsDeserializer(DeserializationSchema deserialization @Override public SeaTunnelRow deserializeRow(String row) { try { - return deserializationSchema.deserialize(row.getBytes(), TablePath.of("")); + return deserializationSchema.deserialize(row.getBytes()); } catch (IOException e) { return null; } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java index 667373b186a..0535241af0f 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.exception.CommonError; @@ -94,8 +93,7 @@ public void read(String path, String tableId, Collector output) line -> { try { SeaTunnelRow seaTunnelRow = - deserializationSchema.deserialize( - line.getBytes(), TablePath.of("")); + deserializationSchema.deserialize(line.getBytes()); if (isMergePartition) { int index = seaTunnelRowType.getTotalFields(); for (String value : partitionsMap.values()) { diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java index e80e8a1018c..76cce870c72 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializer.java @@ -21,7 +21,6 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.api.serialization.DeserializationSchema; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.CommonError; import org.apache.seatunnel.connectors.seatunnel.google.sheets.exception.GoogleSheetsError; @@ -54,7 +53,7 @@ public SeaTunnelRow deserializeRow(List row) { try { String rowStr = objectMapper.writeValueAsString(map); - return deserializationSchema.deserialize(rowStr.getBytes(), TablePath.of("")); + return deserializationSchema.deserialize(rowStr.getBytes()); } catch (JsonProcessingException e) { throw CommonError.jsonOperationError("GoogleSheets", map.toString(), e); } catch (IOException e) { diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/DeserializationCollector.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/DeserializationCollector.java index 946b7f8b00e..59743c7832e 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/DeserializationCollector.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/DeserializationCollector.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.format.json.JsonDeserializationSchema; @@ -36,7 +35,7 @@ public void collect(byte[] message, Collector out) throws IOExcept if (deserializationSchema instanceof JsonDeserializationSchema) { ((JsonDeserializationSchema) deserializationSchema).collect(message, out); } else { - SeaTunnelRow deserialize = deserializationSchema.deserialize(message, TablePath.of("")); + SeaTunnelRow deserialize = deserializationSchema.deserialize(message); out.collect(deserialize); } } diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java index a1b8b93d7b1..3a6f773f9f8 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java @@ -25,16 +25,22 @@ import lombok.AllArgsConstructor; +import java.io.IOException; + @AllArgsConstructor public class SimpleTextDeserializationSchema implements DeserializationSchema { private SeaTunnelRowType rowType; @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) { - SeaTunnelRow seaTunnelRow = new SeaTunnelRow(new Object[] {new String(message)}); - seaTunnelRow.setTableId(tablePath.toString()); - return seaTunnelRow; + public SeaTunnelRow deserialize(byte[] message) { + return new SeaTunnelRow(new Object[] {new String(message)}); + } + + @Override + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + throw new UnsupportedOperationException( + "Please invoke SimpleTextDeserializationSchema#deserialize(byte[]) instead."); } @Override diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java index 0644f9df7ff..46642f3ad65 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java @@ -52,17 +52,18 @@ public PulsarCanalDecorator(CanalJsonDeserializationSchema canalJsonDeserializat this.canalJsonDeserializationSchema = canalJsonDeserializationSchema; } + @Override + public SeaTunnelRow deserialize(byte[] message) throws IOException { + throw new UnsupportedOperationException(); + } + @Override public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { throw new UnsupportedOperationException(); } @Override - public void deserialize(byte[] message, Collector out, TablePath tablePath) - throws IOException { - if (message == null || message.length == 0) { - return; - } + public void deserialize(byte[] message, Collector out) throws IOException { JsonNode pulsarCanal = JsonUtils.parseObject(message); ArrayNode canalList = JsonUtils.parseArray(pulsarCanal.get(MESSAGE).asText()); Iterator canalIterator = canalList.elements(); @@ -71,6 +72,7 @@ public void deserialize(byte[] message, Collector out, TablePath t // reconvert pulsar handler, reference to // https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/MessageUtils.java ObjectNode root = reconvertPulsarData((ObjectNode) next); + // todo Wait for TablePath functionality to be added canalJsonDeserializationSchema.deserialize(root, out, TablePath.of("")); } } diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/reader/PulsarSourceReader.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/reader/PulsarSourceReader.java index 9516f1b63b6..d811f49af31 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/reader/PulsarSourceReader.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/reader/PulsarSourceReader.java @@ -21,7 +21,6 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.common.Handover; import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; import org.apache.seatunnel.connectors.seatunnel.pulsar.config.PulsarClientConfig; @@ -135,7 +134,7 @@ public void pollNext(Collector output) throws Exception { final Message message = recordWithSplitId.get().getMessage(); synchronized (output.getCheckpointLock()) { splitStates.get(splitId).setLatestConsumedId(message.getMessageId()); - deserialization.deserialize(message.getData(), output, TablePath.of("")); + deserialization.deserialize(message.getData(), output); } } if (noMoreSplitsAssignment && finishedSplits.size() == splitStates.size()) { diff --git a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java index e5110a65b7e..0076bbbdbea 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java @@ -18,7 +18,6 @@ package org.apache.seatunnel.connectors.seatunnel.pulsar.source; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -63,8 +62,7 @@ void decoder() throws IOException { new PulsarCanalDecorator(canalJsonDeserializationSchema); SimpleCollector simpleCollector = new SimpleCollector(); - pulsarCanalDecorator.deserialize( - json.getBytes(StandardCharsets.UTF_8), simpleCollector, TablePath.of("")); + pulsarCanalDecorator.deserialize(json.getBytes(StandardCharsets.UTF_8), simpleCollector); Assertions.assertFalse(simpleCollector.getList().isEmpty()); for (SeaTunnelRow seaTunnelRow : simpleCollector.list) { for (Object field : seaTunnelRow.getFields()) { diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceReader.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceReader.java index ae40f151180..df80294f147 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceReader.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceReader.java @@ -21,7 +21,6 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.Handover; import org.apache.seatunnel.connectors.seatunnel.rabbitmq.client.RabbitmqClient; @@ -128,7 +127,7 @@ public void pollNext(Collector output) throws Exception { return; } deliveryTagsProcessedForCurrentSnapshot.add(envelope.getDeliveryTag()); - deserializationSchema.deserialize(body, output, TablePath.of("")); + deserializationSchema.deserialize(body, output); } if (Boundedness.BOUNDED.equals(context.getBoundedness())) { diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java index 7a141e99fd6..9aeb7daa69e 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java @@ -92,8 +92,7 @@ public void internalPollNext(Collector output) throws Exception { TablePath.of("")); } } else { - deserializationSchema.deserialize( - value.getBytes(), output, TablePath.of("")); + deserializationSchema.deserialize(value.getBytes(), output); } } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index 44d082b6f30..d86aa891865 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -20,7 +20,6 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -378,8 +377,7 @@ public void testFakeSourceToKafkaAvroFormat(TestContainer container) "test_avro_topic_fake_source", value -> { try { - return avroDeserializationSchema.deserialize( - value, TablePath.of("")); + return avroDeserializationSchema.deserialize(value); } catch (IOException e) { throw new RuntimeException(e); } @@ -429,8 +427,7 @@ public void testKafkaAvroToAssert(TestContainer container) "test_avro_topic", value -> { try { - return avroDeserializationSchema.deserialize( - value, TablePath.of("")); + return avroDeserializationSchema.deserialize(value); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java index 1e9be047e60..3e53c4847c6 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java @@ -42,12 +42,19 @@ public AvroDeserializationSchema(SeaTunnelRowType rowType) { } @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + public SeaTunnelRow deserialize(byte[] message) throws IOException { BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(message, null); GenericRecord record = this.converter.getReader().read(null, decoder); return converter.converter(record, rowType); } + @Override + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + SeaTunnelRow deserialize = deserialize(message); + deserialize.setTableId(tablePath.toString()); + return deserialize(message); + } + @Override public SeaTunnelDataType getProducedType() { return this.rowType; diff --git a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java index 1da662c8fda..430a5628d67 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.format.avro; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -163,7 +162,7 @@ public void testSerialization() throws IOException { AvroSerializationSchema serializationSchema = new AvroSerializationSchema(rowType); byte[] bytes = serializationSchema.serialize(seaTunnelRow); AvroDeserializationSchema deserializationSchema = new AvroDeserializationSchema(rowType); - SeaTunnelRow deserialize = deserializationSchema.deserialize(bytes, TablePath.of("")); + SeaTunnelRow deserialize = deserializationSchema.deserialize(bytes); String[] strArray1 = (String[]) seaTunnelRow.getField(1); String[] strArray2 = (String[]) deserialize.getField(1); Assertions.assertArrayEquals(strArray1, strArray2); diff --git a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java index f54beeece6d..57384cb2803 100644 --- a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java @@ -43,7 +43,6 @@ import lombok.RequiredArgsConstructor; import java.io.IOException; -import java.io.UnsupportedEncodingException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.Collections; @@ -87,9 +86,16 @@ public CompatibleKafkaConnectDeserializationSchema( .createConverter(checkNotNull(seaTunnelRowType)); } + @Override + public SeaTunnelRow deserialize(byte[] message) throws IOException { + throw new UnsupportedOperationException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + @Override public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - throw new UnsupportedEncodingException(); + throw new UnsupportedOperationException( + "Please invoke DeserializationSchema#deserialize(byte[],TablePath, Collector) instead."); } /** diff --git a/seatunnel-formats/seatunnel-format-compatible-debezium-json/src/main/java/org/apache/seatunnel/format/compatible/debezium/json/CompatibleDebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-debezium-json/src/main/java/org/apache/seatunnel/format/compatible/debezium/json/CompatibleDebeziumJsonDeserializationSchema.java index 00b6b86e636..684bd999ba7 100644 --- a/seatunnel-formats/seatunnel-format-compatible-debezium-json/src/main/java/org/apache/seatunnel/format/compatible/debezium/json/CompatibleDebeziumJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-debezium-json/src/main/java/org/apache/seatunnel/format/compatible/debezium/json/CompatibleDebeziumJsonDeserializationSchema.java @@ -51,6 +51,11 @@ public CompatibleDebeziumJsonDeserializationSchema( this.debeziumJsonConverter = new DebeziumJsonConverter(keySchemaEnable, valueSchemaEnable); } + @Override + public SeaTunnelRow deserialize(byte[] message) throws IOException { + throw new UnsupportedEncodingException(); + } + @Override public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { throw new UnsupportedEncodingException(); diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 5bd950fced4..870c6b18714 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -100,12 +100,16 @@ private static boolean hasDecimalType(SeaTunnelDataType dataType) { } @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - if (message == null || message.length == 0) { + public SeaTunnelRow deserialize(byte[] message) throws IOException { + if (message == null) { return null; } - SeaTunnelRow seaTunnelRow = convertJsonNode(convertBytes(message)); - assert seaTunnelRow != null; + return convertJsonNode(convertBytes(message)); + } + + @Override + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + SeaTunnelRow seaTunnelRow = deserialize(message); seaTunnelRow.setTableId(tablePath.toString()); return seaTunnelRow; } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java index f44292958ae..f2ec85216f0 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java @@ -106,11 +106,17 @@ public CanalJsonDeserializationSchema( } @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + public SeaTunnelRow deserialize(byte[] message) throws IOException { throw new UnsupportedOperationException( "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); } + @Override + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + throw new UnsupportedOperationException( + "Please invoke DeserializationSchema#deserialize(byte[],TablePath tablePath , Collector) instead."); + } + @Override public SeaTunnelDataType getProducedType() { return this.physicalRowType; diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java index 190b7f20610..985d907eefd 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java @@ -78,11 +78,17 @@ public DebeziumJsonDeserializationSchema( } @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + public SeaTunnelRow deserialize(byte[] message) throws IOException { throw new UnsupportedOperationException( "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); } + @Override + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + throw new UnsupportedOperationException( + "Please invoke DeserializationSchema#deserialize(byte[],TablePath tablePath,Collector) instead."); + } + @Override public void deserialize(byte[] message, Collector out, TablePath tablePath) throws IOException { diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java index 95a81f4ef16..279e207c830 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java @@ -103,11 +103,17 @@ public OggJsonDeserializationSchema( } @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + public SeaTunnelRow deserialize(byte[] message) throws IOException { throw new UnsupportedOperationException( "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); } + @Override + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + throw new UnsupportedOperationException( + "Please invoke DeserializationSchema#deserialize(byte[],TablePath tablePath,Collector) instead."); + } + @Override public SeaTunnelDataType getProducedType() { return this.physicalRowType; diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java index 3b8d8783399..a25fa9479d5 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java @@ -24,7 +24,6 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.type.LocalTimeType; import org.apache.seatunnel.api.table.type.MapType; @@ -152,8 +151,7 @@ public void testSerDe() throws Exception { expected.setField(10, multiSet); expected.setField(11, nestedMap); - SeaTunnelRow seaTunnelRow = - deserializationSchema.deserialize(serializedJson, TablePath.of("")); + SeaTunnelRow seaTunnelRow = deserializationSchema.deserialize(serializedJson); assertEquals(expected, seaTunnelRow); // test serialization @@ -200,8 +198,7 @@ public void testSerDeMultiRows() throws Exception { row.put("f1", "this is row1"); row.put("f2", 12); byte[] serializedJson = objectMapper.writeValueAsBytes(root); - SeaTunnelRow rowData = - deserializationSchema.deserialize(serializedJson, TablePath.of("")); + SeaTunnelRow rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); assertEquals(new String(serializedJson), new String(actual)); } @@ -221,8 +218,7 @@ public void testSerDeMultiRows() throws Exception { row.put("f1", "this is row2"); row.putNull("f2"); byte[] serializedJson = objectMapper.writeValueAsBytes(root); - SeaTunnelRow rowData = - deserializationSchema.deserialize(serializedJson, TablePath.of("")); + SeaTunnelRow rowData = deserializationSchema.deserialize(serializedJson); byte[] actual = serializationSchema.serialize(rowData); assertEquals(new String(serializedJson), new String(actual)); } @@ -263,7 +259,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { for (int i = 0; i < jsons.length; i++) { String json = jsons[i]; - SeaTunnelRow row = deserializationSchema.deserialize(json.getBytes(), TablePath.of("")); + SeaTunnelRow row = deserializationSchema.deserialize(json.getBytes()); String result = new String(serializationSchema.serialize(row)); assertEquals(expected[i], result); } @@ -286,7 +282,7 @@ public void testDeserializationMissingNode() throws Exception { JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(true, false, schema); - SeaTunnelRow rowData = deserializationSchema.deserialize("".getBytes(), TablePath.of("")); + SeaTunnelRow rowData = deserializationSchema.deserialize("".getBytes()); assertEquals(null, rowData); } @@ -306,7 +302,7 @@ public void testDeserializationPassMissingField() throws Exception { final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, false, schema); SeaTunnelRow expected = new SeaTunnelRow(1); - SeaTunnelRow actual = deser.deserialize(serializedJson, TablePath.of("")); + SeaTunnelRow actual = deser.deserialize(serializedJson); assertEquals(expected, actual); } @@ -331,7 +327,7 @@ public void testDeserializationMissingField() throws Exception { assertThrows( SeaTunnelRuntimeException.class, () -> { - deser.deserialize(serializedJson, TablePath.of("")); + deser.deserialize(serializedJson); }, "expecting exception message: " + expected.getMessage()); assertEquals(actual.getMessage(), expected.getMessage()); @@ -358,7 +354,7 @@ public void testDeserializationIgnoreParseError() throws Exception { // ignore on parse error final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, true, schema); - assertEquals(expected, deser.deserialize(serializedJson, TablePath.of(""))); + assertEquals(expected, deser.deserialize(serializedJson)); } @Test @@ -399,7 +395,7 @@ public void testDeserializationNoJson() throws Exception { assertThrows( SeaTunnelRuntimeException.class, () -> { - deser.deserialize(noJson.getBytes(), TablePath.of("")); + deser.deserialize(noJson.getBytes()); }, "expecting exception message: " + expected.getMessage()); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java index 1226b71b124..c7544f6db8d 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java @@ -19,7 +19,6 @@ package org.apache.seatunnel.format.json.canal; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -66,7 +65,7 @@ public void testDeserializeNullRow() throws Exception { createCanalJsonDeserializationSchema(null, null); final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize((byte[]) null, collector, TablePath.of("")); + deserializationSchema.deserialize((byte[]) null, collector); assertEquals(0, collector.list.size()); } @@ -74,8 +73,7 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali throws IOException { SimpleCollector collector = new SimpleCollector(); for (String line : lines) { - deserializationSchema.deserialize( - line.getBytes(StandardCharsets.UTF_8), collector, TablePath.of("")); + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); } List expected = diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index 7c5b6789d16..9ad9c1202d3 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -19,7 +19,6 @@ package org.apache.seatunnel.format.json.debezium; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -61,8 +60,8 @@ void testNullRowMessages() throws Exception { new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize(null, collector, TablePath.of("")); - deserializationSchema.deserialize(new byte[0], collector, TablePath.of("")); + deserializationSchema.deserialize(null, collector); + deserializationSchema.deserialize(new byte[0], collector); assertEquals(0, collector.list.size()); } @@ -84,8 +83,7 @@ public void testDeserializeNoJson() { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize( - noJsonMsg.getBytes(), collector, TablePath.of("")); + deserializationSchema.deserialize(noJsonMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); } @@ -101,8 +99,7 @@ public void testDeserializeEmptyJson() { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize( - emptyMsg.getBytes(), collector, TablePath.of("")); + deserializationSchema.deserialize(emptyMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); } @@ -118,8 +115,7 @@ public void testDeserializeNoDataJson() { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize( - noDataMsg.getBytes(), collector, TablePath.of("")); + deserializationSchema.deserialize(noDataMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); @@ -151,7 +147,7 @@ public void testDeserializeUnknownOperationTypeJson() { expected.getClass(), () -> { deserializationSchema.deserialize( - unknownOperationMsg.getBytes(), collector, TablePath.of("")); + unknownOperationMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); @@ -170,8 +166,7 @@ private void testSerializationDeserialization(boolean schemaInclude) throws Exce SimpleCollector collector = new SimpleCollector(); for (String line : lines) { - deserializationSchema.deserialize( - line.getBytes(StandardCharsets.UTF_8), collector, TablePath.of("")); + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); } List expected = diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java index ef348a67619..90553ea2cd2 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java @@ -19,7 +19,6 @@ package org.apache.seatunnel.format.json.ogg; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -72,7 +71,7 @@ public void testDeserializeNullRow() throws Exception { createOggJsonDeserializationSchema(null, null); final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize((byte[]) null, collector, TablePath.of("")); + deserializationSchema.deserialize((byte[]) null, collector); assertEquals(0, collector.list.size()); } @@ -88,8 +87,7 @@ public void testDeserializeNoJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize( - noJsonMsg.getBytes(), collector, TablePath.of("")); + deserializationSchema.deserialize(noJsonMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); } @@ -105,8 +103,7 @@ public void testDeserializeEmptyJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize( - emptyMsg.getBytes(), collector, TablePath.of("")); + deserializationSchema.deserialize(emptyMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); } @@ -122,8 +119,7 @@ public void testDeserializeNoDataJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize( - noDataMsg.getBytes(), collector, TablePath.of("")); + deserializationSchema.deserialize(noDataMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); @@ -153,7 +149,7 @@ public void testDeserializeUnknownTypeJson() throws Exception { expected.getClass(), () -> { deserializationSchema.deserialize( - unknownOperationMsg.getBytes(), collector, TablePath.of("")); + unknownOperationMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); @@ -168,8 +164,7 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa throws IOException { SimpleCollector collector = new SimpleCollector(); for (String line : lines) { - deserializationSchema.deserialize( - line.getBytes(StandardCharsets.UTF_8), collector, TablePath.of("")); + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); } List expected = diff --git a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java index 1a2d2a99b86..b8d41300a39 100644 --- a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java @@ -113,7 +113,7 @@ public TextDeserializationSchema build() { } @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + public SeaTunnelRow deserialize(byte[] message) throws IOException { if (message == null || message.length == 0) { return null; } @@ -123,9 +123,14 @@ public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOEx for (int i = 0; i < objects.length; i++) { objects[i] = convert(splitsMap.get(i), seaTunnelRowType.getFieldType(i), 0); } - SeaTunnelRow seaTunnelRow = new SeaTunnelRow(objects); - seaTunnelRow.setTableId(tablePath.toString()); - return seaTunnelRow; + return new SeaTunnelRow(objects); + } + + @Override + public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { + SeaTunnelRow deserialize = deserialize(message); + deserialize.setTableId(tablePath.toString()); + return deserialize; } @Override diff --git a/seatunnel-formats/seatunnel-format-text/src/test/java/org/apache/seatunnel/format/text/TextFormatSchemaTest.java b/seatunnel-formats/seatunnel-format-text/src/test/java/org/apache/seatunnel/format/text/TextFormatSchemaTest.java index 7bff7cdc11b..57e99d49b69 100644 --- a/seatunnel-formats/seatunnel-format-text/src/test/java/org/apache/seatunnel/format/text/TextFormatSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-text/src/test/java/org/apache/seatunnel/format/text/TextFormatSchemaTest.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.format.text; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -137,8 +136,7 @@ public void testParse() throws IOException { .seaTunnelRowType(seaTunnelRowType) .delimiter("\u0001") .build(); - SeaTunnelRow seaTunnelRow = - deserializationSchema.deserialize(content.getBytes(), TablePath.of("")); + SeaTunnelRow seaTunnelRow = deserializationSchema.deserialize(content.getBytes()); String data = new String(serializationSchema.serialize(seaTunnelRow)); Assertions.assertEquals(((Map) (seaTunnelRow.getField(1))).get("tyrantlucifer"), 18); Assertions.assertEquals(((Map) (seaTunnelRow.getField(1))).get("Kris"), 21); From ea6792887e61792c6601d6bf8c3718563ca8625c Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 5 Mar 2024 13:39:48 +0800 Subject: [PATCH 30/71] fix --- seatunnel-connectors-v2/connector-pulsar/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/seatunnel-connectors-v2/connector-pulsar/pom.xml b/seatunnel-connectors-v2/connector-pulsar/pom.xml index 12bbec59187..e519b776be1 100644 --- a/seatunnel-connectors-v2/connector-pulsar/pom.xml +++ b/seatunnel-connectors-v2/connector-pulsar/pom.xml @@ -47,6 +47,11 @@ ${project.version} + + org.apache.seatunnel + seatunnel-api + ${project.version} + From c953284dee6d8cabaef20ffab1a854d66c33ea15 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 5 Mar 2024 18:31:26 +0800 Subject: [PATCH 31/71] fix --- docs/zh/faq.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/zh/faq.md b/docs/zh/faq.md index 7451d81b9bf..d89c8fa6477 100644 --- a/docs/zh/faq.md +++ b/docs/zh/faq.md @@ -293,8 +293,6 @@ log4j.appender.console.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{ 参考: -https://stackoverflow.com/questions/27781187how-to-stop-info-messages-displaying-on-spark-console - http://spark.apache.org/docs/latest/configuration.html#configuring-logging https://medium.com/@iacomini.riccardo/spark-logging-configuration-in-yarn-faf5ba5fdb01 From 082e252f01ed4ed78378132be3b1a5646ab81a6a Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 5 Mar 2024 22:47:20 +0800 Subject: [PATCH 32/71] fix --- .../format/json/canal/CanalJsonSerDeSchemaTest.java | 6 ++++-- .../format/json/debezium/DebeziumJsonSerDeSchemaTest.java | 4 +++- .../seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java | 4 +++- 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java index c7544f6db8d..b00ff2f4428 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java @@ -19,6 +19,7 @@ package org.apache.seatunnel.format.json.canal; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -65,7 +66,7 @@ public void testDeserializeNullRow() throws Exception { createCanalJsonDeserializationSchema(null, null); final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize((byte[]) null, collector); + deserializationSchema.deserialize((byte[]) null, collector, TablePath.of("test")); assertEquals(0, collector.list.size()); } @@ -73,7 +74,8 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali throws IOException { SimpleCollector collector = new SimpleCollector(); for (String line : lines) { - deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + deserializationSchema.deserialize( + line.getBytes(StandardCharsets.UTF_8), collector, TablePath.of("test")); } List expected = diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index 9ad9c1202d3..0575d1e64f1 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -19,6 +19,7 @@ package org.apache.seatunnel.format.json.debezium; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -115,7 +116,8 @@ public void testDeserializeNoDataJson() { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize(noDataMsg.getBytes(), collector); + deserializationSchema.deserialize( + noDataMsg.getBytes(), collector, TablePath.of("test")); }); assertEquals(cause.getMessage(), expected.getMessage()); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java index 90553ea2cd2..4b7e9a5bc2e 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java @@ -19,6 +19,7 @@ package org.apache.seatunnel.format.json.ogg; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -119,7 +120,8 @@ public void testDeserializeNoDataJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize(noDataMsg.getBytes(), collector); + deserializationSchema.deserialize( + noDataMsg.getBytes(), collector, TablePath.of("test")); }); assertEquals(cause.getMessage(), expected.getMessage()); From f6c7c9b1b9bb17b7321e46b05bd136f4b4de1082 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Wed, 6 Mar 2024 10:05:50 +0800 Subject: [PATCH 33/71] fix --- .../seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java index b00ff2f4428..037ada54e46 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java @@ -75,7 +75,7 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali SimpleCollector collector = new SimpleCollector(); for (String line : lines) { deserializationSchema.deserialize( - line.getBytes(StandardCharsets.UTF_8), collector, TablePath.of("test")); + line.getBytes(StandardCharsets.UTF_8), collector, TablePath.of("")); } List expected = From 2fca6c950a665616e9f27ba8390159b2f64c1213 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Wed, 6 Mar 2024 20:14:42 +0800 Subject: [PATCH 34/71] fix --- .../format/json/debezium/DebeziumJsonSerDeSchemaTest.java | 3 ++- .../seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index 0575d1e64f1..f9e824fcddb 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -100,7 +100,8 @@ public void testDeserializeEmptyJson() { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize(emptyMsg.getBytes(), collector); + deserializationSchema.deserialize( + emptyMsg.getBytes(), collector, TablePath.of("")); }); assertEquals(cause.getMessage(), expected.getMessage()); } diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java index 4b7e9a5bc2e..749dd234dfc 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java @@ -104,7 +104,8 @@ public void testDeserializeEmptyJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize(emptyMsg.getBytes(), collector); + deserializationSchema.deserialize( + emptyMsg.getBytes(), collector, TablePath.of("")); }); assertEquals(cause.getMessage(), expected.getMessage()); } From c357cf9fd92acc52b9a8c9fc71ee2a44639f3a3a Mon Sep 17 00:00:00 2001 From: zhilinli Date: Wed, 6 Mar 2024 23:36:47 +0800 Subject: [PATCH 35/71] fix --- .../json/JsonDeserializationSchema.java | 4 +- .../canal/CanalJsonDeserializationSchema.java | 128 ++++++++------- .../DebeziumJsonDeserializationSchema.java | 84 ++++++---- .../ogg/OggJsonDeserializationSchema.java | 153 ++++++++++-------- .../json/JsonRowDataSerDeSchemaTest.java | 2 +- .../json/canal/CanalJsonSerDeSchemaTest.java | 91 ++++++++++- .../debezium/DebeziumJsonSerDeSchemaTest.java | 25 ++- .../json/ogg/OggJsonSerDeSchemaTest.java | 17 +- 8 files changed, 322 insertions(+), 182 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 870c6b18714..82c7befe48d 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -110,7 +110,9 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { @Override public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { SeaTunnelRow seaTunnelRow = deserialize(message); - seaTunnelRow.setTableId(tablePath.toString()); + if (tablePath != null && !tablePath.toString().isEmpty()) { + seaTunnelRow.setTableId(tablePath.toString()); + } return seaTunnelRow; } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java index f2ec85216f0..7370b270880 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java @@ -122,8 +122,24 @@ public SeaTunnelDataType getProducedType() { return this.physicalRowType; } - public void deserialize(ObjectNode jsonNode, Collector out, TablePath tablePath) - throws IOException { + public void deserializeMessage( + byte[] message, Collector out, TablePath tablePath) { + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } + + ObjectNode jsonNode; + try { + jsonNode = convertBytes(message); + } catch (SeaTunnelRuntimeException cause) { + if (!ignoreParseErrors) { + throw cause; + } else { + return; + } + } + try { if (database != null && !databasePattern.matcher(jsonNode.get(FIELD_DATABASE).asText()).matches()) { @@ -135,52 +151,67 @@ public void deserialize(ObjectNode jsonNode, Collector out, TableP } JsonNode dataNode = jsonNode.get(FIELD_DATA); - String type = jsonNode.get(FIELD_TYPE).asText(); + String op = jsonNode.get(FIELD_TYPE).asText(); // When a null value is encountered, an exception needs to be thrown for easy sensing if (dataNode == null || dataNode.isNull()) { // We'll skip the query or create or alter event data - if (OP_QUERY.equals(type) || OP_CREATE.equals(type) || OP_ALTER.equals(type)) { + if (OP_QUERY.equals(op) || OP_CREATE.equals(op) || OP_ALTER.equals(op)) { return; } throw new IllegalStateException( format("Null data value '%s' Cannot send downstream", jsonNode)); } - if (OP_INSERT.equals(type)) { - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow row = convertJsonNode(dataNode.get(i)); - row.setTableId(tablePath.toString()); - out.collect(row); - } - } else if (OP_UPDATE.equals(type)) { - final ArrayNode oldNode = (ArrayNode) jsonNode.get(FIELD_OLD); - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow after = convertJsonNode(dataNode.get(i)); - SeaTunnelRow before = convertJsonNode(oldNode.get(i)); - for (int f = 0; f < fieldCount; f++) { - if (before.isNullAt(f) && oldNode.findValue(fieldNames[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)); + + switch (op) { + case OP_INSERT: + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow row = convertJsonNode(dataNode.get(i)); + if (tablePath != null && !tablePath.toString().isEmpty()) { + row.setTableId(tablePath.toString()); } + out.collect(row); } - before.setRowKind(RowKind.UPDATE_BEFORE); - before.setTableId(tablePath.toString()); - after.setRowKind(RowKind.UPDATE_AFTER); - after.setTableId(tablePath.toString()); - out.collect(before); - out.collect(after); - } - } else if (OP_DELETE.equals(type)) { - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow row = convertJsonNode(dataNode.get(i)); - row.setRowKind(RowKind.DELETE); - row.setTableId(tablePath.toString()); - out.collect(row); - } - } else { - throw new IllegalStateException(format("Unknown operation type '%s'.", type)); + break; + case OP_UPDATE: + final ArrayNode oldNode = (ArrayNode) jsonNode.get(FIELD_OLD); + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow after = convertJsonNode(dataNode.get(i)); + SeaTunnelRow before = convertJsonNode(oldNode.get(i)); + for (int f = 0; f < fieldCount; f++) { + if (before.isNullAt(f) && oldNode.findValue(fieldNames[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); + if (tablePath != null && !tablePath.toString().isEmpty()) { + before.setTableId(tablePath.toString()); + } + after.setRowKind(RowKind.UPDATE_AFTER); + if (tablePath != null && !tablePath.toString().isEmpty()) { + after.setTableId(tablePath.toString()); + } + out.collect(before); + out.collect(after); + } + break; + case OP_DELETE: + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow row = convertJsonNode(dataNode.get(i)); + row.setRowKind(RowKind.DELETE); + if (tablePath != null && !tablePath.toString().isEmpty()) { + row.setTableId(tablePath.toString()); + } + out.collect(row); + } + break; + default: + throw new IllegalStateException( + String.format("Unknown operation type '%s'.", op)); } + } catch (RuntimeException e) { if (!ignoreParseErrors) { throw CommonError.jsonOperationError(FORMAT, jsonNode.toString(), e); @@ -197,24 +228,13 @@ private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException } @Override - public void deserialize(byte[] message, Collector out, TablePath tablePath) - throws IOException { - if (message == null) { - return; - } - - ObjectNode jsonNode; - try { - jsonNode = convertBytes(message); - } catch (SeaTunnelRuntimeException cause) { - if (!ignoreParseErrors) { - throw cause; - } else { - return; - } - } + public void deserialize(byte[] message, Collector out) { + deserializeMessage(message, out, null); + } - deserialize(jsonNode, out, tablePath); + @Override + public void deserialize(byte[] message, Collector out, TablePath tablePath) { + deserializeMessage(message, out, tablePath); } private SeaTunnelRow convertJsonNode(JsonNode root) { diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java index 985d907eefd..1cb2616e057 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java @@ -89,9 +89,19 @@ public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOEx "Please invoke DeserializationSchema#deserialize(byte[],TablePath tablePath,Collector) instead."); } + @Override + public void deserialize(byte[] message, Collector out) { + deserializeMessage(message, out, null); + } + @Override public void deserialize(byte[] message, Collector out, TablePath tablePath) throws IOException { + deserializeMessage(message, out, tablePath); + } + + private void deserializeMessage( + byte[] message, Collector out, TablePath tablePath) { if (message == null || message.length == 0) { // skip tombstone messages return; @@ -101,36 +111,50 @@ public void deserialize(byte[] message, Collector out, TablePath t JsonNode payload = getPayload(convertBytes(message)); String op = payload.get("op").asText(); - if (OP_CREATE.equals(op) || OP_READ.equals(op)) { - SeaTunnelRow insert = convertJsonNode(payload.get("after")); - insert.setRowKind(RowKind.INSERT); - insert.setTableId(tablePath.toString()); - out.collect(insert); - } else if (OP_UPDATE.equals(op)) { - SeaTunnelRow before = convertJsonNode(payload.get("before")); - if (before == null) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - before.setRowKind(RowKind.UPDATE_BEFORE); - before.setTableId(tablePath.toString()); - out.collect(before); - - SeaTunnelRow after = convertJsonNode(payload.get("after")); - after.setRowKind(RowKind.UPDATE_AFTER); - after.setTableId(tablePath.toString()); - out.collect(after); - } else if (OP_DELETE.equals(op)) { - SeaTunnelRow delete = convertJsonNode(payload.get("before")); - if (delete == null) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - delete.setRowKind(RowKind.DELETE); - delete.setTableId(tablePath.toString()); - out.collect(delete); - } else { - throw new IllegalStateException(format("Unknown operation type '%s'.", op)); + switch (op) { + case OP_CREATE: + case OP_READ: + SeaTunnelRow insert = convertJsonNode(payload.get("after")); + insert.setRowKind(RowKind.INSERT); + if (tablePath != null && !tablePath.toString().isEmpty()) { + insert.setTableId(tablePath.toString()); + } + out.collect(insert); + break; + case OP_UPDATE: + SeaTunnelRow before = convertJsonNode(payload.get("before")); + if (before == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + before.setRowKind(RowKind.UPDATE_BEFORE); + if (tablePath != null && !tablePath.toString().isEmpty()) { + before.setTableId(tablePath.toString()); + } + out.collect(before); + + SeaTunnelRow after = convertJsonNode(payload.get("after")); + after.setRowKind(RowKind.UPDATE_AFTER); + + if (tablePath != null && !tablePath.toString().isEmpty()) { + after.setTableId(tablePath.toString()); + } + out.collect(after); + break; + case OP_DELETE: + SeaTunnelRow delete = convertJsonNode(payload.get("before")); + if (delete == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + delete.setRowKind(RowKind.DELETE); + if (tablePath != null && !tablePath.toString().isEmpty()) { + delete.setTableId(tablePath.toString()); + } + out.collect(delete); + break; + default: + throw new IllegalStateException(format("Unknown operation type '%s'.", op)); } } catch (RuntimeException e) { // a big try catch to protect the processing. diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java index 279e207c830..972c1b03e22 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java @@ -35,8 +35,6 @@ import java.io.IOException; import java.util.regex.Pattern; -import static java.lang.String.format; - public class OggJsonDeserializationSchema implements DeserializationSchema { private static final long serialVersionUID = 1L; @@ -119,16 +117,25 @@ public SeaTunnelDataType getProducedType() { return this.physicalRowType; } - private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException { + public void deserializeMessage( + byte[] message, Collector out, TablePath tablePath) { + + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } + + ObjectNode jsonNode; try { - return (ObjectNode) jsonDeserializer.deserializeToJsonNode(message); - } catch (Throwable t) { - throw CommonError.jsonOperationError(FORMAT, new String(message), t); + jsonNode = convertBytes(message); + } catch (RuntimeException e) { + if (!ignoreParseErrors) { + throw e; + } else { + return; + } } - } - public void deserialize(ObjectNode jsonNode, Collector out, TablePath tablePath) - throws IOException { try { if (database != null && !databasePattern @@ -144,49 +151,64 @@ public void deserialize(ObjectNode jsonNode, Collector out, TableP } String op = jsonNode.get(FIELD_TYPE).asText().trim(); - if (OP_INSERT.equals(op)) { - // Gets the data for the INSERT operation - JsonNode dataAfter = jsonNode.get(DATA_AFTER); - SeaTunnelRow row = convertJsonNode(dataAfter); - row.setTableId(tablePath.toString()); - out.collect(row); - } else if (OP_UPDATE.equals(op)) { - JsonNode dataBefore = jsonNode.get(DATA_BEFORE); - // Modify Operation Data cannot be empty before modification - if (dataBefore == null || dataBefore.isNull()) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - JsonNode dataAfter = jsonNode.get(DATA_AFTER); - // Gets the data for the UPDATE BEFORE operation - SeaTunnelRow before = convertJsonNode(dataBefore); - // Gets the data for the UPDATE AFTER operation - SeaTunnelRow after = convertJsonNode(dataAfter); - before.setRowKind(RowKind.UPDATE_BEFORE); - before.setTableId(tablePath.toString()); - out.collect(before); - - after.setRowKind(RowKind.UPDATE_AFTER); - after.setTableId(tablePath.toString()); - out.collect(after); - } else if (OP_DELETE.equals(op)) { - JsonNode dataBefore = jsonNode.get(DATA_BEFORE); - if (dataBefore == null || dataBefore.isNull()) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); - } - // Gets the data for the DELETE BEFORE operation - SeaTunnelRow before = convertJsonNode(dataBefore); - if (before == null) { + + switch (op) { + case OP_INSERT: + // Gets the data for the INSERT operation + JsonNode dataInsert = jsonNode.get(DATA_AFTER); + SeaTunnelRow row = convertJsonNode(dataInsert); + if (tablePath != null && !tablePath.toString().isEmpty()) { + row.setTableId(tablePath.toString()); + } + out.collect(row); + break; + case OP_UPDATE: + JsonNode dataBefore = jsonNode.get(DATA_BEFORE); + // Modify Operation Data cannot be empty before modification + if (dataBefore == null || dataBefore.isNull()) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + JsonNode dataAfter = jsonNode.get(DATA_AFTER); + // Gets the data for the UPDATE BEFORE operation + SeaTunnelRow before = convertJsonNode(dataBefore); + // Gets the data for the UPDATE AFTER operation + SeaTunnelRow after = convertJsonNode(dataAfter); + before.setRowKind(RowKind.UPDATE_BEFORE); + if (tablePath != null && !tablePath.toString().isEmpty()) { + before.setTableId(tablePath.toString()); + } + out.collect(before); + + after.setRowKind(RowKind.UPDATE_AFTER); + if (tablePath != null && !tablePath.toString().isEmpty()) { + after.setTableId(tablePath.toString()); + } + out.collect(after); + break; + case OP_DELETE: + JsonNode dataBeforeDel = jsonNode.get(DATA_BEFORE); + if (dataBeforeDel == null || dataBeforeDel.isNull()) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); + } + // Gets the data for the DELETE BEFORE operation + SeaTunnelRow beforeDelete = convertJsonNode(dataBeforeDel); + if (beforeDelete == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); + } + beforeDelete.setRowKind(RowKind.DELETE); + if (tablePath != null && !tablePath.toString().isEmpty()) { + beforeDelete.setTableId(tablePath.toString()); + } + out.collect(beforeDelete); + break; + default: throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); - } - before.setRowKind(RowKind.DELETE); - before.setTableId(tablePath.toString()); - out.collect(before); - } else { - throw new IllegalStateException(format("Unknown operation type '%s'.", op)); + String.format("Unknown operation type '%s'.", op)); } + } catch (RuntimeException e) { if (!ignoreParseErrors) { throw CommonError.jsonOperationError(FORMAT, jsonNode.toString(), e); @@ -194,25 +216,22 @@ public void deserialize(ObjectNode jsonNode, Collector out, TableP } } - @Override - public void deserialize(byte[] message, Collector out, TablePath tablePath) - throws IOException { - if (message == null || message.length == 0) { - // skip tombstone messages - return; - } - - ObjectNode jsonNode; + private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException { try { - jsonNode = convertBytes(message); - } catch (RuntimeException e) { - if (!ignoreParseErrors) { - throw e; - } else { - return; - } + return (ObjectNode) jsonDeserializer.deserializeToJsonNode(message); + } catch (Throwable t) { + throw CommonError.jsonOperationError(FORMAT, new String(message), t); } - deserialize(jsonNode, out, tablePath); + } + + @Override + public void deserialize(byte[] message, Collector out) { + deserializeMessage(message, out, null); + } + + @Override + public void deserialize(byte[] message, Collector out, TablePath tablePath) { + deserializeMessage(message, out, tablePath); } private SeaTunnelRow convertJsonNode(JsonNode root) { diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java index a25fa9479d5..7a816258115 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java @@ -333,7 +333,7 @@ public void testDeserializationMissingField() throws Exception { assertEquals(actual.getMessage(), expected.getMessage()); SeaTunnelRuntimeException expectedCause = - CommonError.jsonOperationError("Common", "Field $.name in " + root); + CommonError.jsonOperationError("Common", "Field $.name in " + root.toString()); Throwable cause = actual.getCause(); assertEquals(cause.getClass(), expectedCause.getClass()); assertEquals(cause.getMessage(), expectedCause.getMessage()); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java index 037ada54e46..601e9c738c6 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java @@ -19,10 +19,11 @@ package org.apache.seatunnel.format.json.canal; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonError; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.junit.jupiter.api.Test; @@ -41,8 +42,10 @@ import static org.apache.seatunnel.api.table.type.BasicType.INT_TYPE; import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; public class CanalJsonSerDeSchemaTest { + private static final String FORMAT = "Canal"; private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = new SeaTunnelRowType( @@ -66,16 +69,96 @@ public void testDeserializeNullRow() throws Exception { createCanalJsonDeserializationSchema(null, null); final SimpleCollector collector = new SimpleCollector(); - deserializationSchema.deserialize((byte[]) null, collector, TablePath.of("test")); + deserializationSchema.deserialize((byte[]) null, collector); assertEquals(0, collector.list.size()); } + @Test + public void testDeserializeNoJson() throws Exception { + final CanalJsonDeserializationSchema deserializationSchema = + createCanalJsonDeserializationSchema(null, null); + final SimpleCollector collector = new SimpleCollector(); + String noJsonMsg = "{]"; + + SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, noJsonMsg); + SeaTunnelRuntimeException cause = + assertThrows( + expected.getClass(), + () -> { + deserializationSchema.deserialize(noJsonMsg.getBytes(), collector); + }); + assertEquals(cause.getMessage(), expected.getMessage()); + } + + @Test + public void testDeserializeEmptyJson() throws Exception { + final CanalJsonDeserializationSchema deserializationSchema = + createCanalJsonDeserializationSchema(null, null); + final SimpleCollector collector = new SimpleCollector(); + String emptyMsg = "{}"; + SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, emptyMsg); + SeaTunnelRuntimeException cause = + assertThrows( + expected.getClass(), + () -> { + deserializationSchema.deserialize(emptyMsg.getBytes(), collector); + }); + assertEquals(cause.getMessage(), expected.getMessage()); + } + + @Test + public void testDeserializeNoDataJson() throws Exception { + final CanalJsonDeserializationSchema deserializationSchema = + createCanalJsonDeserializationSchema(null, null); + final SimpleCollector collector = new SimpleCollector(); + String noDataMsg = "{\"type\":\"INSERT\"}"; + SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, noDataMsg); + SeaTunnelRuntimeException cause = + assertThrows( + expected.getClass(), + () -> { + deserializationSchema.deserialize(noDataMsg.getBytes(), collector); + }); + assertEquals(cause.getMessage(), expected.getMessage()); + + Throwable noDataCause = cause.getCause(); + assertEquals(noDataCause.getClass(), IllegalStateException.class); + assertEquals( + noDataCause.getMessage(), + String.format("Null data value '%s' Cannot send downstream", noDataMsg)); + } + + @Test + public void testDeserializeUnknownTypeJson() throws Exception { + final CanalJsonDeserializationSchema deserializationSchema = + createCanalJsonDeserializationSchema(null, null); + final SimpleCollector collector = new SimpleCollector(); + String unknownType = "XX"; + String unknownOperationMsg = + "{\"data\":{\"id\":101,\"name\":\"scooter\"},\"type\":\"" + unknownType + "\"}"; + SeaTunnelRuntimeException expected = + CommonError.jsonOperationError(FORMAT, unknownOperationMsg); + SeaTunnelRuntimeException cause = + assertThrows( + expected.getClass(), + () -> { + deserializationSchema.deserialize( + unknownOperationMsg.getBytes(), collector); + }); + assertEquals(cause.getMessage(), expected.getMessage()); + + Throwable unknownTypeCause = cause.getCause(); + assertEquals(unknownTypeCause.getClass(), IllegalStateException.class); + assertEquals( + unknownTypeCause.getMessage(), + String.format("Unknown operation type '%s'.", unknownType)); + } + public void runTest(List lines, CanalJsonDeserializationSchema deserializationSchema) throws IOException { SimpleCollector collector = new SimpleCollector(); for (String line : lines) { - deserializationSchema.deserialize( - line.getBytes(StandardCharsets.UTF_8), collector, TablePath.of("")); + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); } List expected = diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index f9e824fcddb..e74e58e19e0 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -19,7 +19,6 @@ package org.apache.seatunnel.format.json.debezium; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -48,7 +47,6 @@ public class DebeziumJsonSerDeSchemaTest { private static final String FORMAT = "Debezium"; - private static final String FILE_DATA_NAME = "debezium-data.txt"; private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = new SeaTunnelRowType( @@ -68,11 +66,11 @@ void testNullRowMessages() throws Exception { @Test public void testSerializationAndSchemaExcludeDeserialization() throws Exception { - testSerializationDeserialization(false); + testSerializationDeserialization("debezium-data.txt", false); } @Test - public void testDeserializeNoJson() { + public void testDeserializeNoJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); final SimpleCollector collector = new SimpleCollector(); @@ -90,7 +88,7 @@ public void testDeserializeNoJson() { } @Test - public void testDeserializeEmptyJson() { + public void testDeserializeEmptyJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); final SimpleCollector collector = new SimpleCollector(); @@ -100,14 +98,13 @@ public void testDeserializeEmptyJson() { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize( - emptyMsg.getBytes(), collector, TablePath.of("")); + deserializationSchema.deserialize(emptyMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); } @Test - public void testDeserializeNoDataJson() { + public void testDeserializeNoDataJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); final SimpleCollector collector = new SimpleCollector(); @@ -117,8 +114,7 @@ public void testDeserializeNoDataJson() { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize( - noDataMsg.getBytes(), collector, TablePath.of("test")); + deserializationSchema.deserialize(noDataMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); @@ -134,7 +130,7 @@ public void testDeserializeNoDataJson() { } @Test - public void testDeserializeUnknownOperationTypeJson() { + public void testDeserializeUnknownOperationTypeJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); final SimpleCollector collector = new SimpleCollector(); @@ -161,8 +157,9 @@ public void testDeserializeUnknownOperationTypeJson() { String.format("Unknown operation type '%s'.", unknownType)); } - private void testSerializationDeserialization(boolean schemaInclude) throws Exception { - List lines = readLines(FILE_DATA_NAME); + private void testSerializationDeserialization(String resourceFile, boolean schemaInclude) + throws Exception { + List lines = readLines(resourceFile); DebeziumJsonDeserializationSchema deserializationSchema = new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, true, schemaInclude); @@ -243,7 +240,7 @@ private static List readLines(String resource) throws IOException { private static class SimpleCollector implements Collector { - private final List list = new ArrayList<>(); + private List list = new ArrayList<>(); @Override public void collect(SeaTunnelRow record) { diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java index 749dd234dfc..1f44c7cfbcb 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java @@ -19,7 +19,6 @@ package org.apache.seatunnel.format.json.ogg; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -48,7 +47,6 @@ public class OggJsonSerDeSchemaTest { private static final String FORMAT = "Ogg"; - private static final String RESOURCE_OGG_DATA_NAME = "ogg-data-filter-table.txt"; private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = new SeaTunnelRowType( @@ -57,7 +55,7 @@ public class OggJsonSerDeSchemaTest { @Test public void testFilteringTables() throws Exception { - List lines = readLines(); + List lines = readLines("ogg-data-filter-table.txt"); OggJsonDeserializationSchema deserializationSchema = new OggJsonDeserializationSchema.Builder(PHYSICAL_DATA_TYPE) .setDatabase("^OG.*") @@ -104,8 +102,7 @@ public void testDeserializeEmptyJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize( - emptyMsg.getBytes(), collector, TablePath.of("")); + deserializationSchema.deserialize(emptyMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); } @@ -121,8 +118,7 @@ public void testDeserializeNoDataJson() throws Exception { assertThrows( expected.getClass(), () -> { - deserializationSchema.deserialize( - noDataMsg.getBytes(), collector, TablePath.of("test")); + deserializationSchema.deserialize(noDataMsg.getBytes(), collector); }); assertEquals(cause.getMessage(), expected.getMessage()); @@ -243,9 +239,8 @@ private OggJsonDeserializationSchema createOggJsonDeserializationSchema( .build(); } - private static List readLines() throws IOException { - final URL url = - OggJsonSerDeSchemaTest.class.getClassLoader().getResource(RESOURCE_OGG_DATA_NAME); + private static List readLines(String resource) throws IOException { + final URL url = OggJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); Assertions.assertNotNull(url); Path path = new File(url.getFile()).toPath(); return Files.readAllLines(path); @@ -253,7 +248,7 @@ private static List readLines() throws IOException { private static class SimpleCollector implements Collector { - private final List list = new ArrayList<>(); + private List list = new ArrayList<>(); @Override public void collect(SeaTunnelRow record) { From 101206297f08080a43636583e51734c6506729cb Mon Sep 17 00:00:00 2001 From: zhilinli Date: Thu, 7 Mar 2024 00:12:55 +0800 Subject: [PATCH 36/71] fix --- .../source/format/PulsarCanalDecorator.java | 13 +++++- .../canal/CanalJsonDeserializationSchema.java | 45 +++++++++---------- 2 files changed, 33 insertions(+), 25 deletions(-) diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java index 46642f3ad65..5c1830f743a 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java @@ -64,6 +64,17 @@ public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOEx @Override public void deserialize(byte[] message, Collector out) throws IOException { + deserializeMessage(message, out, null); + } + + public void deserialize(byte[] message, Collector out, TablePath tablePath) + throws IOException { + + deserializeMessage(message, out, tablePath); + } + + public void deserializeMessage(byte[] message, Collector out, TablePath tablePath) + throws IOException { JsonNode pulsarCanal = JsonUtils.parseObject(message); ArrayNode canalList = JsonUtils.parseArray(pulsarCanal.get(MESSAGE).asText()); Iterator canalIterator = canalList.elements(); @@ -73,7 +84,7 @@ public void deserialize(byte[] message, Collector out) throws IOEx // https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/MessageUtils.java ObjectNode root = reconvertPulsarData((ObjectNode) next); // todo Wait for TablePath functionality to be added - canalJsonDeserializationSchema.deserialize(root, out, TablePath.of("")); + canalJsonDeserializationSchema.deserialize(root, out, tablePath); } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java index 7370b270880..b9dd19c285d 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java @@ -122,24 +122,8 @@ public SeaTunnelDataType getProducedType() { return this.physicalRowType; } - public void deserializeMessage( - byte[] message, Collector out, TablePath tablePath) { - if (message == null || message.length == 0) { - // skip tombstone messages - return; - } - - ObjectNode jsonNode; - try { - jsonNode = convertBytes(message); - } catch (SeaTunnelRuntimeException cause) { - if (!ignoreParseErrors) { - throw cause; - } else { - return; - } - } - + public void deserialize(ObjectNode jsonNode, Collector out, TablePath tablePath) + throws IOException { try { if (database != null && !databasePattern.matcher(jsonNode.get(FIELD_DATABASE).asText()).matches()) { @@ -211,7 +195,6 @@ public void deserializeMessage( throw new IllegalStateException( String.format("Unknown operation type '%s'.", op)); } - } catch (RuntimeException e) { if (!ignoreParseErrors) { throw CommonError.jsonOperationError(FORMAT, jsonNode.toString(), e); @@ -220,21 +203,35 @@ public void deserializeMessage( } private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException { + if (message == null || message.length == 0) { + return null; + } + try { return (ObjectNode) jsonDeserializer.deserializeToJsonNode(message); } catch (Throwable t) { - throw CommonError.jsonOperationError(FORMAT, new String(message), t); + if (!ignoreParseErrors) { + throw CommonError.jsonOperationError(FORMAT, new String(message), t); + } + return null; } } @Override - public void deserialize(byte[] message, Collector out) { - deserializeMessage(message, out, null); + public void deserialize(byte[] message, Collector out) throws IOException { + ObjectNode jsonNodes = convertBytes(message); + if (jsonNodes != null) { + deserialize(convertBytes(message), out, null); + } } @Override - public void deserialize(byte[] message, Collector out, TablePath tablePath) { - deserializeMessage(message, out, tablePath); + public void deserialize(byte[] message, Collector out, TablePath tablePath) + throws IOException { + ObjectNode jsonNodes = convertBytes(message); + if (jsonNodes != null) { + deserialize(convertBytes(message), out, tablePath); + } } private SeaTunnelRow convertJsonNode(JsonNode root) { From 42a251d8530b9dfc7146d532fd8b8cf4d37bacff Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 12 Mar 2024 19:37:55 +0800 Subject: [PATCH 37/71] fix --- .../seatunnel/file/source/reader/TextReadStrategy.java | 4 +--- .../pulsar/source/format/PulsarCanalDecorator.java | 1 - .../seatunnel/redis/source/RedisSourceReader.java | 5 +---- .../seatunnel/rocketmq/source/RocketMqSourceReader.java | 5 +---- .../json/CompatibleKafkaConnectDeserializationSchema.java | 8 ++++++-- 5 files changed, 9 insertions(+), 14 deletions(-) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/TextReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/TextReadStrategy.java index 3a8406044b2..c3a0315fe0b 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/TextReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/TextReadStrategy.java @@ -22,7 +22,6 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -88,8 +87,7 @@ public void read(String path, String tableId, Collector output) line -> { try { SeaTunnelRow seaTunnelRow = - deserializationSchema.deserialize( - line.getBytes(), TablePath.of(tableId)); + deserializationSchema.deserialize(line.getBytes()); if (!readColumns.isEmpty()) { // need column projection Object[] fields; diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java index 5c1830f743a..f46e01f4e43 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java @@ -69,7 +69,6 @@ public void deserialize(byte[] message, Collector out) throws IOEx public void deserialize(byte[] message, Collector out, TablePath tablePath) throws IOException { - deserializeMessage(message, out, tablePath); } diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java index 9aeb7daa69e..7dcea693fa3 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceReader.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -87,9 +86,7 @@ public void internalPollNext(Collector output) throws Exception { deserializationSchema.getProducedType(); valuesMap.put(((SeaTunnelRowType) seaTunnelRowType).getFieldName(0), k); deserializationSchema.deserialize( - JsonUtils.toJsonString(valuesMap).getBytes(), - output, - TablePath.of("")); + JsonUtils.toJsonString(valuesMap).getBytes(), output); } } else { deserializationSchema.deserialize(value.getBytes(), output); diff --git a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java index 9019fb82a64..fd4f9860724 100644 --- a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java +++ b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java @@ -24,7 +24,6 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.rocketmq.exception.RocketMqConnectorErrorCode; import org.apache.seatunnel.connectors.seatunnel.rocketmq.exception.RocketMqConnectorException; @@ -162,9 +161,7 @@ record -> groupRecords.get(messageQueue); for (MessageExt record : messages) { deserializationSchema.deserialize( - record.getBody(), - output, - TablePath.of("")); + record.getBody(), output); if (Boundedness.BOUNDED.equals( context.getBoundedness()) && record.getQueueOffset() diff --git a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java index 57384cb2803..a0d82c57949 100644 --- a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java @@ -124,13 +124,17 @@ public void deserialize( for (int i = 0; i < arrayNode.size(); i++) { SeaTunnelRow row = convertJsonNode(arrayNode.get(i)); row.setRowKind(rowKind); - row.setTableId(tablePath.toString()); + if (tablePath != null && !tablePath.toString().isEmpty()) { + row.setTableId(tablePath.toString()); + } out.collect(row); } } else { SeaTunnelRow row = convertJsonNode(payload); row.setRowKind(rowKind); - row.setTableId(tablePath.toString()); + if (tablePath != null && !tablePath.toString().isEmpty()) { + row.setTableId(tablePath.toString()); + } out.collect(row); } } From 07cb3e92cce88749f335c2a4769b645256c795e2 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 9 Apr 2024 23:31:42 +0800 Subject: [PATCH 38/71] fix --- docs/en/connector-v2/source/kafka.md | 8 ++-- .../serialization/DeserializationSchema.java | 19 --------- .../fake/source/FakeDataGenerator.java | 4 +- .../seatunnel/kafka/config/Config.java | 6 +-- .../kafka/source/KafkaSourceConfig.java | 21 +++++++--- .../kafka/source/KafkaSourceFactory.java | 2 +- .../kafka/source/KafkaSourceReader.java | 7 +--- .../e2e/connector/kafka/KafkaIT.java | 4 +- .../kafka_multi_source_to_assert.conf | 2 +- .../avro/AvroDeserializationSchema.java | 20 +++++----- .../avro/AvroSerializationSchemaTest.java | 3 +- ...ibleKafkaConnectDeserializationSchema.java | 22 +++++------ ...ibleDebeziumJsonDeserializationSchema.java | 6 --- .../json/JsonDeserializationSchema.java | 28 ++++++++----- .../canal/CanalJsonDeserializationSchema.java | 34 ++++++++-------- .../DebeziumJsonDeserializationSchema.java | 37 +++++++++--------- .../ogg/OggJsonDeserializationSchema.java | 39 ++++++++++--------- .../json/JsonRowDataSerDeSchemaTest.java | 24 +++++++----- .../debezium/DebeziumJsonSerDeSchemaTest.java | 3 +- .../text/TextDeserializationSchema.java | 31 ++++++++++----- 20 files changed, 164 insertions(+), 156 deletions(-) diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index 7119a191b62..631415effcc 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -35,7 +35,7 @@ They can be downloaded via install-plugin.sh or from the Maven central repositor | Name | Type | Required | Default | Description | |-------------------------------------|-----------------------------------------------------------------------------|----------|--------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | topic | String | Yes | - | 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 comma like 'topic-1,topic-2'. | -| topic_list | Map | No | - | Topic list config You can configure only one `topic_list` and one `topic` at the same time | +| table_list | Map | No | - | Topic list config You can configure only one `table_list` and one `topic` at the same time | | bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | | pattern | Boolean | No | false | If `pattern` is set to `true`,the regular expression for a pattern of topic names to read from. All topics in clients with names that match the specified regular expression will be subscribed by the consumer. | | consumer.group | String | No | SeaTunnel-Consumer-Group | `Kafka consumer group id`, used to distinguish different consumer groups. | @@ -183,7 +183,7 @@ source { ### Multiple Kafka Source -> Currently, multiple kafka source reads are supported using the Zeta engine, but note that you can only configure one instance of `bootstrap.servers`, and only one in the `topic_list` and `topic` parameters,Currently, when you use multiple topics in `topic_list`, you are still free to set parameters for each `topic` +> Currently, multiple kafka source reads are supported using the Zeta engine, but note that you can only configure one instance of `bootstrap.servers`, and only one in the `table_list` and `topic` parameters,Currently, when you use multiple topics in `table_list`, you are still free to set parameters for each `topic` ```hocon @@ -195,7 +195,7 @@ env { source { Kafka { bootstrap.servers = "kafka_e2e:9092" - topic_list = [ + table_list = [ { topic = "^test-ogg-sou.*" pattern = "true" @@ -225,7 +225,7 @@ source { format = canal_json } ] - # Each topic in the `topic_list` shares the config + # Each topic in the `table_list` shares the config kafka.config = { client.id = client_1 max.poll.records = 50000 diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java index 6cbce2ea91a..d7f7abb1a51 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/serialization/DeserializationSchema.java @@ -18,7 +18,6 @@ package org.apache.seatunnel.api.serialization; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import java.io.IOException; @@ -35,16 +34,6 @@ public interface DeserializationSchema extends Serializable { */ T deserialize(byte[] message) throws IOException; - /** - * Deserializes the byte message. - * - * @param message The message, as a byte array. - * @param tablePath The Registry name. - * @return The deserialized message as an SeaTunnel Row (null if the message cannot be - * deserialized). - */ - T deserialize(byte[] message, TablePath tablePath) throws IOException; - default void deserialize(byte[] message, Collector out) throws IOException { T deserialize = deserialize(message); if (deserialize != null) { @@ -52,13 +41,5 @@ default void deserialize(byte[] message, Collector out) throws IOException { } } - default void deserialize(byte[] message, Collector out, TablePath tablePath) - throws IOException { - T deserialize = deserialize(message, tablePath); - if (deserialize != null) { - out.collect(deserialize); - } - } - SeaTunnelDataType getProducedType(); } diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java index 8bacb1a94d5..ef5efc8164f 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java @@ -44,7 +44,7 @@ public class FakeDataGenerator { private final FakeConfig fakeConfig; private final JsonDeserializationSchema jsonDeserializationSchema; private final FakeDataRandomUtils fakeDataRandomUtils; - private String tableId; + private final String tableId; public FakeDataGenerator(FakeConfig fakeConfig) { this.catalogTable = fakeConfig.getCatalogTable(); @@ -54,7 +54,7 @@ public FakeDataGenerator(FakeConfig fakeConfig) { fakeConfig.getFakeRows() == null ? null : new JsonDeserializationSchema( - false, false, catalogTable.getSeaTunnelRowType()); + false, false, catalogTable.getSeaTunnelRowType(), catalogTable); this.fakeDataRandomUtils = new FakeDataRandomUtils(fakeConfig); } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java index 5d74c3d2dfe..2f665c33cee 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java @@ -177,10 +177,10 @@ public class Config { .defaultValue(KafkaSemantics.NON) .withDescription( "Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON."); - public static final Option>> TOPIC_LIST = - Options.key("topic_list") + public static final Option>> TABLE_LIST = + Options.key("table_list") .type(new TypeReference>>() {}) .noDefaultValue() .withDescription( - "Topic list config You can configure only one `topic_list` and one `topic` at the same time"); + "Topic list config You can configure only one `table_list` and one `topic` at the same time"); } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index 541fc4294fc..90b0bbde541 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -105,9 +105,9 @@ private Properties createKafkaProperties(ReadonlyConfig readonlyConfig) { private Map createMapConsumerMetadata( ReadonlyConfig readonlyConfig) { List consumerMetadataList; - if (readonlyConfig.getOptional(Config.TOPIC_LIST).isPresent()) { + if (readonlyConfig.getOptional(Config.TABLE_LIST).isPresent()) { consumerMetadataList = - readonlyConfig.get(Config.TOPIC_LIST).stream() + readonlyConfig.get(Config.TABLE_LIST).stream() .map(ReadonlyConfig::fromMap) .map(config -> createConsumerMetadata(config)) .collect(Collectors.toList()); @@ -231,13 +231,14 @@ private DeserializationSchema createDeserializationSchema( return TextDeserializationSchema.builder() .seaTunnelRowType(seaTunnelRowType) .delimiter(TextFormatConstant.PLACEHOLDER) + .setCatalogTable(catalogTable) .build(); } MessageFormat format = readonlyConfig.get(FORMAT); switch (format) { case JSON: - return new JsonDeserializationSchema(false, false, seaTunnelRowType); + return new JsonDeserializationSchema(false, false, seaTunnelRowType, catalogTable); case TEXT: String delimiter = readonlyConfig.get(FIELD_DELIMITER); return TextDeserializationSchema.builder() @@ -247,10 +248,12 @@ private DeserializationSchema createDeserializationSchema( case CANAL_JSON: return CanalJsonDeserializationSchema.builder(seaTunnelRowType) .setIgnoreParseErrors(true) + .setCatalogTable(catalogTable) .build(); case OGG_JSON: return OggJsonDeserializationSchema.builder(seaTunnelRowType) .setIgnoreParseErrors(true) + .setCatalogTable(catalogTable) .build(); case COMPATIBLE_KAFKA_CONNECT_JSON: Boolean keySchemaEnable = @@ -260,12 +263,18 @@ private DeserializationSchema createDeserializationSchema( readonlyConfig.get( KafkaConnectJsonFormatOptions.VALUE_CONVERTER_SCHEMA_ENABLED); return new CompatibleKafkaConnectDeserializationSchema( - seaTunnelRowType, keySchemaEnable, valueSchemaEnable, false, false); + seaTunnelRowType, + keySchemaEnable, + valueSchemaEnable, + false, + false, + catalogTable); case DEBEZIUM_JSON: boolean includeSchema = readonlyConfig.get(DEBEZIUM_RECORD_INCLUDE_SCHEMA); - return new DebeziumJsonDeserializationSchema(seaTunnelRowType, true, includeSchema); + return new DebeziumJsonDeserializationSchema( + seaTunnelRowType, true, includeSchema, catalogTable); case AVRO: - return new AvroDeserializationSchema(seaTunnelRowType); + return new AvroDeserializationSchema(seaTunnelRowType, catalogTable); default: throw new SeaTunnelJsonFormatException( CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java index 7ba63479cd7..431e9a8c195 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java @@ -43,7 +43,7 @@ public String factoryIdentifier() { public OptionRule optionRule() { return OptionRule.builder() .required(Config.BOOTSTRAP_SERVERS) - .exclusive(Config.TOPIC, Config.TOPIC_LIST) + .exclusive(Config.TOPIC, Config.TABLE_LIST) .optional( Config.START_MODE, Config.PATTERN, diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java index f4fc032d9be..d136fabc403 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java @@ -158,13 +158,10 @@ public void pollNext(Collector output) throws Exception { ((CompatibleKafkaConnectDeserializationSchema) deserializationSchema) .deserialize( - record, output, - tablePath); + record, output); } else { deserializationSchema.deserialize( - record.value(), - output, - tablePath); + record.value(), output); } } catch (IOException e) { if (this.messageFormatErrorHandleWay diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index d86aa891865..591d09af38f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -371,7 +371,7 @@ public void testFakeSourceToKafkaAvroFormat(TestContainer container) }; SeaTunnelRowType fake_source_row_type = new SeaTunnelRowType(fieldNames, fieldTypes); AvroDeserializationSchema avroDeserializationSchema = - new AvroDeserializationSchema(fake_source_row_type); + new AvroDeserializationSchema(fake_source_row_type, null); List kafkaSTRow = getKafkaSTRow( "test_avro_topic_fake_source", @@ -421,7 +421,7 @@ public void testKafkaAvroToAssert(TestContainer container) Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); AvroDeserializationSchema avroDeserializationSchema = - new AvroDeserializationSchema(SEATUNNEL_ROW_TYPE); + new AvroDeserializationSchema(SEATUNNEL_ROW_TYPE, null); List kafkaSTRow = getKafkaSTRow( "test_avro_topic", diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf index 4bea3f4d0b4..165249d754f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf @@ -26,7 +26,7 @@ env { source { Kafka { bootstrap.servers = "kafka_e2e:9092" - topic_list = [ + table_list = [ { topic = "^test-ogg-sou.*" pattern = "true" diff --git a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java index 3e53c4847c6..47b3803cb06 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.format.avro; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -28,6 +29,7 @@ import org.apache.avro.io.DecoderFactory; import java.io.IOException; +import java.util.Optional; public class AvroDeserializationSchema implements DeserializationSchema { @@ -35,24 +37,24 @@ public class AvroDeserializationSchema implements DeserializationSchema) instead."); } - @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - throw new UnsupportedOperationException( - "Please invoke DeserializationSchema#deserialize(byte[],TablePath, Collector) instead."); - } - /** * Deserialize kafka consumer record * @@ -105,8 +105,7 @@ public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOEx * @param out * @throws Exception */ - public void deserialize( - ConsumerRecord msg, Collector out, TablePath tablePath) + public void deserialize(ConsumerRecord msg, Collector out) throws InvocationTargetException, IllegalAccessException { tryInitConverter(); if (msg == null) { @@ -119,12 +118,13 @@ public void deserialize( valueConverterMethod.invoke( valueConverter, record.valueSchema(), record.value()); JsonNode payload = jsonNode.get(KAFKA_CONNECT_SINK_RECORD_PAYLOAD); + TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); if (payload.isArray()) { ArrayNode arrayNode = (ArrayNode) payload; for (int i = 0; i < arrayNode.size(); i++) { SeaTunnelRow row = convertJsonNode(arrayNode.get(i)); row.setRowKind(rowKind); - if (tablePath != null && !tablePath.toString().isEmpty()) { + if (tablePath != null) { row.setTableId(tablePath.toString()); } out.collect(row); @@ -132,7 +132,7 @@ public void deserialize( } else { SeaTunnelRow row = convertJsonNode(payload); row.setRowKind(rowKind); - if (tablePath != null && !tablePath.toString().isEmpty()) { + if (tablePath != null) { row.setTableId(tablePath.toString()); } out.collect(row); diff --git a/seatunnel-formats/seatunnel-format-compatible-debezium-json/src/main/java/org/apache/seatunnel/format/compatible/debezium/json/CompatibleDebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-debezium-json/src/main/java/org/apache/seatunnel/format/compatible/debezium/json/CompatibleDebeziumJsonDeserializationSchema.java index 684bd999ba7..a38f19c0e3d 100644 --- a/seatunnel-formats/seatunnel-format-compatible-debezium-json/src/main/java/org/apache/seatunnel/format/compatible/debezium/json/CompatibleDebeziumJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-debezium-json/src/main/java/org/apache/seatunnel/format/compatible/debezium/json/CompatibleDebeziumJsonDeserializationSchema.java @@ -19,7 +19,6 @@ package org.apache.seatunnel.format.compatible.debezium.json; import org.apache.seatunnel.api.serialization.DeserializationSchema; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -56,11 +55,6 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { throw new UnsupportedEncodingException(); } - @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - throw new UnsupportedEncodingException(); - } - public SeaTunnelRow deserialize(SourceRecord record) throws InvocationTargetException, IllegalAccessException { String key = debeziumJsonConverter.serializeKey(record); diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 82c7befe48d..cf2e3a4c1a6 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -28,6 +28,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.CompositeType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -39,6 +40,7 @@ import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; import java.io.IOException; +import java.util.Optional; import static com.google.common.base.Preconditions.checkNotNull; @@ -64,8 +66,13 @@ public class JsonDeserializationSchema implements DeserializationSchema dataType) { @@ -107,15 +115,6 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { return convertJsonNode(convertBytes(message)); } - @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - SeaTunnelRow seaTunnelRow = deserialize(message); - if (tablePath != null && !tablePath.toString().isEmpty()) { - seaTunnelRow.setTableId(tablePath.toString()); - } - return seaTunnelRow; - } - public SeaTunnelRow deserialize(String message) throws IOException { if (message == null) { return null; @@ -129,14 +128,23 @@ public void collect(byte[] message, Collector out) throws IOExcept ArrayNode arrayNode = (ArrayNode) jsonNode; for (int i = 0; i < arrayNode.size(); i++) { SeaTunnelRow deserialize = convertJsonNode(arrayNode.get(i)); + setCollectorTablePath(deserialize, catalogTable); out.collect(deserialize); } } else { SeaTunnelRow deserialize = convertJsonNode(jsonNode); + setCollectorTablePath(deserialize, catalogTable); out.collect(deserialize); } } + public void setCollectorTablePath(SeaTunnelRow deserialize, CatalogTable catalogTable) { + TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); + if (tablePath != null) { + deserialize.setTableId(tablePath.toString()); + } + } + private SeaTunnelRow convertJsonNode(JsonNode jsonNode) { if (jsonNode.isNull()) { return null; diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java index b9dd19c285d..2affe3f137b 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -34,6 +35,7 @@ import org.apache.seatunnel.format.json.JsonDeserializationSchema; import java.io.IOException; +import java.util.Optional; import java.util.regex.Pattern; import static java.lang.String.format; @@ -86,16 +88,18 @@ public class CanalJsonDeserializationSchema implements DeserializationSchema) instead."); } - @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - throw new UnsupportedOperationException( - "Please invoke DeserializationSchema#deserialize(byte[],TablePath tablePath , Collector) instead."); - } - @Override public SeaTunnelDataType getProducedType() { return this.physicalRowType; @@ -221,15 +220,7 @@ private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException public void deserialize(byte[] message, Collector out) throws IOException { ObjectNode jsonNodes = convertBytes(message); if (jsonNodes != null) { - deserialize(convertBytes(message), out, null); - } - } - - @Override - public void deserialize(byte[] message, Collector out, TablePath tablePath) - throws IOException { - ObjectNode jsonNodes = convertBytes(message); - if (jsonNodes != null) { + TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); deserialize(convertBytes(message), out, tablePath); } } @@ -262,6 +253,8 @@ public static class Builder { private final SeaTunnelRowType physicalDataType; + private CatalogTable catalogTable; + public Builder(SeaTunnelRowType physicalDataType) { this.physicalDataType = physicalDataType; } @@ -281,9 +274,14 @@ public Builder setIgnoreParseErrors(boolean ignoreParseErrors) { return this; } + public Builder setCatalogTable(CatalogTable catalogTable) { + this.catalogTable = catalogTable; + return this; + } + public CanalJsonDeserializationSchema build() { return new CanalJsonDeserializationSchema( - physicalDataType, database, table, ignoreParseErrors); + physicalDataType, database, table, ignoreParseErrors, catalogTable); } } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java index 1cb2616e057..6f86169d710 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -30,6 +31,7 @@ import org.apache.seatunnel.format.json.JsonDeserializationSchema; import java.io.IOException; +import java.util.Optional; import static java.lang.String.format; @@ -58,23 +60,31 @@ public class DebeziumJsonDeserializationSchema implements DeserializationSchema< private final boolean debeziumEnabledSchema; + private CatalogTable catalogTable; + public DebeziumJsonDeserializationSchema(SeaTunnelRowType rowType, boolean ignoreParseErrors) { this.rowType = rowType; this.ignoreParseErrors = ignoreParseErrors; this.jsonDeserializer = - new JsonDeserializationSchema(false, ignoreParseErrors, createJsonRowType(rowType)); + new JsonDeserializationSchema( + false, ignoreParseErrors, createJsonRowType(rowType), null); this.debeziumRowConverter = new DebeziumRowConverter(rowType); this.debeziumEnabledSchema = false; } public DebeziumJsonDeserializationSchema( - SeaTunnelRowType rowType, boolean ignoreParseErrors, boolean debeziumEnabledSchema) { + SeaTunnelRowType rowType, + boolean ignoreParseErrors, + boolean debeziumEnabledSchema, + CatalogTable catalogTable) { this.rowType = rowType; this.ignoreParseErrors = ignoreParseErrors; this.jsonDeserializer = - new JsonDeserializationSchema(false, ignoreParseErrors, createJsonRowType(rowType)); + new JsonDeserializationSchema( + false, ignoreParseErrors, createJsonRowType(rowType), catalogTable); this.debeziumRowConverter = new DebeziumRowConverter(rowType); this.debeziumEnabledSchema = debeziumEnabledSchema; + this.catalogTable = catalogTable; } @Override @@ -83,20 +93,9 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); } - @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - throw new UnsupportedOperationException( - "Please invoke DeserializationSchema#deserialize(byte[],TablePath tablePath,Collector) instead."); - } - @Override public void deserialize(byte[] message, Collector out) { - deserializeMessage(message, out, null); - } - - @Override - public void deserialize(byte[] message, Collector out, TablePath tablePath) - throws IOException { + TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); deserializeMessage(message, out, tablePath); } @@ -116,7 +115,7 @@ private void deserializeMessage( case OP_READ: SeaTunnelRow insert = convertJsonNode(payload.get("after")); insert.setRowKind(RowKind.INSERT); - if (tablePath != null && !tablePath.toString().isEmpty()) { + if (tablePath != null) { insert.setTableId(tablePath.toString()); } out.collect(insert); @@ -128,7 +127,7 @@ private void deserializeMessage( String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); } before.setRowKind(RowKind.UPDATE_BEFORE); - if (tablePath != null && !tablePath.toString().isEmpty()) { + if (tablePath != null) { before.setTableId(tablePath.toString()); } out.collect(before); @@ -136,7 +135,7 @@ private void deserializeMessage( SeaTunnelRow after = convertJsonNode(payload.get("after")); after.setRowKind(RowKind.UPDATE_AFTER); - if (tablePath != null && !tablePath.toString().isEmpty()) { + if (tablePath != null) { after.setTableId(tablePath.toString()); } out.collect(after); @@ -148,7 +147,7 @@ private void deserializeMessage( String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); } delete.setRowKind(RowKind.DELETE); - if (tablePath != null && !tablePath.toString().isEmpty()) { + if (tablePath != null) { delete.setTableId(tablePath.toString()); } out.collect(delete); diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java index 972c1b03e22..cebf2f494ef 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -33,6 +34,7 @@ import org.apache.seatunnel.format.json.JsonDeserializationSchema; import java.io.IOException; +import java.util.Optional; import java.util.regex.Pattern; public class OggJsonDeserializationSchema implements DeserializationSchema { @@ -82,15 +84,18 @@ public class OggJsonDeserializationSchema implements DeserializationSchema) instead."); } - @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - throw new UnsupportedOperationException( - "Please invoke DeserializationSchema#deserialize(byte[],TablePath tablePath,Collector) instead."); - } - @Override public SeaTunnelDataType getProducedType() { return this.physicalRowType; @@ -157,7 +157,7 @@ public void deserializeMessage( // Gets the data for the INSERT operation JsonNode dataInsert = jsonNode.get(DATA_AFTER); SeaTunnelRow row = convertJsonNode(dataInsert); - if (tablePath != null && !tablePath.toString().isEmpty()) { + if (tablePath != null) { row.setTableId(tablePath.toString()); } out.collect(row); @@ -175,13 +175,13 @@ public void deserializeMessage( // Gets the data for the UPDATE AFTER operation SeaTunnelRow after = convertJsonNode(dataAfter); before.setRowKind(RowKind.UPDATE_BEFORE); - if (tablePath != null && !tablePath.toString().isEmpty()) { + if (tablePath != null) { before.setTableId(tablePath.toString()); } out.collect(before); after.setRowKind(RowKind.UPDATE_AFTER); - if (tablePath != null && !tablePath.toString().isEmpty()) { + if (tablePath != null) { after.setTableId(tablePath.toString()); } out.collect(after); @@ -199,7 +199,7 @@ public void deserializeMessage( String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); } beforeDelete.setRowKind(RowKind.DELETE); - if (tablePath != null && !tablePath.toString().isEmpty()) { + if (tablePath != null) { beforeDelete.setTableId(tablePath.toString()); } out.collect(beforeDelete); @@ -226,11 +226,7 @@ private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException @Override public void deserialize(byte[] message, Collector out) { - deserializeMessage(message, out, null); - } - - @Override - public void deserialize(byte[] message, Collector out, TablePath tablePath) { + TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); deserializeMessage(message, out, tablePath); } @@ -262,6 +258,8 @@ public static class Builder { private final SeaTunnelRowType physicalDataType; + private CatalogTable catalogTable; + public Builder(SeaTunnelRowType physicalDataType) { this.physicalDataType = physicalDataType; } @@ -281,9 +279,14 @@ public Builder setIgnoreParseErrors(boolean ignoreParseErrors) { return this; } + public Builder setCatalogTable(CatalogTable catalogTable) { + this.catalogTable = catalogTable; + return this; + } + public OggJsonDeserializationSchema build() { return new OggJsonDeserializationSchema( - physicalDataType, database, table, ignoreParseErrors); + physicalDataType, database, table, ignoreParseErrors, catalogTable); } } } diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java index 7a816258115..94fb5f05ff2 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java @@ -135,7 +135,7 @@ public void testSerDe() throws Exception { }); JsonDeserializationSchema deserializationSchema = - new JsonDeserializationSchema(false, false, schema); + new JsonDeserializationSchema(false, false, schema, null); SeaTunnelRow expected = new SeaTunnelRow(12); expected.setField(0, true); @@ -178,7 +178,7 @@ public void testSerDeMultiRows() throws Exception { }); JsonDeserializationSchema deserializationSchema = - new JsonDeserializationSchema(false, false, schema); + new JsonDeserializationSchema(false, false, schema, null); JsonSerializationSchema serializationSchema = new JsonSerializationSchema(schema); ObjectMapper objectMapper = new ObjectMapper(); @@ -254,7 +254,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { }); JsonDeserializationSchema deserializationSchema = - new JsonDeserializationSchema(false, true, rowType); + new JsonDeserializationSchema(false, true, rowType, null); JsonSerializationSchema serializationSchema = new JsonSerializationSchema(rowType); for (int i = 0; i < jsons.length; i++) { @@ -270,7 +270,7 @@ public void testDeserializationNullRow() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); JsonDeserializationSchema deserializationSchema = - new JsonDeserializationSchema(true, false, schema); + new JsonDeserializationSchema(true, false, schema, null); String s = null; assertNull(deserializationSchema.deserialize(s)); } @@ -281,7 +281,7 @@ public void testDeserializationMissingNode() throws Exception { new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); JsonDeserializationSchema deserializationSchema = - new JsonDeserializationSchema(true, false, schema); + new JsonDeserializationSchema(true, false, schema, null); SeaTunnelRow rowData = deserializationSchema.deserialize("".getBytes()); assertEquals(null, rowData); } @@ -299,7 +299,8 @@ public void testDeserializationPassMissingField() throws Exception { new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); // pass on missing field - final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, false, schema); + final JsonDeserializationSchema deser = + new JsonDeserializationSchema(false, false, schema, null); SeaTunnelRow expected = new SeaTunnelRow(1); SeaTunnelRow actual = deser.deserialize(serializedJson); @@ -319,7 +320,8 @@ public void testDeserializationMissingField() throws Exception { new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); // fail on missing field - final JsonDeserializationSchema deser = new JsonDeserializationSchema(true, false, schema); + final JsonDeserializationSchema deser = + new JsonDeserializationSchema(true, false, schema, null); SeaTunnelRuntimeException expected = CommonError.jsonOperationError("Common", root.toString()); @@ -353,7 +355,8 @@ public void testDeserializationIgnoreParseError() throws Exception { SeaTunnelRow expected = new SeaTunnelRow(1); // ignore on parse error - final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, true, schema); + final JsonDeserializationSchema deser = + new JsonDeserializationSchema(false, true, schema, null); assertEquals(expected, deser.deserialize(serializedJson)); } @@ -366,7 +369,7 @@ public void testDeserializationFailOnMissingFieldIgnoreParseError() throws Excep assertThrows( SeaTunnelJsonFormatException.class, () -> { - new JsonDeserializationSchema(true, true, null); + new JsonDeserializationSchema(true, true, null, null); }, "expecting exception message: " + errorMessage); assertEquals(actual.getMessage(), errorMessage); @@ -378,7 +381,8 @@ public void testDeserializationNoJson() throws Exception { new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); String noJson = "{]"; - final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, false, schema); + final JsonDeserializationSchema deser = + new JsonDeserializationSchema(false, false, schema, null); SeaTunnelRuntimeException expected = CommonError.jsonOperationError("Common", noJson); SeaTunnelRuntimeException actual = diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index e74e58e19e0..c15be09deea 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -161,7 +161,8 @@ private void testSerializationDeserialization(String resourceFile, boolean schem throws Exception { List lines = readLines(resourceFile); DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, true, schemaInclude); + new DebeziumJsonDeserializationSchema( + PHYSICAL_DATA_TYPE, true, schemaInclude, null); SimpleCollector collector = new SimpleCollector(); diff --git a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java index 7d7c36db366..4ab8c77c74c 100644 --- a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.format.text; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; @@ -45,6 +46,7 @@ import java.util.ArrayList; import java.util.LinkedHashMap; import java.util.Map; +import java.util.Optional; public class TextDeserializationSchema implements DeserializationSchema { private final SeaTunnelRowType seaTunnelRowType; @@ -55,6 +57,8 @@ public class TextDeserializationSchema implements DeserializationSchema Date: Tue, 9 Apr 2024 23:38:27 +0800 Subject: [PATCH 39/71] fix --- .../format/text/TextDeserializationSchema.java | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java index edd2cb77980..0275d7b95ee 100644 --- a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java @@ -19,13 +19,14 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.DateUtils; import org.apache.seatunnel.common.utils.EncodingUtils; @@ -149,15 +150,7 @@ public Builder textLineSplitor(TextLineSplitor splitor) { public TextDeserializationSchema build() { return new TextDeserializationSchema( - seaTunnelRowType, separators, encoding, textLineSplitor); - seaTunnelRowType, - separators, - dateFormatter, - dateTimeFormatter, - timeFormatter, - encoding, - textLineSplitor, - catalogTable); + seaTunnelRowType, separators, encoding, textLineSplitor, catalogTable); } } From cabb092668877e6777f511fcd4353f37fd15ee1b Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 9 Apr 2024 23:38:46 +0800 Subject: [PATCH 40/71] fix --- .../json/CompatibleKafkaConnectDeserializationSchema.java | 1 - 1 file changed, 1 deletion(-) diff --git a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java index 80eec16f84e..0272c658855 100644 --- a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java @@ -90,7 +90,6 @@ public CompatibleKafkaConnectDeserializationSchema( new JsonToRowConverters(failOnMissingField, ignoreParseErrors) .createRowConverter(checkNotNull(seaTunnelRowType)); this.catalogTable = catalogTable; - } @Override From 86f5176799fc6bc923f2f882ede98f9ed97016ab Mon Sep 17 00:00:00 2001 From: zhilinli Date: Wed, 10 Apr 2024 17:48:04 +0800 Subject: [PATCH 41/71] fix --- .../amazonsqs/source/AmazonSqsSource.java | 6 ++- .../file/source/reader/JsonReadStrategy.java | 5 +- .../google/sheets/source/SheetsSource.java | 3 +- .../GoogleSheetsDeserializerTest.java | 2 +- .../seatunnel/http/source/HttpSource.java | 2 +- .../SimpleTextDeserializationSchema.java | 9 ---- .../seatunnel/pulsar/source/PulsarSource.java | 6 ++- .../source/format/PulsarCanalDecorator.java | 5 -- .../rabbitmq/source/RabbitmqSource.java | 2 +- .../seatunnel/redis/source/RedisSource.java | 2 +- .../rocketmq/source/RocketMqSource.java | 3 +- .../seatunnel-engine-examples/pom.xml | 5 ++ .../resources/examples/fake_to_console.conf | 47 +++++++++++++------ 13 files changed, 56 insertions(+), 41 deletions(-) diff --git a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java index 9022abe1f10..8e8047647d4 100644 --- a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java +++ b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java @@ -113,7 +113,8 @@ private void setDeserialization(Config config) { MessageFormat format = ReadonlyConfig.fromConfig(config).get(FORMAT); switch (format) { case JSON: - deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + deserializationSchema = + new JsonDeserializationSchema(false, false, typeInfo, null); break; case TEXT: String delimiter = DEFAULT_FIELD_DELIMITER; @@ -138,7 +139,8 @@ private void setDeserialization(Config config) { includeSchema = config.getBoolean(DEBEZIUM_RECORD_INCLUDE_SCHEMA.key()); } deserializationSchema = - new DebeziumJsonDeserializationSchema(typeInfo, true, includeSchema); + new DebeziumJsonDeserializationSchema( + typeInfo, true, includeSchema, null); break; default: throw new SeaTunnelJsonFormatException( diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java index 6c58e368721..96e0ea5fd80 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java @@ -65,10 +65,11 @@ public void setSeaTunnelRowTypeInfo(SeaTunnelRowType seaTunnelRowType) { super.setSeaTunnelRowTypeInfo(seaTunnelRowType); if (isMergePartition) { deserializationSchema = - new JsonDeserializationSchema(false, false, this.seaTunnelRowTypeWithPartition); + new JsonDeserializationSchema( + false, false, this.seaTunnelRowTypeWithPartition, null); } else { deserializationSchema = - new JsonDeserializationSchema(false, false, this.seaTunnelRowType); + new JsonDeserializationSchema(false, false, this.seaTunnelRowType, null); } } diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java index 0661f6db366..86e4b6fefd0 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java @@ -80,7 +80,8 @@ public void prepare(Config pluginConfig) throws PrepareFailException { } else { this.seaTunnelRowType = CatalogTableUtil.buildSimpleTextSchema(); } - this.deserializationSchema = new JsonDeserializationSchema(false, false, seaTunnelRowType); + this.deserializationSchema = + new JsonDeserializationSchema(false, false, seaTunnelRowType, null); } @Override diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java index 8c81a51c581..eabbf8f2f81 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java @@ -40,7 +40,7 @@ public void testJsonParseError() { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); final DeserializationSchema deser = - new JsonDeserializationSchema(false, false, schema); + new JsonDeserializationSchema(false, false, schema, null); final GoogleSheetsDeserializer googleSheetsDeser = new GoogleSheetsDeserializer(schema.getFieldNames(), deser); List row = new ArrayList<>(); diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java index 4e3f3ab1a59..33c80cb00f5 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java @@ -128,7 +128,7 @@ protected void buildSchemaWithConfig(Config pluginConfig) { case JSON: this.deserializationSchema = new JsonDeserializationSchema( - false, false, catalogTable.getSeaTunnelRowType()); + false, false, catalogTable.getSeaTunnelRowType(), null); if (pluginConfig.hasPath(HttpConfig.JSON_FIELD.key())) { jsonField = getJsonField(pluginConfig.getConfig(HttpConfig.JSON_FIELD.key())); diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java index 3a6f773f9f8..a7c73c34e01 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/SimpleTextDeserializationSchema.java @@ -18,15 +18,12 @@ package org.apache.seatunnel.connectors.seatunnel.http.source; import org.apache.seatunnel.api.serialization.DeserializationSchema; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import lombok.AllArgsConstructor; -import java.io.IOException; - @AllArgsConstructor public class SimpleTextDeserializationSchema implements DeserializationSchema { @@ -37,12 +34,6 @@ public SeaTunnelRow deserialize(byte[] message) { return new SeaTunnelRow(new Object[] {new String(message)}); } - @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - throw new UnsupportedOperationException( - "Please invoke SimpleTextDeserializationSchema#deserialize(byte[]) instead."); - } - @Override public SeaTunnelDataType getProducedType() { return rowType; diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java index db0bb24ef01..2f0552dee71 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java @@ -307,7 +307,8 @@ private void setDeserialization(Config config) { } switch (format.toUpperCase()) { case "JSON": - deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + deserializationSchema = + new JsonDeserializationSchema(false, false, typeInfo, null); break; case "CANAL_JSON": deserializationSchema = @@ -323,7 +324,8 @@ private void setDeserialization(Config config) { } } else { typeInfo = CatalogTableUtil.buildSimpleTextSchema(); - this.deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + this.deserializationSchema = + new JsonDeserializationSchema(false, false, typeInfo, null); } } diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java index f46e01f4e43..5844694c80a 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java @@ -57,11 +57,6 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { throw new UnsupportedOperationException(); } - @Override - public SeaTunnelRow deserialize(byte[] message, TablePath tablePath) throws IOException { - throw new UnsupportedOperationException(); - } - @Override public void deserialize(byte[] message, Collector out) throws IOException { deserializeMessage(message, out, null); diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java index 049c90c3ecd..c8feca07e82 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java @@ -135,6 +135,6 @@ private void setDeserialization(Config config) { // TODO: format SPI // only support json deserializationSchema SeaTunnelRowType rowType = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); - this.deserializationSchema = new JsonDeserializationSchema(false, false, rowType); + this.deserializationSchema = new JsonDeserializationSchema(false, false, rowType, null); } } diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java index 3818848fc15..dfc065bb728 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java @@ -72,7 +72,7 @@ public RedisSource(ReadonlyConfig readonlyConfig) { this.catalogTable = CatalogTableUtil.buildWithConfig(readonlyConfig); this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); this.deserializationSchema = - new JsonDeserializationSchema(false, false, seaTunnelRowType); + new JsonDeserializationSchema(false, false, seaTunnelRowType, null); } } else { this.catalogTable = CatalogTableUtil.buildSimpleTextTable(); diff --git a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java index ceebd71404a..873cc5859e2 100644 --- a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java +++ b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java @@ -259,7 +259,8 @@ private void setDeserialization(Config config) { } switch (format) { case JSON: - deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + deserializationSchema = + new JsonDeserializationSchema(false, false, typeInfo, null); break; case TEXT: String delimiter = DEFAULT_FIELD_DELIMITER; diff --git a/seatunnel-examples/seatunnel-engine-examples/pom.xml b/seatunnel-examples/seatunnel-engine-examples/pom.xml index 3256bdde885..8ffd8894ab0 100644 --- a/seatunnel-examples/seatunnel-engine-examples/pom.xml +++ b/seatunnel-examples/seatunnel-engine-examples/pom.xml @@ -67,5 +67,10 @@ connector-assert ${project.version} + + org.apache.seatunnel + connector-kafka + ${project.version} + diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/resources/examples/fake_to_console.conf b/seatunnel-examples/seatunnel-engine-examples/src/main/resources/examples/fake_to_console.conf index ec7871359d4..17c85c83439 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/resources/examples/fake_to_console.conf +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/resources/examples/fake_to_console.conf @@ -19,29 +19,46 @@ ###### env { - parallelism = 1 - job.mode = "BATCH" + execution.parallelism = 1 } source { - # This is a example source plugin **only for test and demonstrate the feature source plugin** - FakeSource { - result_table_name = "fake" - parallelism = 1 - schema = { - fields { - name = "string" - age = "int" + Kafka { + bootstrap.servers = "localhost:9092" + table_list = [ + { + topic = "ogg" + consumer.group = "ogg_multi_group" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = ogg_json + }, { + topic = "test" + consumer.group = "canal_multi_group" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = canal_json } - } + ] } } -transform { -} - sink { console { - source_table_name="fake" +// source_table_name = "fake" } } \ No newline at end of file From 751531e6874948342470114f332ac8324a6c598f Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 12 Apr 2024 15:52:21 +0800 Subject: [PATCH 42/71] fix --- .../seatunnel/format/avro/AvroDeserializationSchema.java | 5 +++-- .../json/CompatibleKafkaConnectDeserializationSchema.java | 7 ++++--- .../seatunnel/format/json/JsonDeserializationSchema.java | 5 +++-- .../format/json/canal/CanalJsonDeserializationSchema.java | 3 ++- .../json/debezium/DebeziumJsonDeserializationSchema.java | 3 ++- .../format/json/ogg/OggJsonDeserializationSchema.java | 3 ++- 6 files changed, 16 insertions(+), 10 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java index 47b3803cb06..d54f024a5e6 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java @@ -50,8 +50,9 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(message, null); GenericRecord record = this.converter.getReader().read(null, decoder); SeaTunnelRow seaTunnelRow = converter.converter(record, rowType); - TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); - if (tablePath != null) { + Optional tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath); + if (tablePath.isPresent()) { seaTunnelRow.setTableId(tablePath.toString()); } return seaTunnelRow; diff --git a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java index 0272c658855..9856f2ea30d 100644 --- a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java @@ -118,13 +118,14 @@ public void deserialize(ConsumerRecord msg, Collector tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath); if (payload.isArray()) { ArrayNode arrayNode = (ArrayNode) payload; for (int i = 0; i < arrayNode.size(); i++) { SeaTunnelRow row = convertJsonNode(arrayNode.get(i)); row.setRowKind(rowKind); - if (tablePath != null) { + if (tablePath.isPresent()) { row.setTableId(tablePath.toString()); } out.collect(row); @@ -132,7 +133,7 @@ public void deserialize(ConsumerRecord msg, Collector out) throws IOExcept } public void setCollectorTablePath(SeaTunnelRow deserialize, CatalogTable catalogTable) { - TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); - if (tablePath != null) { + Optional tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath); + if (tablePath.isPresent()) { deserialize.setTableId(tablePath.toString()); } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java index 2affe3f137b..1b5ed0cadf2 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java @@ -220,7 +220,8 @@ private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException public void deserialize(byte[] message, Collector out) throws IOException { ObjectNode jsonNodes = convertBytes(message); if (jsonNodes != null) { - TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); + TablePath tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath).orElse(null); deserialize(convertBytes(message), out, tablePath); } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java index 6f86169d710..14b191a2725 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java @@ -95,7 +95,8 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { @Override public void deserialize(byte[] message, Collector out) { - TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); + TablePath tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath).orElse(null); deserializeMessage(message, out, tablePath); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java index cebf2f494ef..8dc41897fb3 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java @@ -226,7 +226,8 @@ private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException @Override public void deserialize(byte[] message, Collector out) { - TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); + TablePath tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath).orElse(null); deserializeMessage(message, out, tablePath); } From 0aae5ce65cd3a42a10a8d733fc708aa9d6d16f86 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Mon, 15 Apr 2024 18:46:48 +0800 Subject: [PATCH 43/71] fix --- .../seatunnel/format/text/TextDeserializationSchema.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java index 0275d7b95ee..ea5a6e02279 100644 --- a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java @@ -171,8 +171,9 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { seaTunnelRowType.getFieldNames()[i]); } SeaTunnelRow seaTunnelRow = new SeaTunnelRow(objects); - TablePath tablePath = Optional.ofNullable(catalogTable.getTablePath()).orElse(null); - if (tablePath != null) { + Optional tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath); + if (tablePath.isPresent()) { seaTunnelRow.setTableId(tablePath.toString()); } return seaTunnelRow; From 35d13e7f5fbabd6a514bcea89d4c1204f87db725 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 19 Apr 2024 11:57:08 +0800 Subject: [PATCH 44/71] fix --- .../connectors/seatunnel/kafka/source/KafkaSourceConfig.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index 90b0bbde541..0de7d87ffe1 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -191,9 +191,6 @@ private ConsumerMetadata createConsumerMetadata(ReadonlyConfig readonlyConfig) { private CatalogTable createCatalogTable(ReadonlyConfig readonlyConfig) { Optional> schemaOptions = readonlyConfig.getOptional(TableSchemaOptions.SCHEMA); - if (readonlyConfig.get(TOPIC) == null) { - throw new RuntimeException("Make sure the `topic` configuration option is not empty"); - } TablePath tablePath = TablePath.of(readonlyConfig.get(TOPIC)); TableSchema tableSchema; if (schemaOptions.isPresent()) { From 041ce8b818f8900e79fc8b7d846baa6bf707e234 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 19 Apr 2024 13:43:25 +0800 Subject: [PATCH 45/71] fix --- seatunnel-connectors-v2/connector-pulsar/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/seatunnel-connectors-v2/connector-pulsar/pom.xml b/seatunnel-connectors-v2/connector-pulsar/pom.xml index e519b776be1..12bbec59187 100644 --- a/seatunnel-connectors-v2/connector-pulsar/pom.xml +++ b/seatunnel-connectors-v2/connector-pulsar/pom.xml @@ -47,11 +47,6 @@ ${project.version} - - org.apache.seatunnel - seatunnel-api - ${project.version} - From 0fdd8cadad3395920c542add528996b4eaa8ddb9 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 19 Apr 2024 13:47:53 +0800 Subject: [PATCH 46/71] fix --- .../seatunnel-engine-examples/pom.xml | 5 -- .../resources/examples/fake_to_console.conf | 47 ++++++------------- 2 files changed, 15 insertions(+), 37 deletions(-) diff --git a/seatunnel-examples/seatunnel-engine-examples/pom.xml b/seatunnel-examples/seatunnel-engine-examples/pom.xml index 8ffd8894ab0..3256bdde885 100644 --- a/seatunnel-examples/seatunnel-engine-examples/pom.xml +++ b/seatunnel-examples/seatunnel-engine-examples/pom.xml @@ -67,10 +67,5 @@ connector-assert ${project.version} - - org.apache.seatunnel - connector-kafka - ${project.version} - diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/resources/examples/fake_to_console.conf b/seatunnel-examples/seatunnel-engine-examples/src/main/resources/examples/fake_to_console.conf index 17c85c83439..ec7871359d4 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/resources/examples/fake_to_console.conf +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/resources/examples/fake_to_console.conf @@ -19,46 +19,29 @@ ###### env { - execution.parallelism = 1 + parallelism = 1 + job.mode = "BATCH" } source { - Kafka { - bootstrap.servers = "localhost:9092" - table_list = [ - { - topic = "ogg" - consumer.group = "ogg_multi_group" - start_mode = earliest - schema = { - fields { - id = "int" - name = "string" - description = "string" - weight = "string" - } - }, - format = ogg_json - }, { - topic = "test" - consumer.group = "canal_multi_group" - start_mode = earliest - schema = { - fields { - id = "int" - name = "string" - description = "string" - weight = "string" - } - }, - format = canal_json + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + result_table_name = "fake" + parallelism = 1 + schema = { + fields { + name = "string" + age = "int" } - ] + } } } +transform { +} + sink { console { -// source_table_name = "fake" + source_table_name="fake" } } \ No newline at end of file From 4b7af7c3f6cdae5a2619707d7ae4b86162db0fc6 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 19 Apr 2024 18:36:45 +0800 Subject: [PATCH 47/71] fix --- .../amazonsqs/source/AmazonSqsSource.java | 3 +-- .../file/source/reader/JsonReadStrategy.java | 2 +- .../google/sheets/source/SheetsSource.java | 3 +-- .../GoogleSheetsDeserializerTest.java | 2 +- .../seatunnel/pulsar/source/PulsarSource.java | 6 ++--- .../rabbitmq/source/RabbitmqSource.java | 2 +- .../seatunnel/redis/source/RedisSource.java | 2 +- .../rocketmq/source/RocketMqSource.java | 3 +-- .../json/JsonDeserializationSchema.java | 5 ++++ .../json/JsonRowDataSerDeSchemaTest.java | 24 ++++++++----------- 10 files changed, 24 insertions(+), 28 deletions(-) diff --git a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java index 8e8047647d4..33a9856df3e 100644 --- a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java +++ b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java @@ -113,8 +113,7 @@ private void setDeserialization(Config config) { MessageFormat format = ReadonlyConfig.fromConfig(config).get(FORMAT); switch (format) { case JSON: - deserializationSchema = - new JsonDeserializationSchema(false, false, typeInfo, null); + deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); break; case TEXT: String delimiter = DEFAULT_FIELD_DELIMITER; diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java index 96e0ea5fd80..82d0a40c0bd 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java @@ -69,7 +69,7 @@ public void setSeaTunnelRowTypeInfo(SeaTunnelRowType seaTunnelRowType) { false, false, this.seaTunnelRowTypeWithPartition, null); } else { deserializationSchema = - new JsonDeserializationSchema(false, false, this.seaTunnelRowType, null); + new JsonDeserializationSchema(false, false, this.seaTunnelRowType); } } diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java index 86e4b6fefd0..0661f6db366 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java @@ -80,8 +80,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { } else { this.seaTunnelRowType = CatalogTableUtil.buildSimpleTextSchema(); } - this.deserializationSchema = - new JsonDeserializationSchema(false, false, seaTunnelRowType, null); + this.deserializationSchema = new JsonDeserializationSchema(false, false, seaTunnelRowType); } @Override diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java index eabbf8f2f81..8c81a51c581 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java @@ -40,7 +40,7 @@ public void testJsonParseError() { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); final DeserializationSchema deser = - new JsonDeserializationSchema(false, false, schema, null); + new JsonDeserializationSchema(false, false, schema); final GoogleSheetsDeserializer googleSheetsDeser = new GoogleSheetsDeserializer(schema.getFieldNames(), deser); List row = new ArrayList<>(); diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java index 2f0552dee71..db0bb24ef01 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java @@ -307,8 +307,7 @@ private void setDeserialization(Config config) { } switch (format.toUpperCase()) { case "JSON": - deserializationSchema = - new JsonDeserializationSchema(false, false, typeInfo, null); + deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); break; case "CANAL_JSON": deserializationSchema = @@ -324,8 +323,7 @@ private void setDeserialization(Config config) { } } else { typeInfo = CatalogTableUtil.buildSimpleTextSchema(); - this.deserializationSchema = - new JsonDeserializationSchema(false, false, typeInfo, null); + this.deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); } } diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java index c8feca07e82..049c90c3ecd 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java @@ -135,6 +135,6 @@ private void setDeserialization(Config config) { // TODO: format SPI // only support json deserializationSchema SeaTunnelRowType rowType = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); - this.deserializationSchema = new JsonDeserializationSchema(false, false, rowType, null); + this.deserializationSchema = new JsonDeserializationSchema(false, false, rowType); } } diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java index dfc065bb728..3818848fc15 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java @@ -72,7 +72,7 @@ public RedisSource(ReadonlyConfig readonlyConfig) { this.catalogTable = CatalogTableUtil.buildWithConfig(readonlyConfig); this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); this.deserializationSchema = - new JsonDeserializationSchema(false, false, seaTunnelRowType, null); + new JsonDeserializationSchema(false, false, seaTunnelRowType); } } else { this.catalogTable = CatalogTableUtil.buildSimpleTextTable(); diff --git a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java index 873cc5859e2..ceebd71404a 100644 --- a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java +++ b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java @@ -259,8 +259,7 @@ private void setDeserialization(Config config) { } switch (format) { case JSON: - deserializationSchema = - new JsonDeserializationSchema(false, false, typeInfo, null); + deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); break; case TEXT: String delimiter = DEFAULT_FIELD_DELIMITER; diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index f98580f3ccd..f1ff7daabc2 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -68,6 +68,11 @@ public class JsonDeserializationSchema implements DeserializationSchema { - new JsonDeserializationSchema(true, true, null, null); + new JsonDeserializationSchema(true, true, null); }, "expecting exception message: " + errorMessage); assertEquals(actual.getMessage(), errorMessage); @@ -430,8 +427,7 @@ public void testDeserializationNoJson() throws Exception { new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); String noJson = "{]"; - final JsonDeserializationSchema deser = - new JsonDeserializationSchema(false, false, schema, null); + final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, false, schema); SeaTunnelRuntimeException expected = CommonError.jsonOperationError("Common", noJson); SeaTunnelRuntimeException actual = From c2056c5ddae2f9dceae8959838cd9a6f342ec7f9 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 19 Apr 2024 18:37:18 +0800 Subject: [PATCH 48/71] fix --- .../format/json/JsonDeserializationSchema.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index f1ff7daabc2..d05b0004f33 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -50,23 +50,23 @@ public class JsonDeserializationSchema implements DeserializationSchema Date: Fri, 19 Apr 2024 18:43:43 +0800 Subject: [PATCH 49/71] fix --- .../seatunnel/file/source/reader/JsonReadStrategy.java | 3 +-- .../seatunnel/connectors/seatunnel/http/source/HttpSource.java | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java index 82d0a40c0bd..6c58e368721 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java @@ -65,8 +65,7 @@ public void setSeaTunnelRowTypeInfo(SeaTunnelRowType seaTunnelRowType) { super.setSeaTunnelRowTypeInfo(seaTunnelRowType); if (isMergePartition) { deserializationSchema = - new JsonDeserializationSchema( - false, false, this.seaTunnelRowTypeWithPartition, null); + new JsonDeserializationSchema(false, false, this.seaTunnelRowTypeWithPartition); } else { deserializationSchema = new JsonDeserializationSchema(false, false, this.seaTunnelRowType); diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java index 33c80cb00f5..4e3f3ab1a59 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java @@ -128,7 +128,7 @@ protected void buildSchemaWithConfig(Config pluginConfig) { case JSON: this.deserializationSchema = new JsonDeserializationSchema( - false, false, catalogTable.getSeaTunnelRowType(), null); + false, false, catalogTable.getSeaTunnelRowType()); if (pluginConfig.hasPath(HttpConfig.JSON_FIELD.key())) { jsonField = getJsonField(pluginConfig.getConfig(HttpConfig.JSON_FIELD.key())); From 0bb4cf2d045ef55f52f5317cd504933363e699d9 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Sat, 20 Apr 2024 20:20:20 +0800 Subject: [PATCH 50/71] fix --- .../seatunnel/amazonsqs/source/AmazonSqsSource.java | 3 +-- .../seatunnel/fake/source/FakeDataGenerator.java | 4 ++-- .../format/json/JsonDeserializationSchema.java | 2 +- .../debezium/DebeziumJsonDeserializationSchema.java | 11 +++++++++++ 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java index 33a9856df3e..9022abe1f10 100644 --- a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java +++ b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java @@ -138,8 +138,7 @@ private void setDeserialization(Config config) { includeSchema = config.getBoolean(DEBEZIUM_RECORD_INCLUDE_SCHEMA.key()); } deserializationSchema = - new DebeziumJsonDeserializationSchema( - typeInfo, true, includeSchema, null); + new DebeziumJsonDeserializationSchema(typeInfo, true, includeSchema); break; default: throw new SeaTunnelJsonFormatException( diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java index ef5efc8164f..8bacb1a94d5 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java @@ -44,7 +44,7 @@ public class FakeDataGenerator { private final FakeConfig fakeConfig; private final JsonDeserializationSchema jsonDeserializationSchema; private final FakeDataRandomUtils fakeDataRandomUtils; - private final String tableId; + private String tableId; public FakeDataGenerator(FakeConfig fakeConfig) { this.catalogTable = fakeConfig.getCatalogTable(); @@ -54,7 +54,7 @@ public FakeDataGenerator(FakeConfig fakeConfig) { fakeConfig.getFakeRows() == null ? null : new JsonDeserializationSchema( - false, false, catalogTable.getSeaTunnelRowType(), catalogTable); + false, false, catalogTable.getSeaTunnelRowType()); this.fakeDataRandomUtils = new FakeDataRandomUtils(fakeConfig); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index d05b0004f33..93740e27368 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -70,7 +70,7 @@ public class JsonDeserializationSchema implements DeserializationSchema Date: Sat, 20 Apr 2024 21:44:47 +0800 Subject: [PATCH 51/71] fix --- .../pulsar/source/format/PulsarCanalDecorator.java | 14 +------------- .../json/canal/CanalJsonDeserializationSchema.java | 10 +++++----- 2 files changed, 6 insertions(+), 18 deletions(-) diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java index 5844694c80a..e16d098d6c9 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java @@ -23,7 +23,6 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.utils.JsonUtils; @@ -59,16 +58,6 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { @Override public void deserialize(byte[] message, Collector out) throws IOException { - deserializeMessage(message, out, null); - } - - public void deserialize(byte[] message, Collector out, TablePath tablePath) - throws IOException { - deserializeMessage(message, out, tablePath); - } - - public void deserializeMessage(byte[] message, Collector out, TablePath tablePath) - throws IOException { JsonNode pulsarCanal = JsonUtils.parseObject(message); ArrayNode canalList = JsonUtils.parseArray(pulsarCanal.get(MESSAGE).asText()); Iterator canalIterator = canalList.elements(); @@ -77,8 +66,7 @@ public void deserializeMessage(byte[] message, Collector out, Tabl // reconvert pulsar handler, reference to // https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/MessageUtils.java ObjectNode root = reconvertPulsarData((ObjectNode) next); - // todo Wait for TablePath functionality to be added - canalJsonDeserializationSchema.deserialize(root, out, tablePath); + canalJsonDeserializationSchema.deserialize(root, out); } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java index 1b5ed0cadf2..b3b4b8118a1 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java @@ -121,8 +121,10 @@ public SeaTunnelDataType getProducedType() { return this.physicalRowType; } - public void deserialize(ObjectNode jsonNode, Collector out, TablePath tablePath) - throws IOException { + public void deserialize(ObjectNode jsonNode, Collector out) throws IOException { + TablePath tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath).orElse(null); + try { if (database != null && !databasePattern.matcher(jsonNode.get(FIELD_DATABASE).asText()).matches()) { @@ -220,9 +222,7 @@ private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException public void deserialize(byte[] message, Collector out) throws IOException { ObjectNode jsonNodes = convertBytes(message); if (jsonNodes != null) { - TablePath tablePath = - Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath).orElse(null); - deserialize(convertBytes(message), out, tablePath); + deserialize(convertBytes(message), out); } } From 1433f006635a4c8b87486f4d7007848db32a39e5 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Mon, 22 Apr 2024 10:29:03 +0800 Subject: [PATCH 52/71] fix --- .../kafka/source/KafkaSourceSplitEnumerator.java | 2 -- .../format/json/JsonDeserializationSchema.java | 10 +++++----- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index f89fd8bb798..a7471ae0869 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -270,9 +270,7 @@ private Set getTopicInfo() throws ExecutionException, Interrup currentPathTopics.forEach(topic -> topicMappingTablePathMap.put(topic, tablePath)); topics.addAll(currentPathTopics); } - log.info("Discovered topics: {}", topics); - Collection partitions = adminClient.describeTopics(topics).all().get().values().stream() .flatMap( diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 93740e27368..db401a3da0d 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -50,23 +50,23 @@ public class JsonDeserializationSchema implements DeserializationSchema Date: Sun, 28 Apr 2024 18:11:58 +0800 Subject: [PATCH 53/71] fix --- docs/en/connector-v2/source/kafka.md | 2 +- .../seatunnel/connectors/seatunnel/kafka/config/Config.java | 2 +- .../json/CompatibleKafkaConnectDeserializationSchema.java | 5 ++--- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index 631415effcc..c234767cb2f 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -183,7 +183,7 @@ source { ### Multiple Kafka Source -> Currently, multiple kafka source reads are supported using the Zeta engine, but note that you can only configure one instance of `bootstrap.servers`, and only one in the `table_list` and `topic` parameters,Currently, when you use multiple topics in `table_list`, you are still free to set parameters for each `topic` +> Currently, multiple kafka source reads are supported using the Zeta engine, but note that you can only configure one instance of `bootstrap.servers`, and only one the `table_list` or `topic` parameters. Currently, when you use multiple topics in `table_list`, you are still free to set parameters for each `topic` ```hocon diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java index 2f665c33cee..6d6f43dc4c2 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java @@ -182,5 +182,5 @@ public class Config { .type(new TypeReference>>() {}) .noDefaultValue() .withDescription( - "Topic list config You can configure only one `table_list` and one `topic` at the same time"); + "Topic list config. You can configure only one `table_list` or one `topic` at the same time"); } diff --git a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java index 9856f2ea30d..c0fe30a183d 100644 --- a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java @@ -74,14 +74,13 @@ public class CompatibleKafkaConnectDeserializationSchema private final CatalogTable catalogTable; public CompatibleKafkaConnectDeserializationSchema( - @NonNull SeaTunnelRowType seaTunnelRowType, boolean keySchemaEnable, boolean valueSchemaEnable, boolean failOnMissingField, boolean ignoreParseErrors, - CatalogTable catalogTable) { + @NonNull CatalogTable catalogTable) { - this.seaTunnelRowType = seaTunnelRowType; + this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); this.keySchemaEnable = keySchemaEnable; this.valueSchemaEnable = valueSchemaEnable; From 7bb6353c660cd8d4b4c991c404e4ad9d9536584d Mon Sep 17 00:00:00 2001 From: zhilinli Date: Sun, 28 Apr 2024 18:18:03 +0800 Subject: [PATCH 54/71] fix --- .../seatunnel/kafka/source/KafkaSourceConfig.java | 7 +------ .../json/CompatibleKafkaConnectDeserializationSchema.java | 1 - 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index 0de7d87ffe1..260aac690d2 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -260,12 +260,7 @@ private DeserializationSchema createDeserializationSchema( readonlyConfig.get( KafkaConnectJsonFormatOptions.VALUE_CONVERTER_SCHEMA_ENABLED); return new CompatibleKafkaConnectDeserializationSchema( - seaTunnelRowType, - keySchemaEnable, - valueSchemaEnable, - false, - false, - catalogTable); + keySchemaEnable, valueSchemaEnable, false, false, catalogTable); case DEBEZIUM_JSON: boolean includeSchema = readonlyConfig.get(DEBEZIUM_RECORD_INCLUDE_SCHEMA); return new DebeziumJsonDeserializationSchema( diff --git a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java index c0fe30a183d..4acdcccc74b 100644 --- a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java @@ -79,7 +79,6 @@ public CompatibleKafkaConnectDeserializationSchema( boolean failOnMissingField, boolean ignoreParseErrors, @NonNull CatalogTable catalogTable) { - this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); this.keySchemaEnable = keySchemaEnable; this.valueSchemaEnable = valueSchemaEnable; From 89cb3fec985f3bd2d2a33cc9c15b0d53926a030a Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 7 May 2024 00:07:25 +0800 Subject: [PATCH 55/71] fix --- docs/en/connector-v2/source/kafka.md | 25 ++-- .../e2e/connector/kafka/KafkaFormatIT.java | 112 ++++++++++++++---- .../src/test/resources/canal/canal_data.txt | 10 +- ...ert.conf => kafka_multi_source_to_pg.conf} | 20 ++-- 4 files changed, 116 insertions(+), 51 deletions(-) rename seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/{kafka_multi_source_to_assert.conf => kafka_multi_source_to_pg.conf} (86%) diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index c234767cb2f..c42c84a8a5a 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -183,7 +183,7 @@ source { ### Multiple Kafka Source -> Currently, multiple kafka source reads are supported using the Zeta engine, but note that you can only configure one instance of `bootstrap.servers`, and only one the `table_list` or `topic` parameters. Currently, when you use multiple topics in `table_list`, you are still free to set parameters for each `topic` +> This is written to the same pg table according to different formats and topics of parsing kafka Perform upsert operations based on the id ```hocon @@ -210,9 +210,9 @@ source { } }, format = ogg_json - }, { + }, + { topic = "test-cdc_mds" - consumer.group = "canal_multi_group" start_mode = earliest schema = { fields { @@ -225,20 +225,19 @@ source { format = canal_json } ] - # Each topic in the `table_list` shares the config - kafka.config = { - client.id = client_1 - max.poll.records = 50000 - } } } sink { - Assert { - rules { - // The current table name is the same as the topic name in order to maintain fewer configuration parameters - table-names = ["^test-ogg-sou.*","test-cdc_mds"] - } + Jdbc { + driver = org.postgresql.Driver + url = "jdbc:postgresql://postgresql:5432/test?loggerLevel=OFF" + user = test + password = test + generate_sink_sql = true + database = test + table = public.sink + primary_keys = ["id"] } } ``` diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java index d51ce35b80d..25fa9b4aab4 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java @@ -340,10 +340,13 @@ public void testMultiFormatCheck(TestContainer container) throws IOException, InterruptedException { LOG.info( "====================== Multi Source Format Canal and Ogg Check ======================"); - Container.ExecResult execCanalResultKafka = - container.executeJob("/multiFormatIT/kafka_multi_source_to_assert.conf"); + Container.ExecResult execCanalAndOggResultKafka = + container.executeJob("/multiFormatIT/kafka_multi_source_to_pg.conf"); Assertions.assertEquals( - 0, execCanalResultKafka.getExitCode(), execCanalResultKafka.getStderr()); + 0, + execCanalAndOggResultKafka.getExitCode(), + execCanalAndOggResultKafka.getStderr()); + checkFormatCanalAndOgg(); } @TestTemplate @@ -415,23 +418,79 @@ public void testFormatCompatibleCheck(TestContainer container) checkCompatibleFormat(); } + private void checkFormatCanalAndOgg() { + List> postgreSinkTableList = getPostgreSinkTableList(PG_SINK_TABLE1); + List> checkArraysResult = + Stream.>of( + Arrays.asList( + 101, + "scooter", + "Small 2-wheel scooter", + "3.140000104904175"), + Arrays.asList( + 102, "car battery", "12V car battery", "8.100000381469727"), + Arrays.asList( + 103, + "12-pack drill bits", + "12-pack of drill bits with sizes ranging from #40 to #3", + "0.800000011920929"), + Arrays.asList(104, "hammer", "12oz carpenter's hammer", "0.75"), + Arrays.asList(105, "hammer", "14oz carpenter's hammer", "0.875"), + Arrays.asList(106, "hammer", "18oz carpenter hammer", "1"), + Arrays.asList( + 107, "rocks", "box of assorted rocks", "5.099999904632568"), + Arrays.asList( + 108, + "jacket", + "water resistent black wind breaker", + "0.10000000149011612"), + Arrays.asList( + 109, + "spare tire", + "24 inch spare tire", + "22.200000762939453"), + Arrays.asList( + 110, + "jacket", + "new water resistent white wind breaker", + "0.5"), + Arrays.asList(1101, "scooter", "Small 2-wheel scooter", "4.56"), + Arrays.asList(1102, "car battery", "12V car battery", "8.1"), + Arrays.asList( + 1103, + "12-pack drill bits", + "12-pack of drill bits with sizes ranging from #40 to #3", + "0.8"), + Arrays.asList(1104, "hammer", "12oz carpenter's hammer", "0.75"), + Arrays.asList(1105, "hammer", "14oz carpenter's hammer", "0.875"), + Arrays.asList(1106, "hammer", "16oz carpenter's hammer", "1.0"), + Arrays.asList(1107, "rocks", "box of assorted rocks", "7.88"), + Arrays.asList( + 1108, + "jacket", + "water resistent black wind breaker", + "0.1")) + .collect(Collectors.toList()); + Assertions.assertIterableEquals(postgreSinkTableList, checkArraysResult); + } + private void checkCanalFormat() { 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\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"DELETE\"}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"4.56\"},\"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\":\"7.88\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"DELETE\"}"); + "{\"data\":{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":\"8.1\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1103,\"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\":1104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":\"0.75\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":\"0.875\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1.0\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":\"0.1\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"DELETE\"}", + "{\"data\":{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"4.56\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"},\"type\":\"DELETE\"}", + "{\"data\":{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"7.88\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"DELETE\"}"); ArrayList result = new ArrayList<>(); ArrayList topics = new ArrayList<>(); @@ -454,19 +513,22 @@ private void checkCanalFormat() { List> expected = Stream.>of( - Arrays.asList(101, "scooter", "Small 2-wheel scooter", "4.56"), - Arrays.asList(102, "car battery", "12V car battery", "8.1"), + Arrays.asList(1101, "scooter", "Small 2-wheel scooter", "4.56"), + Arrays.asList(1102, "car battery", "12V car battery", "8.1"), Arrays.asList( - 103, + 1103, "12-pack drill bits", "12-pack of drill bits with sizes ranging from #40 to #3", "0.8"), - Arrays.asList(104, "hammer", "12oz carpenter's hammer", "0.75"), - Arrays.asList(105, "hammer", "14oz carpenter's hammer", "0.875"), - Arrays.asList(106, "hammer", "16oz carpenter's hammer", "1.0"), - Arrays.asList(107, "rocks", "box of assorted rocks", "7.88"), + Arrays.asList(1104, "hammer", "12oz carpenter's hammer", "0.75"), + Arrays.asList(1105, "hammer", "14oz carpenter's hammer", "0.875"), + Arrays.asList(1106, "hammer", "16oz carpenter's hammer", "1.0"), + Arrays.asList(1107, "rocks", "box of assorted rocks", "7.88"), Arrays.asList( - 108, "jacket", "water resistent black wind breaker", "0.1")) + 1108, + "jacket", + "water resistent black wind breaker", + "0.1")) .collect(Collectors.toList()); Assertions.assertIterableEquals(expected, postgreSinkTableList); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canal/canal_data.txt b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canal/canal_data.txt index 6b4d4dfe544..d959351555a 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canal/canal_data.txt +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canal/canal_data.txt @@ -1,7 +1,7 @@ {"data":null,"database":"canal_hvygfc","es":1697788899000,"id":1,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE DATABASE IF NOT EXISTS canal_hvygfc","sqlType":null,"table":"","ts":1697788899992,"type":"QUERY"} {"data":null,"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE TABLE products (\nid INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,\nname VARCHAR(255) NOT NULL DEFAULT 'SeaTunnel',\ndescription VARCHAR(512),\nweight VARCHAR(512)\n)","sqlType":null,"table":"products","ts":1697788900618,"type":"CREATE"} -{"data":null,"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"ALTER TABLE products AUTO_INCREMENT = 101","sqlType":null,"table":"products","ts":1697788900618,"type":"ALTER"} -{"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":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900618,"type":"INSERT"} -{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"4.56"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":[{"weight":"3.14"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"UPDATE"} -{"data":[{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"7.88"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"UPDATE"} -{"data":[{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"DELETE"} \ No newline at end of file +{"data":null,"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"ALTER TABLE products AUTO_INCREMENT = 1101","sqlType":null,"table":"products","ts":1697788900618,"type":"ALTER"} +{"data":[{"id":"1101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"1102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"1103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"1104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"1105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"1106","name":"hammer","description":"16oz carpenter's hammer","weight":"1.0"},{"id":"1107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"1108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"1109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900618,"type":"INSERT"} +{"data":[{"id":"1101","name":"scooter","description":"Small 2-wheel scooter","weight":"4.56"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":[{"weight":"3.14"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"UPDATE"} +{"data":[{"id":"1107","name":"rocks","description":"box of assorted rocks","weight":"7.88"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"UPDATE"} +{"data":[{"id":"1109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"DELETE"} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf similarity index 86% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf index 165249d754f..8bc6d41cd3f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_assert.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf @@ -41,9 +41,9 @@ source { } }, format = ogg_json - }, { + }, + { topic = "test-cdc_mds" - consumer.group = "canal_multi_group" start_mode = earliest schema = { fields { @@ -60,10 +60,14 @@ source { } sink { - Assert { - rules { - // The current table name is the same as the topic name in order to maintain fewer configuration parameters - table-names = ["^test-ogg-sou.*","test-cdc_mds"] - } + Jdbc { + driver = org.postgresql.Driver + url = "jdbc:postgresql://postgresql:5432/test?loggerLevel=OFF" + user = test + password = test + generate_sink_sql = true + database = test + table = public.sink + primary_keys = ["id"] } -} +} \ No newline at end of file From 0cde5c825c1f8d009f31659cdef323e948614065 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Thu, 9 May 2024 20:36:16 +0800 Subject: [PATCH 56/71] fix --- .../amazonsqs/source/AmazonSqsSource.java | 8 ++- .../kafka/source/KafkaSourceConfig.java | 18 ++---- .../seatunnel/pulsar/source/PulsarSource.java | 12 ++-- ...ibleKafkaConnectDeserializationSchema.java | 11 ++-- .../json/JsonDeserializationSchema.java | 15 ++--- .../canal/CanalJsonDeserializationSchema.java | 34 +++++----- .../DebeziumJsonDeserializationSchema.java | 30 +++------ .../ogg/OggJsonDeserializationSchema.java | 39 +++++------ .../json/canal/CanalJsonSerDeSchemaTest.java | 64 ++++++++++--------- .../debezium/DebeziumJsonSerDeSchemaTest.java | 61 +++++++++--------- .../json/ogg/OggJsonSerDeSchemaTest.java | 52 ++++++++------- 11 files changed, 162 insertions(+), 182 deletions(-) diff --git a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java index 9022abe1f10..086ad1ffc47 100644 --- a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java +++ b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportColumnProjection; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -109,7 +110,7 @@ public AbstractSingleSplitReader createReader( private void setDeserialization(Config config) { if (config.hasPath(TableSchemaOptions.SCHEMA.key())) { - typeInfo = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); + CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); MessageFormat format = ReadonlyConfig.fromConfig(config).get(FORMAT); switch (format) { case JSON: @@ -128,7 +129,7 @@ private void setDeserialization(Config config) { break; case CANAL_JSON: deserializationSchema = - CanalJsonDeserializationSchema.builder(typeInfo) + CanalJsonDeserializationSchema.builder(catalogTable) .setIgnoreParseErrors(true) .build(); break; @@ -138,7 +139,8 @@ private void setDeserialization(Config config) { includeSchema = config.getBoolean(DEBEZIUM_RECORD_INCLUDE_SCHEMA.key()); } deserializationSchema = - new DebeziumJsonDeserializationSchema(typeInfo, true, includeSchema); + new DebeziumJsonDeserializationSchema( + catalogTable, true, includeSchema); break; default: throw new SeaTunnelJsonFormatException( diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index 0de7d87ffe1..6afff6e83e3 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -235,7 +235,7 @@ private DeserializationSchema createDeserializationSchema( MessageFormat format = readonlyConfig.get(FORMAT); switch (format) { case JSON: - return new JsonDeserializationSchema(false, false, seaTunnelRowType, catalogTable); + return new JsonDeserializationSchema(catalogTable, false, false); case TEXT: String delimiter = readonlyConfig.get(FIELD_DELIMITER); return TextDeserializationSchema.builder() @@ -243,14 +243,12 @@ private DeserializationSchema createDeserializationSchema( .delimiter(delimiter) .build(); case CANAL_JSON: - return CanalJsonDeserializationSchema.builder(seaTunnelRowType) + return CanalJsonDeserializationSchema.builder(catalogTable) .setIgnoreParseErrors(true) - .setCatalogTable(catalogTable) .build(); case OGG_JSON: - return OggJsonDeserializationSchema.builder(seaTunnelRowType) + return OggJsonDeserializationSchema.builder(catalogTable) .setIgnoreParseErrors(true) - .setCatalogTable(catalogTable) .build(); case COMPATIBLE_KAFKA_CONNECT_JSON: Boolean keySchemaEnable = @@ -260,16 +258,10 @@ private DeserializationSchema createDeserializationSchema( readonlyConfig.get( KafkaConnectJsonFormatOptions.VALUE_CONVERTER_SCHEMA_ENABLED); return new CompatibleKafkaConnectDeserializationSchema( - seaTunnelRowType, - keySchemaEnable, - valueSchemaEnable, - false, - false, - catalogTable); + catalogTable, keySchemaEnable, valueSchemaEnable, false, false); case DEBEZIUM_JSON: boolean includeSchema = readonlyConfig.get(DEBEZIUM_RECORD_INCLUDE_SCHEMA); - return new DebeziumJsonDeserializationSchema( - seaTunnelRowType, true, includeSchema, catalogTable); + return new DebeziumJsonDeserializationSchema(catalogTable, true, includeSchema); case AVRO: return new AvroDeserializationSchema(seaTunnelRowType, catalogTable); default: diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java index db0bb24ef01..032bac62bf2 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java @@ -27,6 +27,7 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -52,7 +53,6 @@ import org.apache.seatunnel.connectors.seatunnel.pulsar.source.format.PulsarCanalDecorator; import org.apache.seatunnel.connectors.seatunnel.pulsar.source.reader.PulsarSourceReader; import org.apache.seatunnel.connectors.seatunnel.pulsar.source.split.PulsarPartitionSplit; -import org.apache.seatunnel.format.json.JsonDeserializationSchema; import org.apache.seatunnel.format.json.canal.CanalJsonDeserializationSchema; import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; @@ -300,19 +300,20 @@ private void setPartitionDiscoverer(Config config) { private void setDeserialization(Config config) { if (config.hasPath(SCHEMA.key())) { - typeInfo = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); + CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); String format = FORMAT.defaultValue(); if (config.hasPath(FORMAT.key())) { format = config.getString(FORMAT.key()); } switch (format.toUpperCase()) { case "JSON": - deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + // deserializationSchema = new + // JsonDeserializationSchema(false, false, typeInfo); break; case "CANAL_JSON": deserializationSchema = new PulsarCanalDecorator( - CanalJsonDeserializationSchema.builder(typeInfo) + CanalJsonDeserializationSchema.builder(catalogTable) .setIgnoreParseErrors(true) .build()); break; @@ -323,7 +324,8 @@ private void setDeserialization(Config config) { } } else { typeInfo = CatalogTableUtil.buildSimpleTextSchema(); - this.deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + // this.deserializationSchema = new JsonDeserializationSchema(false, false, + // typeInfo); } } diff --git a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java index 9856f2ea30d..256d41b3574 100644 --- a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java @@ -74,22 +74,19 @@ public class CompatibleKafkaConnectDeserializationSchema private final CatalogTable catalogTable; public CompatibleKafkaConnectDeserializationSchema( - @NonNull SeaTunnelRowType seaTunnelRowType, + @NonNull CatalogTable catalogTable, boolean keySchemaEnable, boolean valueSchemaEnable, boolean failOnMissingField, - boolean ignoreParseErrors, - CatalogTable catalogTable) { - - this.seaTunnelRowType = seaTunnelRowType; + boolean ignoreParseErrors) { + this.catalogTable = catalogTable; + this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); this.keySchemaEnable = keySchemaEnable; this.valueSchemaEnable = valueSchemaEnable; - // Runtime converter this.runtimeConverter = new JsonToRowConverters(failOnMissingField, ignoreParseErrors) .createRowConverter(checkNotNull(seaTunnelRowType)); - this.catalogTable = catalogTable; } @Override diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index db401a3da0d..6666a8dad9e 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -61,29 +61,28 @@ public class JsonDeserializationSchema implements DeserializationSchema getProducedType() { - return this.physicalRowType; + return this.seaTunnelRowType; } public void deserialize(ObjectNode jsonNode, Collector out) throws IOException { @@ -240,8 +240,8 @@ private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType physicalDataT // ------------------------------------------------------------------------------------------ /** Creates A builder for building a {@link CanalJsonDeserializationSchema}. */ - public static Builder builder(SeaTunnelRowType physicalDataType) { - return new Builder(physicalDataType); + public static Builder builder(CatalogTable catalogTable) { + return new Builder(catalogTable); } public static class Builder { @@ -252,12 +252,10 @@ public static class Builder { private String table = null; - private final SeaTunnelRowType physicalDataType; - private CatalogTable catalogTable; - public Builder(SeaTunnelRowType physicalDataType) { - this.physicalDataType = physicalDataType; + public Builder(CatalogTable catalogTable) { + this.catalogTable = catalogTable; } public Builder setDatabase(String database) { @@ -282,7 +280,7 @@ public Builder setCatalogTable(CatalogTable catalogTable) { public CanalJsonDeserializationSchema build() { return new CanalJsonDeserializationSchema( - physicalDataType, database, table, ignoreParseErrors, catalogTable); + catalogTable, database, table, ignoreParseErrors); } } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java index 79d8a066f68..0329c68e643 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java @@ -62,37 +62,23 @@ public class DebeziumJsonDeserializationSchema implements DeserializationSchema< private CatalogTable catalogTable; - public DebeziumJsonDeserializationSchema(SeaTunnelRowType rowType, boolean ignoreParseErrors) { - this.rowType = rowType; + public DebeziumJsonDeserializationSchema(CatalogTable catalogTable, boolean ignoreParseErrors) { + this.catalogTable = catalogTable; + this.rowType = catalogTable.getSeaTunnelRowType(); this.ignoreParseErrors = ignoreParseErrors; this.jsonDeserializer = - new JsonDeserializationSchema( - false, ignoreParseErrors, createJsonRowType(rowType), null); + new JsonDeserializationSchema(catalogTable, false, ignoreParseErrors); this.debeziumRowConverter = new DebeziumRowConverter(rowType); this.debeziumEnabledSchema = false; } public DebeziumJsonDeserializationSchema( - SeaTunnelRowType rowType, boolean ignoreParseErrors, boolean debeziumEnabledSchema) { - this.rowType = rowType; - this.ignoreParseErrors = ignoreParseErrors; - this.jsonDeserializer = - new JsonDeserializationSchema( - false, ignoreParseErrors, createJsonRowType(rowType), catalogTable); - this.debeziumRowConverter = new DebeziumRowConverter(rowType); - this.debeziumEnabledSchema = debeziumEnabledSchema; - } - - public DebeziumJsonDeserializationSchema( - SeaTunnelRowType rowType, - boolean ignoreParseErrors, - boolean debeziumEnabledSchema, - CatalogTable catalogTable) { - this.rowType = rowType; + CatalogTable catalogTable, boolean ignoreParseErrors, boolean debeziumEnabledSchema) { + this.catalogTable = catalogTable; + this.rowType = catalogTable.getSeaTunnelRowType(); this.ignoreParseErrors = ignoreParseErrors; this.jsonDeserializer = - new JsonDeserializationSchema( - false, ignoreParseErrors, createJsonRowType(rowType), catalogTable); + new JsonDeserializationSchema(catalogTable, false, ignoreParseErrors); this.debeziumRowConverter = new DebeziumRowConverter(rowType); this.debeziumEnabledSchema = debeziumEnabledSchema; this.catalogTable = catalogTable; diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java index 8dc41897fb3..14f3b5944ed 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java @@ -33,6 +33,8 @@ import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.format.json.JsonDeserializationSchema; +import lombok.NonNull; + import java.io.IOException; import java.util.Optional; import java.util.regex.Pattern; @@ -82,28 +84,26 @@ public class OggJsonDeserializationSchema implements DeserializationSchema getProducedType() { - return this.physicalRowType; + return this.seaTunnelRowType; } public void deserializeMessage( @@ -245,8 +245,8 @@ private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType physicalDataT // ------------------------------------------------------------------------------------------ /** Creates A builder for building a {@link OggJsonDeserializationSchema}. */ - public static Builder builder(SeaTunnelRowType physicalDataType) { - return new Builder(physicalDataType); + public static Builder builder(CatalogTable catalogTable) { + return new Builder(catalogTable); } public static class Builder { @@ -257,12 +257,10 @@ public static class Builder { private String table = null; - private final SeaTunnelRowType physicalDataType; - private CatalogTable catalogTable; - public Builder(SeaTunnelRowType physicalDataType) { - this.physicalDataType = physicalDataType; + public Builder(CatalogTable catalogTable) { + this.catalogTable = catalogTable; } public Builder setDatabase(String database) { @@ -280,14 +278,9 @@ public Builder setIgnoreParseErrors(boolean ignoreParseErrors) { return this; } - public Builder setCatalogTable(CatalogTable catalogTable) { - this.catalogTable = catalogTable; - return this; - } - public OggJsonDeserializationSchema build() { return new OggJsonDeserializationSchema( - physicalDataType, database, table, ignoreParseErrors, catalogTable); + catalogTable, database, table, ignoreParseErrors); } } } diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java index 601e9c738c6..d35849e8bd4 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java @@ -19,6 +19,8 @@ package org.apache.seatunnel.format.json.canal; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -47,16 +49,18 @@ public class CanalJsonSerDeSchemaTest { private static final String FORMAT = "Canal"; - private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = + private static final SeaTunnelRowType SEATUNNEL_ROW_TYPE = new SeaTunnelRowType( new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); + private static final CatalogTable catalogTables = + CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); @Test public void testFilteringTables() throws Exception { List lines = readLines("canal-data-filter-table.txt"); CanalJsonDeserializationSchema deserializationSchema = - new CanalJsonDeserializationSchema.Builder(PHYSICAL_DATA_TYPE) + new CanalJsonDeserializationSchema.Builder(catalogTables) .setDatabase("^my.*") .setTable("^prod.*") .build(); @@ -163,39 +167,39 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali List expected = Arrays.asList( - "SeaTunnelRow{tableId=, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[106, hammer, null, 1.0]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[106, hammer, null, 1.0]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[101, scooter, Small 2-wheel scooter, 5.17]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[102, car battery, 12V car battery, 5.17]}", - "SeaTunnelRow{tableId=, kind=-D, fields=[102, car battery, 12V car battery, 5.17]}", - "SeaTunnelRow{tableId=, kind=-D, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}"); + "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, null, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, null, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[101, scooter, Small 2-wheel scooter, 5.17]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[102, car battery, 12V car battery, 5.17]}", + "SeaTunnelRow{tableId=.., kind=-D, fields=[102, car battery, 12V car battery, 5.17]}", + "SeaTunnelRow{tableId=.., kind=-D, fields=[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(PHYSICAL_DATA_TYPE); + new CanalJsonSerializationSchema(SEATUNNEL_ROW_TYPE); List result = new ArrayList<>(); for (SeaTunnelRow rowData : collector.list) { result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); @@ -238,7 +242,7 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali private CanalJsonDeserializationSchema createCanalJsonDeserializationSchema( String database, String table) { - return CanalJsonDeserializationSchema.builder(PHYSICAL_DATA_TYPE) + return CanalJsonDeserializationSchema.builder(catalogTables) .setDatabase(database) .setTable(table) .setIgnoreParseErrors(false) diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index c15be09deea..67d499efd91 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -19,6 +19,8 @@ package org.apache.seatunnel.format.json.debezium; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -48,15 +50,17 @@ public class DebeziumJsonSerDeSchemaTest { private static final String FORMAT = "Debezium"; - private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = + private static final SeaTunnelRowType SEATUNNEL_ROW_TYPE = new SeaTunnelRowType( new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); + private static final CatalogTable catalogTables = + CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); @Test void testNullRowMessages() throws Exception { DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + new DebeziumJsonDeserializationSchema(catalogTables, false); SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(null, collector); @@ -72,7 +76,7 @@ public void testSerializationAndSchemaExcludeDeserialization() throws Exception @Test public void testDeserializeNoJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + new DebeziumJsonDeserializationSchema(catalogTables, false); final SimpleCollector collector = new SimpleCollector(); String noJsonMsg = "{]"; @@ -90,7 +94,7 @@ public void testDeserializeNoJson() throws Exception { @Test public void testDeserializeEmptyJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + new DebeziumJsonDeserializationSchema(catalogTables, false); final SimpleCollector collector = new SimpleCollector(); String emptyMsg = "{}"; SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, emptyMsg); @@ -106,7 +110,7 @@ public void testDeserializeEmptyJson() throws Exception { @Test public void testDeserializeNoDataJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + new DebeziumJsonDeserializationSchema(catalogTables, false); final SimpleCollector collector = new SimpleCollector(); String noDataMsg = "{\"op\":\"u\"}"; SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, noDataMsg); @@ -132,7 +136,7 @@ public void testDeserializeNoDataJson() throws Exception { @Test public void testDeserializeUnknownOperationTypeJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + new DebeziumJsonDeserializationSchema(catalogTables, false); final SimpleCollector collector = new SimpleCollector(); String unknownType = "XX"; String unknownOperationMsg = @@ -161,8 +165,7 @@ private void testSerializationDeserialization(String resourceFile, boolean schem throws Exception { List lines = readLines(resourceFile); DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema( - PHYSICAL_DATA_TYPE, true, schemaInclude, null); + new DebeziumJsonDeserializationSchema(catalogTables, true, schemaInclude); SimpleCollector collector = new SimpleCollector(); @@ -172,32 +175,32 @@ private void testSerializationDeserialization(String resourceFile, boolean schem List expected = Arrays.asList( - "SeaTunnelRow{tableId=, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); + "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=.., kind=-D, fields=[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(PHYSICAL_DATA_TYPE); + new DebeziumJsonSerializationSchema(SEATUNNEL_ROW_TYPE); actual = new ArrayList<>(); for (SeaTunnelRow rowData : collector.list) { diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java index 1f44c7cfbcb..04fea16ecad 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java @@ -19,6 +19,8 @@ package org.apache.seatunnel.format.json.ogg; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -48,16 +50,18 @@ public class OggJsonSerDeSchemaTest { private static final String FORMAT = "Ogg"; - private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = + private static final SeaTunnelRowType SEATUNNEL_ROW_TYPE = new SeaTunnelRowType( new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); + private static final CatalogTable catalogTables = + CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); @Test public void testFilteringTables() throws Exception { List lines = readLines("ogg-data-filter-table.txt"); OggJsonDeserializationSchema deserializationSchema = - new OggJsonDeserializationSchema.Builder(PHYSICAL_DATA_TYPE) + new OggJsonDeserializationSchema.Builder(catalogTables) .setDatabase("^OG.*") .setTable("^TBL.*") .build(); @@ -168,33 +172,33 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa List expected = Arrays.asList( - "SeaTunnelRow{tableId=, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); + "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); assertEquals(expected, actual); // test Serialization OggJsonSerializationSchema serializationSchema = - new OggJsonSerializationSchema(PHYSICAL_DATA_TYPE); + new OggJsonSerializationSchema(SEATUNNEL_ROW_TYPE); List result = new ArrayList<>(); for (SeaTunnelRow rowData : collector.list) { result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); @@ -232,7 +236,7 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa private OggJsonDeserializationSchema createOggJsonDeserializationSchema( String database, String table) { - return OggJsonDeserializationSchema.builder(PHYSICAL_DATA_TYPE) + return OggJsonDeserializationSchema.builder(catalogTables) .setDatabase(database) .setTable(table) .setIgnoreParseErrors(false) From 925adcb34f9596397ef9f3eca6990c99dd912483 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Thu, 9 May 2024 20:38:24 +0800 Subject: [PATCH 57/71] fix --- .../connectors/seatunnel/kafka/source/KafkaSourceConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index bbca260f3ca..6afff6e83e3 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -204,7 +204,7 @@ private CatalogTable createCatalogTable(ReadonlyConfig readonlyConfig) { new SeaTunnelRowType( new String[] {"content"}, new SeaTunnelDataType[] { - BasicType.STRING_TYPE + BasicType.STRING_TYPE }), 0, false, From 412936966d658d43e2e3860074e9e4a2719c3067 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 10 May 2024 10:24:38 +0800 Subject: [PATCH 58/71] fix --- .../seatunnel/amazonsqs/source/AmazonSqsSource.java | 11 ++++++----- .../seatunnel/fake/source/FakeDataGenerator.java | 3 +-- .../seatunnel/rocketmq/source/RocketMqSource.java | 6 ++++-- .../format/json/JsonDeserializationSchema.java | 2 +- 4 files changed, 12 insertions(+), 10 deletions(-) diff --git a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java index 086ad1ffc47..c4c3ad372d2 100644 --- a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java +++ b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java @@ -67,6 +67,7 @@ public class AmazonSqsSource extends AbstractSingleSplitSource private AmazonSqsSourceOptions amazonSqsSourceOptions; private DeserializationSchema deserializationSchema; private SeaTunnelRowType typeInfo; + private CatalogTable catalogTable; @Override public String getPluginName() { @@ -85,9 +86,9 @@ public void prepare(Config pluginConfig) throws PrepareFailException { "PluginName: %s, PluginType: %s, Message: %s", getPluginName(), PluginType.SOURCE, result.getMsg())); } - amazonSqsSourceOptions = new AmazonSqsSourceOptions(pluginConfig); - typeInfo = CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); - + this.amazonSqsSourceOptions = new AmazonSqsSourceOptions(pluginConfig); + this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); + this.typeInfo = catalogTable.getSeaTunnelRowType(); setDeserialization(pluginConfig); } @@ -110,11 +111,11 @@ public AbstractSingleSplitReader createReader( private void setDeserialization(Config config) { if (config.hasPath(TableSchemaOptions.SCHEMA.key())) { - CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); MessageFormat format = ReadonlyConfig.fromConfig(config).get(FORMAT); switch (format) { case JSON: - deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + deserializationSchema = + new JsonDeserializationSchema(catalogTable, false, false); break; case TEXT: String delimiter = DEFAULT_FIELD_DELIMITER; diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java index 8bacb1a94d5..8a15e73cd3d 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java @@ -53,8 +53,7 @@ public FakeDataGenerator(FakeConfig fakeConfig) { this.jsonDeserializationSchema = fakeConfig.getFakeRows() == null ? null - : new JsonDeserializationSchema( - false, false, catalogTable.getSeaTunnelRowType()); + : new JsonDeserializationSchema(catalogTable, false, false); this.fakeDataRandomUtils = new FakeDataRandomUtils(fakeConfig); } diff --git a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java index ceebd71404a..c3bff2dfe0d 100644 --- a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java +++ b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java @@ -30,6 +30,7 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -252,14 +253,15 @@ public SourceSplitEnumerator restoreEn private void setDeserialization(Config config) { if (config.hasPath(ConsumerConfig.SCHEMA.key())) { - typeInfo = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); + CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); SchemaFormat format = SchemaFormat.JSON; if (config.hasPath(FORMAT.key())) { format = SchemaFormat.find(config.getString(FORMAT.key())); } switch (format) { case JSON: - deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + deserializationSchema = + new JsonDeserializationSchema(catalogTable, false, false); break; case TEXT: String delimiter = DEFAULT_FIELD_DELIMITER; diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 6666a8dad9e..215a771f69f 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -82,6 +82,7 @@ public JsonDeserializationSchema( CommonErrorCodeDeprecated.ILLEGAL_ARGUMENT, "JSON format doesn't support failOnMissingField and ignoreParseErrors are both enabled."); } + this.catalogTable = catalogTable; this.rowType = checkNotNull(catalogTable.getSeaTunnelRowType()); this.failOnMissingField = failOnMissingField; this.ignoreParseErrors = ignoreParseErrors; @@ -93,7 +94,6 @@ public JsonDeserializationSchema( objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); } objectMapper.configure(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(), true); - this.catalogTable = catalogTable; } private static boolean hasDecimalType(SeaTunnelDataType dataType) { From d68ed26b8ae0e9a7eb521ddaedfc7fcde36ad7ad Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 10 May 2024 13:59:42 +0800 Subject: [PATCH 59/71] fix --- .../connectors/seatunnel/rocketmq/source/RocketMqSource.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java index c3bff2dfe0d..39052e51721 100644 --- a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java +++ b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java @@ -77,6 +77,7 @@ public class RocketMqSource private final ConsumerMetadata metadata = new ConsumerMetadata(); private JobContext jobContext; private SeaTunnelRowType typeInfo; + private CatalogTable catalogTable; private DeserializationSchema deserializationSchema; private long discoveryIntervalMillis = ConsumerConfig.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS.defaultValue(); @@ -216,6 +217,8 @@ public void prepare(Config config) throws PrepareFailException { this.discoveryIntervalMillis = config.getLong(ConsumerConfig.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS.key()); } + this.catalogTable = CatalogTableUtil.buildWithConfig(config); + this.typeInfo = catalogTable.getSeaTunnelRowType(); // set deserialization setDeserialization(config); @@ -253,7 +256,6 @@ public SourceSplitEnumerator restoreEn private void setDeserialization(Config config) { if (config.hasPath(ConsumerConfig.SCHEMA.key())) { - CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); SchemaFormat format = SchemaFormat.JSON; if (config.hasPath(FORMAT.key())) { format = SchemaFormat.find(config.getString(FORMAT.key())); From 4a75154a76ce4a10b717e931a2835e8fd4b946f5 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 10 May 2024 16:31:07 +0800 Subject: [PATCH 60/71] fix --- .../source/reader/AbstractReadStrategy.java | 4 ++ .../file/source/reader/JsonReadStrategy.java | 10 +++-- .../json/JsonDeserializationSchema.java | 12 +++--- .../json/JsonRowDataSerDeSchemaTest.java | 43 ++++++++++++++----- 4 files changed, 48 insertions(+), 21 deletions(-) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java index 33b688af060..efa54e6fbac 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java @@ -19,6 +19,8 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -59,6 +61,7 @@ public abstract class AbstractReadStrategy implements ReadStrategy { protected HadoopConf hadoopConf; protected SeaTunnelRowType seaTunnelRowType; + protected CatalogTable catalogTable; protected SeaTunnelRowType seaTunnelRowTypeWithPartition; protected Config pluginConfig; protected List fileNames = new ArrayList<>(); @@ -146,6 +149,7 @@ public void setPluginConfig(Config pluginConfig) { pluginConfig.getString(BaseSourceConfigOptions.FILE_FILTER_PATTERN.key()); this.pattern = Pattern.compile(Matcher.quoteReplacement(filterPattern)); } + this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); } @Override diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java index 6c58e368721..30f459e05c6 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.exception.CommonError; @@ -45,6 +46,7 @@ public class JsonReadStrategy extends AbstractReadStrategy { private DeserializationSchema deserializationSchema; private CompressFormat compressFormat = BaseSourceConfigOptions.COMPRESS_CODEC.defaultValue(); private String encoding = BaseSourceConfigOptions.ENCODING.defaultValue(); + private CatalogTable catalogTable; @Override public void init(HadoopConf conf) { @@ -58,17 +60,17 @@ public void init(HadoopConf conf) { ReadonlyConfig.fromConfig(pluginConfig) .getOptional(BaseSourceConfigOptions.ENCODING) .orElse(StandardCharsets.UTF_8.name()); + CatalogTable catalogTable1 = super.catalogTable; } @Override public void setSeaTunnelRowTypeInfo(SeaTunnelRowType seaTunnelRowType) { super.setSeaTunnelRowTypeInfo(seaTunnelRowType); + catalogTable = super.catalogTable; if (isMergePartition) { - deserializationSchema = - new JsonDeserializationSchema(false, false, this.seaTunnelRowTypeWithPartition); + deserializationSchema = new JsonDeserializationSchema(catalogTable, false, false); } else { - deserializationSchema = - new JsonDeserializationSchema(false, false, this.seaTunnelRowType); + deserializationSchema = new JsonDeserializationSchema(catalogTable, false, false); } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 215a771f69f..6929670ca50 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -68,12 +68,12 @@ public class JsonDeserializationSchema implements DeserializationSchema { - new JsonDeserializationSchema(true, true, null); + new JsonDeserializationSchema(null, true, true); }, "expecting exception message: " + errorMessage); assertEquals(actual.getMessage(), errorMessage); @@ -426,8 +444,11 @@ public void testDeserializationNoJson() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + String noJson = "{]"; - final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, false, schema); + final JsonDeserializationSchema deser = + new JsonDeserializationSchema(catalogTables, false, false); SeaTunnelRuntimeException expected = CommonError.jsonOperationError("Common", noJson); SeaTunnelRuntimeException actual = From 24d1592e1c1ea4373c2a156653e97c6e9b236232 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 10 May 2024 18:42:16 +0800 Subject: [PATCH 61/71] fix --- .../seatunnel/connectors/seatunnel/http/source/HttpSource.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java index 4e3f3ab1a59..754a7b93664 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java @@ -127,8 +127,7 @@ protected void buildSchemaWithConfig(Config pluginConfig) { switch (format) { case JSON: this.deserializationSchema = - new JsonDeserializationSchema( - false, false, catalogTable.getSeaTunnelRowType()); + new JsonDeserializationSchema(catalogTable, false, false); if (pluginConfig.hasPath(HttpConfig.JSON_FIELD.key())) { jsonField = getJsonField(pluginConfig.getConfig(HttpConfig.JSON_FIELD.key())); From 65587e04545bb64475a441d4b18c67d2a2eb55b4 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 10 May 2024 19:02:17 +0800 Subject: [PATCH 62/71] fix --- .../file/source/reader/JsonReadStrategy.java | 6 +++--- .../seatunnel/google/sheets/source/SheetsSource.java | 11 +++++++---- .../deserialize/GoogleSheetsDeserializerTest.java | 7 ++++++- .../pulsar/source/PulsarCanalDecoratorTest.java | 7 +++++-- .../seatunnel/rabbitmq/source/RabbitmqSource.java | 6 +++--- .../seatunnel/redis/source/RedisSource.java | 2 +- 6 files changed, 25 insertions(+), 14 deletions(-) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java index 30f459e05c6..a39de3fa7b0 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.exception.CommonError; @@ -56,17 +57,16 @@ public void init(HadoopConf conf) { pluginConfig.getString(BaseSourceConfigOptions.COMPRESS_CODEC.key()); compressFormat = CompressFormat.valueOf(compressCodec.toUpperCase()); } - encoding = + this.encoding = ReadonlyConfig.fromConfig(pluginConfig) .getOptional(BaseSourceConfigOptions.ENCODING) .orElse(StandardCharsets.UTF_8.name()); - CatalogTable catalogTable1 = super.catalogTable; + this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); } @Override public void setSeaTunnelRowTypeInfo(SeaTunnelRowType seaTunnelRowType) { super.setSeaTunnelRowTypeInfo(seaTunnelRowType); - catalogTable = super.catalogTable; if (isMergePartition) { deserializationSchema = new JsonDeserializationSchema(catalogTable, false, false); } else { diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java index 0661f6db366..279a39256ec 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -46,6 +47,7 @@ public class SheetsSource extends AbstractSingleSplitSource { private SeaTunnelRowType seaTunnelRowType; + private CatalogTable catalogTable; private SheetsParameters sheetsParameters; @@ -75,12 +77,13 @@ public void prepare(Config pluginConfig) throws PrepareFailException { } this.sheetsParameters = new SheetsParameters().buildWithConfig(pluginConfig); if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { - this.seaTunnelRowType = - CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); + this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); } else { - this.seaTunnelRowType = CatalogTableUtil.buildSimpleTextSchema(); + this.catalogTable = CatalogTableUtil.buildSimpleTextTable(); } - this.deserializationSchema = new JsonDeserializationSchema(false, false, seaTunnelRowType); + + this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); + this.deserializationSchema = new JsonDeserializationSchema(catalogTable, false, false); } @Override diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java index 8c81a51c581..c55228471c2 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java @@ -18,6 +18,8 @@ package org.apache.seatunnel.connectors.seatunnel.google.sheets.deserialize; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -39,8 +41,11 @@ public class GoogleSheetsDeserializerTest { public void testJsonParseError() { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); + + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + final DeserializationSchema deser = - new JsonDeserializationSchema(false, false, schema); + new JsonDeserializationSchema(catalogTables, false, false); final GoogleSheetsDeserializer googleSheetsDeser = new GoogleSheetsDeserializer(schema.getFieldNames(), deser); List row = new ArrayList<>(); diff --git a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java index 0076bbbdbea..7b1ee39fd48 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java @@ -18,6 +18,8 @@ package org.apache.seatunnel.connectors.seatunnel.pulsar.source; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -55,9 +57,10 @@ void decoder() throws IOException { }; SeaTunnelRowType seaTunnelRowType = new SeaTunnelRowType(fieldNames, dataTypes); - + CatalogTable catalogTables = + CatalogTableUtil.getCatalogTable("", "", "", "", seaTunnelRowType); CanalJsonDeserializationSchema canalJsonDeserializationSchema = - CanalJsonDeserializationSchema.builder(seaTunnelRowType).build(); + CanalJsonDeserializationSchema.builder(catalogTables).build(); PulsarCanalDecorator pulsarCanalDecorator = new PulsarCanalDecorator(canalJsonDeserializationSchema); diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java index 049c90c3ecd..5684b21d4f3 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java @@ -28,11 +28,11 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; import org.apache.seatunnel.common.constants.JobMode; @@ -134,7 +134,7 @@ public void setJobContext(JobContext jobContext) { private void setDeserialization(Config config) { // TODO: format SPI // only support json deserializationSchema - SeaTunnelRowType rowType = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); - this.deserializationSchema = new JsonDeserializationSchema(false, false, rowType); + CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); + this.deserializationSchema = new JsonDeserializationSchema(catalogTable, false, false); } } diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java index 3818848fc15..be1ba32acb9 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java @@ -72,7 +72,7 @@ public RedisSource(ReadonlyConfig readonlyConfig) { this.catalogTable = CatalogTableUtil.buildWithConfig(readonlyConfig); this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); this.deserializationSchema = - new JsonDeserializationSchema(false, false, seaTunnelRowType); + new JsonDeserializationSchema(catalogTable, false, false); } } else { this.catalogTable = CatalogTableUtil.buildSimpleTextTable(); From 0dbd15c4eec4bfee9a91d0a4c7f3a2c121f127fd Mon Sep 17 00:00:00 2001 From: zhilinli Date: Sat, 11 May 2024 14:15:59 +0800 Subject: [PATCH 63/71] fix --- .../file/source/reader/AbstractReadStrategy.java | 4 ---- .../file/source/reader/JsonReadStrategy.java | 12 +++++------- .../format/json/JsonDeserializationSchema.java | 12 ++++++------ 3 files changed, 11 insertions(+), 17 deletions(-) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java index efa54e6fbac..33b688af060 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java @@ -19,8 +19,6 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -61,7 +59,6 @@ public abstract class AbstractReadStrategy implements ReadStrategy { protected HadoopConf hadoopConf; protected SeaTunnelRowType seaTunnelRowType; - protected CatalogTable catalogTable; protected SeaTunnelRowType seaTunnelRowTypeWithPartition; protected Config pluginConfig; protected List fileNames = new ArrayList<>(); @@ -149,7 +146,6 @@ public void setPluginConfig(Config pluginConfig) { pluginConfig.getString(BaseSourceConfigOptions.FILE_FILTER_PATTERN.key()); this.pattern = Pattern.compile(Matcher.quoteReplacement(filterPattern)); } - this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); } @Override diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java index a39de3fa7b0..6c58e368721 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/JsonReadStrategy.java @@ -20,8 +20,6 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.exception.CommonError; @@ -47,7 +45,6 @@ public class JsonReadStrategy extends AbstractReadStrategy { private DeserializationSchema deserializationSchema; private CompressFormat compressFormat = BaseSourceConfigOptions.COMPRESS_CODEC.defaultValue(); private String encoding = BaseSourceConfigOptions.ENCODING.defaultValue(); - private CatalogTable catalogTable; @Override public void init(HadoopConf conf) { @@ -57,20 +54,21 @@ public void init(HadoopConf conf) { pluginConfig.getString(BaseSourceConfigOptions.COMPRESS_CODEC.key()); compressFormat = CompressFormat.valueOf(compressCodec.toUpperCase()); } - this.encoding = + encoding = ReadonlyConfig.fromConfig(pluginConfig) .getOptional(BaseSourceConfigOptions.ENCODING) .orElse(StandardCharsets.UTF_8.name()); - this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); } @Override public void setSeaTunnelRowTypeInfo(SeaTunnelRowType seaTunnelRowType) { super.setSeaTunnelRowTypeInfo(seaTunnelRowType); if (isMergePartition) { - deserializationSchema = new JsonDeserializationSchema(catalogTable, false, false); + deserializationSchema = + new JsonDeserializationSchema(false, false, this.seaTunnelRowTypeWithPartition); } else { - deserializationSchema = new JsonDeserializationSchema(catalogTable, false, false); + deserializationSchema = + new JsonDeserializationSchema(false, false, this.seaTunnelRowType); } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 6929670ca50..215a771f69f 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -68,12 +68,12 @@ public class JsonDeserializationSchema implements DeserializationSchema Date: Sat, 11 May 2024 15:17:39 +0800 Subject: [PATCH 64/71] fix --- .../seatunnel/format/json/JsonDeserializationSchema.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 215a771f69f..7ab39192c58 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -73,6 +73,14 @@ public JsonDeserializationSchema( this.failOnMissingField = failOnMissingField; this.ignoreParseErrors = ignoreParseErrors; this.rowType = checkNotNull(rowType); + this.runtimeConverter = + new JsonToRowConverters(failOnMissingField, ignoreParseErrors) + .createRowConverter(checkNotNull(rowType)); + + if (hasDecimalType(rowType)) { + objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); + } + objectMapper.configure(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(), true); } public JsonDeserializationSchema( From 1b9c1924ef698303b9f1d7d1c1b762e428a2a536 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Sat, 11 May 2024 17:02:57 +0800 Subject: [PATCH 65/71] fix --- .../connectors/seatunnel/pulsar/source/PulsarSource.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java index 032bac62bf2..ffb26e77ada 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java @@ -53,6 +53,7 @@ import org.apache.seatunnel.connectors.seatunnel.pulsar.source.format.PulsarCanalDecorator; import org.apache.seatunnel.connectors.seatunnel.pulsar.source.reader.PulsarSourceReader; import org.apache.seatunnel.connectors.seatunnel.pulsar.source.split.PulsarPartitionSplit; +import org.apache.seatunnel.format.json.JsonDeserializationSchema; import org.apache.seatunnel.format.json.canal.CanalJsonDeserializationSchema; import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; @@ -307,8 +308,7 @@ private void setDeserialization(Config config) { } switch (format.toUpperCase()) { case "JSON": - // deserializationSchema = new - // JsonDeserializationSchema(false, false, typeInfo); + deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); break; case "CANAL_JSON": deserializationSchema = @@ -324,8 +324,7 @@ private void setDeserialization(Config config) { } } else { typeInfo = CatalogTableUtil.buildSimpleTextSchema(); - // this.deserializationSchema = new JsonDeserializationSchema(false, false, - // typeInfo); + this.deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); } } From d4fe2d5bd86bb8ca1c22bfd2d3b66e7d2ee2d65e Mon Sep 17 00:00:00 2001 From: zhilinli Date: Mon, 13 May 2024 10:53:59 +0800 Subject: [PATCH 66/71] fix --- .../connectors/seatunnel/pulsar/source/PulsarSource.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java index ffb26e77ada..f44b99ccc5e 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java @@ -308,7 +308,8 @@ private void setDeserialization(Config config) { } switch (format.toUpperCase()) { case "JSON": - deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + deserializationSchema = + new JsonDeserializationSchema(catalogTable, false, false); break; case "CANAL_JSON": deserializationSchema = From 878d47144009c4e191ad41ba3b49e29f9763c0f4 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Mon, 13 May 2024 17:49:22 +0800 Subject: [PATCH 67/71] fix --- .../connectors/seatunnel/pulsar/source/PulsarSource.java | 8 +++----- .../seatunnel/format/json/JsonDeserializationSchema.java | 7 ++++++- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java index f44b99ccc5e..9d15856164d 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java @@ -27,7 +27,6 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportParallelism; -import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -301,20 +300,19 @@ private void setPartitionDiscoverer(Config config) { private void setDeserialization(Config config) { if (config.hasPath(SCHEMA.key())) { - CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); + typeInfo = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); String format = FORMAT.defaultValue(); if (config.hasPath(FORMAT.key())) { format = config.getString(FORMAT.key()); } switch (format.toUpperCase()) { case "JSON": - deserializationSchema = - new JsonDeserializationSchema(catalogTable, false, false); + deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); break; case "CANAL_JSON": deserializationSchema = new PulsarCanalDecorator( - CanalJsonDeserializationSchema.builder(catalogTable) + CanalJsonDeserializationSchema.builder(null) .setIgnoreParseErrors(true) .build()); break; diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index 7ab39192c58..6580f3d49e4 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -70,9 +70,14 @@ public class JsonDeserializationSchema implements DeserializationSchema Date: Mon, 13 May 2024 19:38:48 +0800 Subject: [PATCH 68/71] fix --- .../connectors/seatunnel/pulsar/source/PulsarSource.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java index 9d15856164d..b998d85762a 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java @@ -27,6 +27,7 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -300,7 +301,8 @@ private void setPartitionDiscoverer(Config config) { private void setDeserialization(Config config) { if (config.hasPath(SCHEMA.key())) { - typeInfo = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); + CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); + typeInfo = catalogTable.getSeaTunnelRowType(); String format = FORMAT.defaultValue(); if (config.hasPath(FORMAT.key())) { format = config.getString(FORMAT.key()); @@ -312,7 +314,7 @@ private void setDeserialization(Config config) { case "CANAL_JSON": deserializationSchema = new PulsarCanalDecorator( - CanalJsonDeserializationSchema.builder(null) + CanalJsonDeserializationSchema.builder(catalogTable) .setIgnoreParseErrors(true) .build()); break; From ac2f9254f937b26ed023a5eba21bbd123776b47b Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 14 May 2024 20:07:32 +0800 Subject: [PATCH 69/71] fix --- .../seatunnel/format/avro/AvroDeserializationSchema.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java index d54f024a5e6..32396c245ea 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java @@ -39,10 +39,10 @@ public class AvroDeserializationSchema implements DeserializationSchema Date: Tue, 14 May 2024 20:14:50 +0800 Subject: [PATCH 70/71] fix --- .../connectors/seatunnel/kafka/source/KafkaSourceConfig.java | 2 +- .../seatunnel/format/avro/AvroSerializationSchemaTest.java | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index 6afff6e83e3..232ee2f0e0d 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -263,7 +263,7 @@ private DeserializationSchema createDeserializationSchema( boolean includeSchema = readonlyConfig.get(DEBEZIUM_RECORD_INCLUDE_SCHEMA); return new DebeziumJsonDeserializationSchema(catalogTable, true, includeSchema); case AVRO: - return new AvroDeserializationSchema(seaTunnelRowType, catalogTable); + return new AvroDeserializationSchema(catalogTable); default: throw new SeaTunnelJsonFormatException( CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, diff --git a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java index e290dc8ca01..1e1554be712 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.format.avro; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -158,11 +160,12 @@ private SeaTunnelRowType buildSeaTunnelRowType() { @Test public void testSerialization() throws IOException { SeaTunnelRowType rowType = buildSeaTunnelRowType(); + CatalogTable catalogTable = CatalogTableUtil.getCatalogTable("", "", "", "", rowType); SeaTunnelRow seaTunnelRow = buildSeaTunnelRow(); AvroSerializationSchema serializationSchema = new AvroSerializationSchema(rowType); byte[] bytes = serializationSchema.serialize(seaTunnelRow); AvroDeserializationSchema deserializationSchema = - new AvroDeserializationSchema(rowType, null); + new AvroDeserializationSchema(catalogTable); SeaTunnelRow deserialize = deserializationSchema.deserialize(bytes); String[] strArray1 = (String[]) seaTunnelRow.getField(1); String[] strArray2 = (String[]) deserialize.getField(1); From 56c900cc4e1ae58fb8a70796e245a70ae8da1daf Mon Sep 17 00:00:00 2001 From: zhilinli Date: Tue, 14 May 2024 23:12:09 +0800 Subject: [PATCH 71/71] fix --- .../apache/seatunnel/e2e/connector/kafka/KafkaIT.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index 591d09af38f..2f1c92048e0 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -20,6 +20,8 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -370,8 +372,10 @@ public void testFakeSourceToKafkaAvroFormat(TestContainer container) subRow }; SeaTunnelRowType fake_source_row_type = new SeaTunnelRowType(fieldNames, fieldTypes); + CatalogTable catalogTable = + CatalogTableUtil.getCatalogTable("", "", "", "", fake_source_row_type); AvroDeserializationSchema avroDeserializationSchema = - new AvroDeserializationSchema(fake_source_row_type, null); + new AvroDeserializationSchema(catalogTable); List kafkaSTRow = getKafkaSTRow( "test_avro_topic_fake_source", @@ -420,8 +424,11 @@ public void testKafkaAvroToAssert(TestContainer container) Container.ExecResult execResult = container.executeJob("/avro/kafka_avro_to_assert.conf"); Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + CatalogTable catalogTable = + CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + AvroDeserializationSchema avroDeserializationSchema = - new AvroDeserializationSchema(SEATUNNEL_ROW_TYPE, null); + new AvroDeserializationSchema(catalogTable); List kafkaSTRow = getKafkaSTRow( "test_avro_topic",