Hisoka-X commented on code in PR #5635:
URL: https://github.com/apache/seatunnel/pull/5635#discussion_r1364831676
##########
seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/textFormatIT/kafka_source_text_to_console.conf:
##########
@@ -35,23 +35,84 @@ source {
kafka.auto.offset.reset = "earliest"
format_error_handle_way = fail
schema = {
- fields {
- id = bigint
- c_map = "map<string, smallint>"
- c_array = "array<tinyint>"
- c_string = string
- c_boolean = boolean
- c_tinyint = tinyint
- c_smallint = smallint
- c_int = int
- c_bigint = bigint
- c_float = float
- c_double = double
- c_decimal = "decimal(2, 1)"
- c_bytes = bytes
- c_date = date
- c_timestamp = timestamp
+ columns = [
Review Comment:
Could you create new conf not replace old one? I think we should make sure
both two config way worked fine for now.
##########
seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java:
##########
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.kafka.source;
+
+import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode;
+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.TableSchema;
+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.config.CheckResult;
+import org.apache.seatunnel.common.constants.PluginType;
+import org.apache.seatunnel.common.exception.CommonErrorCode;
+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;
+import
org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorException;
+import
org.apache.seatunnel.format.compatible.kafka.connect.json.CompatibleKafkaConnectDeserializationSchema;
+import
org.apache.seatunnel.format.compatible.kafka.connect.json.KafkaConnectJsonFormatOptions;
+import org.apache.seatunnel.format.json.JsonDeserializationSchema;
+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.text.TextDeserializationSchema;
+import org.apache.seatunnel.format.text.constant.TextFormatConstant;
+
+import org.apache.commons.collections4.MapUtils;
+import org.apache.kafka.common.TopicPartition;
+
+import lombok.Getter;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+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;
+import static
org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.FORMAT;
+import static
org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.KAFKA_CONFIG;
+import static
org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS;
+import static
org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.MESSAGE_FORMAT_ERROR_HANDLE_WAY_OPTION;
+import static
org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PATTERN;
+import static
org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE;
+import static
org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE_OFFSETS;
+import static
org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE_TIMESTAMP;
+import static
org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.TOPIC;
+
+public class KafkaSourceConfig implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ @Getter private final ConsumerMetadata metadata;
+
+ @Getter private final DeserializationSchema<SeaTunnelRow>
deserializationSchema;
+
+ @Getter private final CatalogTable catalogTable;
+
+ @Getter private final MessageFormatErrorHandleWay
messageFormatErrorHandleWay;
+
+ @Getter private final long discoveryIntervalMillis;
+
+ public KafkaSourceConfig(ReadonlyConfig readonlyConfig) {
+ this.metadata = createConsumerMetadata(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);
+ }
+
+ private ConsumerMetadata createConsumerMetadata(ReadonlyConfig
readonlyConfig) {
+ CheckResult result = readonlyConfig.checkAllExists(TOPIC,
BOOTSTRAP_SERVERS);
+ if (!result.isSuccess()) {
+ throw new KafkaConnectorException(
+ SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED,
+ String.format(
+ "PluginName: %s, PluginType: %s, Message: %s",
+ CONNECTOR_IDENTITY, PluginType.SOURCE,
result.getMsg()));
+ }
Review Comment:
I think `checkAllExists` is unnecessary. As `ReadonlyConfig`, if it
implement `TableSourceFactory`, we will check whether all configs are legal.
Please refer
https://github.com/apache/seatunnel/blob/aefc135195f5a10e37aec84e0b7c29708342cfcd/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java#L116
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]