hailin0 commented on code in PR #3230: URL: https://github.com/apache/incubator-seatunnel/pull/3230#discussion_r1013934844
########## seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/log4j.properties: ########## @@ -0,0 +1,22 @@ +# Review Comment: Remove this file. This is the common configuration of e2e https://github.com/apache/incubator-seatunnel/blob/dev/seatunnel-e2e/seatunnel-e2e-common/src/test/resources/log4j2.properties ########## seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java: ########## @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.e2e.connector.kafka; + +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.BasicType; +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.PrimitiveByteArrayType; +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.connectors.seatunnel.kafka.serialize.DefaultSeaTunnelRowSerializer; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.format.text.TextSerializationSchema; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.shaded.com.google.common.collect.Lists; +import org.testcontainers.shaded.org.awaitility.Awaitility; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.math.BigDecimal; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +@Slf4j +public class KafkaIT extends TestSuiteBase implements TestResource { + private static final int KAFKA_PORT = 9093; + + private static final String KAFKA_HOST = "kafkaCluster"; + + private KafkaProducer<byte[], byte[]> producer; + + private KafkaContainer kafkaContainer; + + @BeforeAll + @Override + public void startUp() throws Exception { + kafkaContainer = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:6.2.1")) + .withNetwork(NETWORK) + .withNetworkAliases(KAFKA_HOST) + .withLogConsumer(new Slf4jLogConsumer(log)); Review Comment: ```suggestion .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger("confluentinc/cp-kafka:6.2.1"))); ``` ########## seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java: ########## @@ -188,23 +207,32 @@ private void restoreState(List<KafkaSinkState> states) { } } - private Function<SeaTunnelRow, String> createPartitionExtractor(Config pluginConfig, - SeaTunnelRowType seaTunnelRowType) { - if (!pluginConfig.hasPath(PARTITION_KEY)){ + private Function<SeaTunnelRow, SeaTunnelRow> createPartitionExtractor(SeaTunnelRowType seaTunnelRowType) { Review Comment: suggestion remove this method ########## seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java: ########## @@ -25,34 +25,42 @@ public class DefaultSeaTunnelRowSerializer implements SeaTunnelRowSerializer<byte[], byte[]> { Review Comment: suggestion ```java public class DefaultSeaTunnelRowSerializer implements SeaTunnelRowSerializer<byte[], byte[]> { private Integer partition; private final String topic; private final SerializationSchema keySerialization; private final SerializationSchema valueSerialization; public DefaultSeaTunnelRowSerializer(String topic, SeaTunnelRowType seaTunnelRowType) { this(topic, element -> null, createSerializationSchema(seaTunnelRowType)); } public DefaultSeaTunnelRowSerializer(String topic, Integer partition, SeaTunnelRowType seaTunnelRowType) { this(topic, seaTunnelRowType); this.partition = partition; } public DefaultSeaTunnelRowSerializer(String topic, List<String> keyFieldNames, SeaTunnelRowType seaTunnelRowType) { this(topic, createKeySerializationSchema(keyFieldNames, seaTunnelRowType), createSerializationSchema(seaTunnelRowType)); } public DefaultSeaTunnelRowSerializer(String topic, SerializationSchema keySerialization, SerializationSchema valueSerialization) { this.topic = topic; this.keySerialization = keySerialization; this.valueSerialization = valueSerialization; } @Override public ProducerRecord<byte[], byte[]> serializeRow(SeaTunnelRow row) { return new ProducerRecord<>(topic, partition, keySerialization.serialize(row), valueSerialization.serialize(row)); } private static SerializationSchema createSerializationSchema(SeaTunnelRowType rowType) { return new JsonSerializationSchema(rowType); } private static SerializationSchema createKeySerializationSchema(List<String> keyFieldNames, SeaTunnelRowType seaTunnelRowType) { int[] keyFieldIndexArr = new int[keyFieldNames.size()]; SeaTunnelDataType[] keyFieldDataTypeArr = new SeaTunnelDataType[keyFieldNames.size()]; for (int i = 0; i < keyFieldNames.size(); i++) { String keyFieldName = keyFieldNames.get(i); int rowFieldIndex = seaTunnelRowType.indexOf(keyFieldName); keyFieldIndexArr[i] = rowFieldIndex; keyFieldDataTypeArr[i] = seaTunnelRowType.getFieldType(rowFieldIndex); } SeaTunnelRowType keyType = new SeaTunnelRowType(keyFieldNames.toArray(new String[0]), keyFieldDataTypeArr); SerializationSchema keySerializationSchema = createSerializationSchema(keyType); Function<SeaTunnelRow, SeaTunnelRow> keyDataExtractor = row -> { Object[] keyFields = new Object[keyFieldIndexArr.length]; for (int i = 0; i < keyFieldIndexArr.length; i++) { keyFields[i] = row.getField(keyFieldIndexArr[i]); } return new SeaTunnelRow(keyFields); }; return row -> keySerializationSchema.serialize(keyDataExtractor.apply(row)); } } ``` ########## seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java: ########## @@ -188,23 +207,32 @@ private void restoreState(List<KafkaSinkState> states) { } } - private Function<SeaTunnelRow, String> createPartitionExtractor(Config pluginConfig, - SeaTunnelRowType seaTunnelRowType) { - if (!pluginConfig.hasPath(PARTITION_KEY)){ + private Function<SeaTunnelRow, SeaTunnelRow> createPartitionExtractor(SeaTunnelRowType seaTunnelRowType) { + if (CollectionUtils.isEmpty(this.partitionKeyFields)) { return row -> null; } - String partitionKey = pluginConfig.getString(PARTITION_KEY); - List<String> fieldNames = Arrays.asList(seaTunnelRowType.getFieldNames()); - if (!fieldNames.contains(partitionKey)) { - return row -> partitionKey; - } - int partitionFieldIndex = seaTunnelRowType.indexOf(partitionKey); return row -> { - Object partitionFieldValue = row.getField(partitionFieldIndex); - if (partitionFieldValue != null) { - return partitionFieldValue.toString(); + SeaTunnelRow keySeaTunnelRow = new SeaTunnelRow(this.partitionKeyFields.size()); + int index = 0; + for (int i = 0; i < seaTunnelRowType.getFieldNames().length; i++) { + String fieldName = seaTunnelRowType.getFieldNames()[i]; + if (this.partitionKeyFields.contains(fieldName)) { + int partitionFieldIndex = seaTunnelRowType.indexOf(fieldName); + Object partitionFieldValue = row.getField(partitionFieldIndex); + keySeaTunnelRow.setField(index, partitionFieldValue); + ++index; + } } - return null; + return keySeaTunnelRow; }; } + + private List<String> createPartitionKeys(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { + if (pluginConfig.hasPath(PARTITION_KEY_FIELDS)) { + return pluginConfig.getStringList(PARTITION_KEY_FIELDS).stream() + .filter(f -> Arrays.asList(seaTunnelRowType.getFieldNames()).contains(f)) + .collect(Collectors.toList()); + } + return null; + } Review Comment: suggestion ```java private List<String> getPartitionKeyFields(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { if (pluginConfig.hasPath(PARTITION_KEY_FIELDS)) { List<String> partitionKeyFields = pluginConfig.getStringList(PARTITION_KEY_FIELDS); List<String> rowTypeFieldNames = Arrays.asList(seaTunnelRowType.getFieldNames()); for (String partitionKeyField : partitionKeyFields) { if (!rowTypeFieldNames.contains(partitionKeyField)) { throw new IllegalArgumentException(String.format( "Partition key field not found: %s, rowType: %s", partitionKeyField, rowTypeFieldNames)); } } return partitionKeyFields; } return Collections.emptyList(); } ``` ########## seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java: ########## @@ -162,12 +168,25 @@ private Properties getKafkaProperties(Config pluginConfig) { // todo: parse the target field from config private SeaTunnelRowSerializer<byte[], byte[]> getSerializer(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { Review Comment: suggestion ```java private SeaTunnelRowSerializer<byte[], byte[]> getSerializer(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { if (pluginConfig.hasPath(PARTITION)){ return new DefaultSeaTunnelRowSerializer(pluginConfig.getString(TOPIC), pluginConfig.getInt(PARTITION), seaTunnelRowType); } else { return new DefaultSeaTunnelRowSerializer(pluginConfig.getString(TOPIC), getPartitionKeyFields(pluginConfig, seaTunnelRowType), seaTunnelRowType); } } ``` ########## seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/SeaTunnelRowSerializer.java: ########## @@ -34,9 +34,9 @@ /** * Use Key serialize the {@link SeaTunnelRow} to a Kafka {@link ProducerRecord}. * - * @param key String + * @param key seatunnel row * @param row seatunnel row * @return kafka record. */ - ProducerRecord<K, V> serializeRowByKey(String key, SeaTunnelRow row); + ProducerRecord<K, V> serializeRowByKey(SeaTunnelRow key, SeaTunnelRow row); Review Comment: suggestion remove this method ```suggestion ``` ########## seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java: ########## @@ -67,13 +72,12 @@ public class KafkaSinkWriter implements SinkWriter<SeaTunnelRow, KafkaCommitInfo // check config @Override public void write(SeaTunnelRow element) { Review Comment: suggestion ```java public void write(SeaTunnelRow element) { ProducerRecord<byte[], byte[]> producerRecord = seaTunnelRowSerializer.serializeRow(element); kafkaProducerSender.send(producerRecord); } ``` -- 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]
