lvyanquan commented on code in PR #3495:
URL: https://github.com/apache/flink-cdc/pull/3495#discussion_r1691213209


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/pom.xml:
##########
@@ -0,0 +1,211 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0";
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.flink</groupId>
+        <artifactId>flink-cdc-pipeline-connectors</artifactId>
+        <version>3.2-SNAPSHOT</version>

Review Comment:
   please use `<version>${revision}</version>` like other connectors.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/serializer/ElasticsearchEventSerializer.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.elasticsearch.serializer;
+
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.cdc.common.data.RecordData;
+import org.apache.flink.cdc.common.event.*;
+import org.apache.flink.cdc.common.schema.Column;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.types.*;
+import org.apache.flink.cdc.common.utils.Preconditions;
+import org.apache.flink.cdc.common.utils.SchemaUtils;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+
+import co.elastic.clients.elasticsearch.core.bulk.BulkOperationVariant;
+import co.elastic.clients.elasticsearch.core.bulk.DeleteOperation;
+import co.elastic.clients.elasticsearch.core.bulk.IndexOperation;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** A serializer for Event to BulkOperationVariant. */
+public class ElasticsearchEventSerializer implements ElementConverter<Event, 
BulkOperationVariant> {
+    private final ObjectMapper objectMapper = new ObjectMapper();
+    private final Map<TableId, Schema> schemaMaps = new HashMap<>();
+
+    /** Format DATE type data. */
+    public static final DateTimeFormatter DATE_FORMATTER =
+            DateTimeFormatter.ofPattern("yyyy-MM-dd");
+
+    /** Format timestamp-related type data. */
+    public static final DateTimeFormatter DATE_TIME_FORMATTER =
+            DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSS");
+
+    /** ZoneId from pipeline config to support timestamp with local time zone. 
*/
+    private final ZoneId pipelineZoneId;
+
+    public ElasticsearchEventSerializer(ZoneId zoneId) {
+        this.pipelineZoneId = zoneId;
+    }
+
+    @Override
+    public BulkOperationVariant apply(Event event, SinkWriter.Context context) 
{
+        try {
+            if (event instanceof DataChangeEvent) {
+                return applyDataChangeEvent((DataChangeEvent) event);
+            } else if (event instanceof SchemaChangeEvent) {
+                IndexOperation<Map<String, Object>> indexOperation =
+                        applySchemaChangeEvent((SchemaChangeEvent) event);
+                if (indexOperation != null) {
+                    return indexOperation;
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException("Failed to serialize event", e);
+        }
+        return null;
+    }
+
+    private IndexOperation<Map<String, Object>> applySchemaChangeEvent(
+            SchemaChangeEvent schemaChangeEvent) throws IOException {
+        TableId tableId = schemaChangeEvent.tableId();
+        if (schemaChangeEvent instanceof CreateTableEvent) {
+            Schema schema = ((CreateTableEvent) schemaChangeEvent).getSchema();
+            schemaMaps.put(tableId, schema);
+            return createSchemaIndexOperation(tableId, schema);
+        } else if (schemaChangeEvent instanceof AddColumnEvent
+                || schemaChangeEvent instanceof DropColumnEvent) {
+            if (!schemaMaps.containsKey(tableId)) {
+                throw new RuntimeException("Schema of " + tableId + " does not 
exist.");
+            }
+            Schema updatedSchema =
+                    
SchemaUtils.applySchemaChangeEvent(schemaMaps.get(tableId), schemaChangeEvent);
+            schemaMaps.put(tableId, updatedSchema);
+            return createSchemaIndexOperation(tableId, updatedSchema);
+        } else {
+            if (!schemaMaps.containsKey(tableId)) {
+                throw new RuntimeException("Schema of " + tableId + " does not 
exist.");
+            }
+            Schema updatedSchema =
+                    
SchemaUtils.applySchemaChangeEvent(schemaMaps.get(tableId), schemaChangeEvent);
+            schemaMaps.put(tableId, updatedSchema);

Review Comment:
   If we modify column name in upstream, do we need to call 
createSchemaIndexOperation method



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/serializer/RecordData.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.elasticsearch.serializer;
+
+import java.math.BigDecimal;
+import java.time.*;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * A class representing a record with multiple fields of various types. 
Provides methods to access
+ * fields by position and type.
+ */
+public class RecordData {

Review Comment:
   I don't see anywhere to use this class, is it necessary?



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkFactory.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.elasticsearch.sink;
+
+import org.apache.flink.cdc.common.configuration.ConfigOption;
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.factories.DataSinkFactory;
+import org.apache.flink.cdc.common.factories.FactoryHelper;
+import org.apache.flink.cdc.common.pipeline.PipelineOptions;
+import org.apache.flink.cdc.common.sink.DataSink;
+import 
org.apache.flink.cdc.connectors.elasticsearch.config.ElasticsearchSinkOptions;
+import org.apache.flink.cdc.connectors.elasticsearch.v2.NetworkConfig;
+
+import org.apache.http.HttpHost;
+
+import java.time.ZoneId;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.cdc.connectors.elasticsearch.sink.ElasticsearchDataSinkOptions.*;
+
+/**
+ * Factory for creating {@link ElasticsearchDataSink}.
+ */
+public class ElasticsearchDataSinkFactory implements DataSinkFactory {
+
+    public static final String IDENTIFIER = "elasticsearch";
+
+    @Override
+    public DataSink createDataSink(Context context) {
+        FactoryHelper.createFactoryHelper(this, context).validate();
+
+        Configuration configuration =
+                
Configuration.fromMap(context.getFactoryConfiguration().toMap());
+        ZoneId zoneId = determineZoneId(context);
+
+        ElasticsearchSinkOptions sinkOptions = 
buildSinkConnectorOptions(configuration);
+        return new ElasticsearchDataSink(sinkOptions, zoneId);
+    }
+
+    private ZoneId determineZoneId(Context context) {
+        String configuredZone = 
context.getPipelineConfiguration().get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE);
+        String defaultZone = 
PipelineOptions.PIPELINE_LOCAL_TIME_ZONE.defaultValue();
+
+        return Objects.equals(configuredZone, defaultZone)
+                ? ZoneId.systemDefault()
+                : ZoneId.of(configuredZone);
+    }
+
+    private ElasticsearchSinkOptions buildSinkConnectorOptions(Configuration 
cdcConfig) {
+        List<HttpHost> hosts = parseHosts(cdcConfig.get(HOSTS));
+        NetworkConfig networkConfig = new NetworkConfig(hosts, null, null, 
null, null, null);

Review Comment:
   It seems that `username` and `password` were not passed.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkITCaseTest.java:
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.elasticsearch.sink;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.*;
+import org.apache.flink.cdc.common.schema.PhysicalColumn;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.sink.FlinkSinkProvider;
+import org.apache.flink.cdc.common.types.DataTypes;
+import 
org.apache.flink.cdc.connectors.elasticsearch.config.ElasticsearchSinkOptions;
+import org.apache.flink.cdc.connectors.elasticsearch.v2.NetworkConfig;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+import co.elastic.clients.elasticsearch.ElasticsearchClient;
+import co.elastic.clients.elasticsearch.core.GetRequest;
+import co.elastic.clients.elasticsearch.core.GetResponse;
+import co.elastic.clients.json.jackson.JacksonJsonpMapper;
+import co.elastic.clients.transport.rest_client.RestClientTransport;
+import org.apache.http.HttpHost;
+import org.elasticsearch.client.RestClient;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.time.Duration;
+import java.time.ZoneId;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase tests for {@link ElasticsearchDataSink}. */
+@Testcontainers
+public class ElasticsearchDataSinkITCaseTest {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ElasticsearchDataSinkITCaseTest.class);
+    private static final String ELASTICSEARCH_VERSION = "8.12.0";
+    private static final DockerImageName ELASTICSEARCH_IMAGE =
+            DockerImageName.parse(
+                            "docker.elastic.co/elasticsearch/elasticsearch:"
+                                    + ELASTICSEARCH_VERSION)
+                    
.asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch");
+
+    @Container
+    private static final ElasticsearchContainer ELASTICSEARCH_CONTAINER =
+            createElasticsearchContainer();
+
+    private ElasticsearchClient client;
+
+    @BeforeEach
+    public void setUp() {
+        client = createElasticsearchClient();
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (client != null) {
+            client.shutdown();
+        }
+    }
+
+    @Test
+    public void testElasticsearchSink() throws Exception {
+        TableId tableId = TableId.tableId("default", "schema", "table");
+        List<Event> events = createTestEvents(tableId);
+
+        runJobWithEvents(events);
+
+        verifyInsertedData(tableId, "2", 2, 2.0, "value2");

Review Comment:
   Please add a test for processing all supported data types.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/pom.xml:
##########
@@ -0,0 +1,211 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0";
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.flink</groupId>
+        <artifactId>flink-cdc-pipeline-connectors</artifactId>
+        <version>3.2-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>flink-cdc-pipeline-connector-elasticsearch</artifactId>
+    <packaging>jar</packaging>
+
+    <name>flink-cdc-pipeline-connector-elasticsearch</name>
+    <url>http://maven.apache.org</url>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <elasticsearch.version>8.12.1</elasticsearch.version>
+        <flink.version>1.18.0</flink.version>
+        <scala.binary.version>2.12</scala.binary.version>
+        <jackson.version>2.13.2</jackson.version>
+        
<surefire.module.config>--add-opens=java.base/java.util=ALL-UNNAMED</surefire.module.config>
+        <testcontainers.version>1.16.0</testcontainers.version>
+    </properties>
+
+    <dependencies>
+        <!-- Flink Dependencies -->
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-core</artifactId>
+            <version>${flink.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java</artifactId>
+            <version>${flink.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-api-java-bridge</artifactId>
+            <version>${flink.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-base</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-json</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+            <version>4.5.13</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-test-utils</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- Jackson Databind -->
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>${jackson.version}</version>
+        </dependency>
+
+        <!-- Testcontainers -->
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>junit-jupiter</artifactId>
+            <version>${testcontainers.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>elasticsearch</artifactId>
+            <version>${testcontainers.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- Elasticsearch Clients -->
+        <dependency>
+            <groupId>co.elastic.clients</groupId>
+            <artifactId>elasticsearch-java</artifactId>
+            <version>${elasticsearch.version}</version>
+        </dependency>
+
+        <!-- SLF4J API -->
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>1.7.32</version>
+        </dependency>
+
+        <!-- Testing -->
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-api</artifactId>
+            <version>5.7.1</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-engine</artifactId>
+            <version>5.7.1</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.assertj</groupId>
+            <artifactId>assertj-core</artifactId>
+            <version>3.18.1</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.13.2</version>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- Architecture Tests -->
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-architecture-tests-test</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-architecture-tests-production</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.platform</groupId>
+            <artifactId>junit-platform-launcher</artifactId>
+            <version>1.10.2</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-cdc-composer</artifactId>
+            <version>3.2-SNAPSHOT</version>

Review Comment:
   use `<version>${project.version}</version>`



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/resources/log4j2-test.properties:
##########
@@ -0,0 +1,25 @@
+################################################################################
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#      http://www.apache.org/licenses/LICENSE-2.0
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+
+# Set root logger level to OFF to not flood build logs
+# set manually to INFO for debugging purposes
+rootLogger.level = INFO
+rootLogger.appenderRef.test.ref = TestLogger
+
+appender.testlogger.name = TestLogger
+appender.testlogger.type = CONSOLE
+appender.testlogger.target = SYSTEM_ERR
+appender.testlogger.layout.type = PatternLayout
+appender.testlogger.layout.pattern = %-4r [%t] %-5p %c - %m%n

Review Comment:
   please add a `org.apache.flink.cdc.common.factories.Factory` file for SPI 
like other connectors.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/pom.xml:
##########
@@ -0,0 +1,211 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0";
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.flink</groupId>
+        <artifactId>flink-cdc-pipeline-connectors</artifactId>
+        <version>3.2-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>flink-cdc-pipeline-connector-elasticsearch</artifactId>
+    <packaging>jar</packaging>
+
+    <name>flink-cdc-pipeline-connector-elasticsearch</name>
+    <url>http://maven.apache.org</url>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <elasticsearch.version>8.12.1</elasticsearch.version>
+        <flink.version>1.18.0</flink.version>
+        <scala.binary.version>2.12</scala.binary.version>
+        <jackson.version>2.13.2</jackson.version>
+        
<surefire.module.config>--add-opens=java.base/java.util=ALL-UNNAMED</surefire.module.config>
+        <testcontainers.version>1.16.0</testcontainers.version>
+    </properties>

Review Comment:
   Some properties were already provided, this can be simplified by:
   ```
       <properties>
           <elasticsearch.version>8.12.1</elasticsearch.version>
       </properties>
   ```



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkITCaseTest.java:
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.elasticsearch.sink;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.*;
+import org.apache.flink.cdc.common.schema.PhysicalColumn;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.sink.FlinkSinkProvider;
+import org.apache.flink.cdc.common.types.DataTypes;
+import 
org.apache.flink.cdc.connectors.elasticsearch.config.ElasticsearchSinkOptions;
+import org.apache.flink.cdc.connectors.elasticsearch.v2.NetworkConfig;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+import co.elastic.clients.elasticsearch.ElasticsearchClient;
+import co.elastic.clients.elasticsearch.core.GetRequest;
+import co.elastic.clients.elasticsearch.core.GetResponse;
+import co.elastic.clients.json.jackson.JacksonJsonpMapper;
+import co.elastic.clients.transport.rest_client.RestClientTransport;
+import org.apache.http.HttpHost;
+import org.elasticsearch.client.RestClient;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.time.Duration;
+import java.time.ZoneId;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase tests for {@link ElasticsearchDataSink}. */
+@Testcontainers
+public class ElasticsearchDataSinkITCaseTest {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ElasticsearchDataSinkITCaseTest.class);
+    private static final String ELASTICSEARCH_VERSION = "8.12.0";
+    private static final DockerImageName ELASTICSEARCH_IMAGE =
+            DockerImageName.parse(
+                            "docker.elastic.co/elasticsearch/elasticsearch:"
+                                    + ELASTICSEARCH_VERSION)
+                    
.asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch");
+
+    @Container
+    private static final ElasticsearchContainer ELASTICSEARCH_CONTAINER =
+            createElasticsearchContainer();
+
+    private ElasticsearchClient client;
+
+    @BeforeEach
+    public void setUp() {
+        client = createElasticsearchClient();
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (client != null) {
+            client.shutdown();
+        }
+    }
+
+    @Test
+    public void testElasticsearchSink() throws Exception {
+        TableId tableId = TableId.tableId("default", "schema", "table");
+        List<Event> events = createTestEvents(tableId);
+
+        runJobWithEvents(events);
+
+        verifyInsertedData(tableId, "2", 2, 2.0, "value2");
+    }
+
+    @Test
+    public void testElasticsearchInsertAndDelete() throws Exception {
+        TableId tableId = TableId.tableId("default", "schema", "table");
+        List<Event> events = createTestEventsWithDelete(tableId);
+
+        runJobWithEvents(events);
+
+        verifyDeletedData(tableId, "2");
+    }
+
+    @Test
+    public void testElasticsearchAddColumn() throws Exception {
+        TableId tableId = TableId.tableId("default", "schema", "table");
+        List<Event> events = createTestEventsWithAddColumn(tableId);
+
+        runJobWithEvents(events);
+
+        verifyInsertedDataWithNewColumn(tableId, "3", 3, 3.0, "value3", true);
+    }
+
+    private static ElasticsearchContainer createElasticsearchContainer() {
+        return new ElasticsearchContainer(ELASTICSEARCH_IMAGE)
+                .withEnv("discovery.type", "single-node")
+                .withEnv("xpack.security.enabled", "false")
+                .withEnv("ES_JAVA_OPTS", "-Xms2g -Xmx2g")
+                .withEnv("logger.org.elasticsearch", "ERROR")
+                .withLogConsumer(new Slf4jLogConsumer(LOG))
+                
.waitingFor(Wait.forListeningPort().withStartupTimeout(Duration.ofMinutes(5)));
+    }
+
+    private ElasticsearchClient createElasticsearchClient() {
+        RestClientTransport transport =
+                new RestClientTransport(
+                        RestClient.builder(
+                                        new HttpHost(
+                                                
ELASTICSEARCH_CONTAINER.getHost(),
+                                                
ELASTICSEARCH_CONTAINER.getFirstMappedPort(),
+                                                "http"))
+                                .build(),
+                        new JacksonJsonpMapper());
+        return new ElasticsearchClient(transport);
+    }
+
+    private void runJobWithEvents(List<Event> events) throws Exception {
+        ElasticsearchSinkOptions options = createSinkOptions();
+        StreamExecutionEnvironment env = createStreamExecutionEnvironment();
+        ElasticsearchDataSink<Event> sink =
+                new ElasticsearchDataSink<>(options, ZoneId.systemDefault());
+
+        DataStream<Event> stream = env.fromCollection(events, 
TypeInformation.of(Event.class));
+        Sink<Event> elasticsearchSink = ((FlinkSinkProvider) 
sink.getEventSinkProvider()).getSink();
+        stream.sinkTo(elasticsearchSink);
+
+        env.execute("Elasticsearch Sink Test");
+    }
+
+    private ElasticsearchSinkOptions createSinkOptions() {
+        NetworkConfig networkConfig =
+                new NetworkConfig(
+                        Collections.singletonList(
+                                new HttpHost(
+                                        ELASTICSEARCH_CONTAINER.getHost(),
+                                        
ELASTICSEARCH_CONTAINER.getFirstMappedPort())),
+                        null,
+                        null,
+                        null,
+                        null,
+                        null);
+
+        return new ElasticsearchSinkOptions(
+                5, 1, 10, 50 * 1024 * 1024, 1000, 10 * 1024 * 1024, 
networkConfig);
+    }
+
+    private StreamExecutionEnvironment createStreamExecutionEnvironment() {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(3000);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        return env;
+    }
+
+    private void verifyInsertedData(
+            TableId tableId, String id, int expectedId, double expectedNumber, 
String expectedName)

Review Comment:
   We can pass `List<String>` for column name and `List<Object>` for expected 
object to simplify verification.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/config/ElasticsearchSinkOptions.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.elasticsearch.config;
+
+import org.apache.flink.cdc.connectors.elasticsearch.v2.NetworkConfig;
+
+import org.apache.http.HttpHost;
+
+import java.io.Serializable;
+import java.util.List;
+
+/** DorisDataSink Options reference {@link ElasticsearchSinkOptions}. */

Review Comment:
   ElasticsearchDataSink



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to