You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by GitBox <gi...@apache.org> on 2022/05/30 02:30:46 UTC

[GitHub] [incubator-inlong] EMsnap opened a new pull request, #4425: [INLONG-4250][Sort] add elasticsearch load node

EMsnap opened a new pull request, #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425

   ### Title Name: [INLONG-4250][Sort] add elasticsearch load node
   
   Fixes #4250
   
   ### Motivation
   
   add elasticsearch load node
   
   ### Modifications
   
   add elasticsearch load node
   
   ### Verifying this change
   
   *(Please pick either of the following options)*
   
   - [ ] This change is a trivial rework/code cleanup without any test coverage.
   
   - [ ] This change is already covered by existing tests, such as:
     *(please describe tests)*
   
   - [ ] This change added tests and can be verified as follows:
   
     *(example:)*
     - *Added integration tests for end-to-end deployment with large payloads (10MB)*
     - *Extended integration test for recovery after broker failure*
   
   ### Documentation
   
     - Does this pull request introduce a new feature? (yes / no)
     - If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)
     - If a feature is not applicable for documentation, explain why?
     - If a feature is not documented yet in this PR, please create a follow-up issue for adding the documentation
   


-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] thexiay commented on a diff in pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
thexiay commented on code in PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425#discussion_r884516718


##########
inlong-sort/sort-connectors/elasticsearch/pom.xml:
##########
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~  Licensed to the Apache Software Foundation (ASF) under one
+  ~  or more contributor license agreements.  See the NOTICE file
+  ~  distributed with this work for additional information
+  ~  regarding copyright ownership.  The ASF licenses this file
+  ~  to you under the Apache License, Version 2.0 (the
+  ~  "License"); you may not use this file except in compliance
+  ~  with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+
+<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">
+    <parent>
+        <artifactId>sort-connectors</artifactId>
+        <groupId>org.apache.inlong</groupId>
+        <version>1.2.0-incubating-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>sort-connector-elasticsearch</artifactId>
+    <name>Apache InLong - Sort-connector-elasticsearch</name>
+    <packaging>jar</packaging>
+
+    <properties>
+        <flink.connector.elasticsearch6.version>1.16-SNAPSHOT</flink.connector.elasticsearch6.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-elasticsearch6</artifactId>
+            <version>${flink.connector.elasticsearch6.version}</version>
+        </dependency>
+    </dependencies>
+
+

Review Comment:
   here should add a shade plugin.Otherwise elasticsearch client jar would not include.



-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] gong commented on pull request #4425: [INLONG-4250][Sort] add elasticsearch load node

Posted by GitBox <gi...@apache.org>.
gong commented on PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425#issuecomment-1140624343

   pr is duplulication with [4409](https://github.com/apache/incubator-inlong/pull/4409/commits)


-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] yunqingmoswu commented on a diff in pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
yunqingmoswu commented on code in PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425#discussion_r886254384


##########
inlong-sort/sort-core/pom.xml:
##########
@@ -112,6 +112,18 @@
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>sort-connector-elasticsearch</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>mysql</groupId>
+            <artifactId>mysql-connector-java</artifactId>

Review Comment:
   Please add it in  mysql-cdc's pom.xml  and remove the scope provided because of it has provided in inlong's pom.xml



-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] dockerzhang commented on a diff in pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
dockerzhang commented on code in PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425#discussion_r885257809


##########
inlong-sort/sort-connectors/elasticsearch/pom.xml:
##########
@@ -0,0 +1,144 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~  Licensed to the Apache Software Foundation (ASF) under one
+  ~  or more contributor license agreements.  See the NOTICE file
+  ~  distributed with this work for additional information
+  ~  regarding copyright ownership.  The ASF licenses this file
+  ~  to you under the Apache License, Version 2.0 (the
+  ~  "License"); you may not use this file except in compliance
+  ~  with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+
+<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">
+    <parent>
+        <artifactId>sort-connectors</artifactId>
+        <groupId>org.apache.inlong</groupId>
+        <version>1.2.0-incubating-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>sort-connector-elasticsearch</artifactId>
+    <name>Apache InLong - Sort-connector-elasticsearch</name>

Review Comment:
   Apache InLong - Sort Connectors - Elasticsearch



-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] healchow commented on a diff in pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425#discussion_r884383818


##########
inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/ElasticSearchLoadNode.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.inlong.sort.protocol.node.load;
+
+import com.google.common.base.Preconditions;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.NoArgsConstructor;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.enums.FilterStrategy;
+import org.apache.inlong.sort.protocol.node.LoadNode;
+import org.apache.inlong.sort.protocol.transformation.FieldRelationShip;
+import org.apache.inlong.sort.protocol.transformation.FilterFunction;
+
+/**
+ * elasticSearch load node
+ */
+@EqualsAndHashCode(callSuper = true)
+@JsonTypeName("ElasticSearchLoadNode")
+@Data
+@NoArgsConstructor
+public class ElasticSearchLoadNode extends LoadNode implements Serializable {
+
+    private static final long serialVersionUID = -1L;
+
+    @JsonProperty("index")
+    @Nonnull
+    private String index;
+
+    @JsonProperty("hosts")
+    @Nonnull
+    private String hosts;
+
+    @JsonProperty("userName")
+    @Nonnull
+    private String userName;

Review Comment:
   Can it be changed to `username`?



-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] healchow commented on a diff in pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425#discussion_r884383370


##########
inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/ElasticSearchLoadNode.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.inlong.sort.protocol.node.load;
+
+import com.google.common.base.Preconditions;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.NoArgsConstructor;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.enums.FilterStrategy;
+import org.apache.inlong.sort.protocol.node.LoadNode;
+import org.apache.inlong.sort.protocol.transformation.FieldRelationShip;
+import org.apache.inlong.sort.protocol.transformation.FilterFunction;
+
+/**
+ * elasticSearch load node
+ */
+@EqualsAndHashCode(callSuper = true)
+@JsonTypeName("ElasticSearchLoadNode")
+@Data
+@NoArgsConstructor
+public class ElasticSearchLoadNode extends LoadNode implements Serializable {

Review Comment:
   The official name is `Elasticsearch`, and it is recommended to be consistent with the official one.



-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] dockerzhang commented on a diff in pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
dockerzhang commented on code in PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425#discussion_r885260167


##########
inlong-sort/sort-connectors/elasticsearch/pom.xml:
##########
@@ -0,0 +1,144 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~  Licensed to the Apache Software Foundation (ASF) under one
+  ~  or more contributor license agreements.  See the NOTICE file
+  ~  distributed with this work for additional information
+  ~  regarding copyright ownership.  The ASF licenses this file
+  ~  to you under the Apache License, Version 2.0 (the
+  ~  "License"); you may not use this file except in compliance
+  ~  with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+
+<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">
+    <parent>
+        <artifactId>sort-connectors</artifactId>
+        <groupId>org.apache.inlong</groupId>
+        <version>1.2.0-incubating-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>sort-connector-elasticsearch</artifactId>
+    <name>Apache InLong - Sort-connector-elasticsearch</name>

Review Comment:
   ```suggestion
       <name>Apache InLong - Sort Connectors - Elasticsearch</name>
   ```



-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] e-mhui commented on a diff in pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
e-mhui commented on code in PR #4425:
URL: https://github.com/apache/inlong/pull/4425#discussion_r1016376106


##########
inlong-sort/sort-connectors/oracle-cdc/pom.xml:
##########
@@ -35,6 +35,10 @@
             <groupId>com.ververica</groupId>
             <artifactId>flink-connector-oracle-cdc</artifactId>
         </dependency>
+        <dependency>
+            <groupId>com.oracle.database.jdbc</groupId>
+            <artifactId>ojdbc8</artifactId>
+        </dependency>
     </dependencies>

Review Comment:
   why add this dependency?



-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] gong commented on a diff in pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
gong commented on code in PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425#discussion_r884572793


##########
inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/ElasticsearchLoadNode.java:
##########
@@ -39,10 +39,10 @@
  * elasticSearch load node
  */
 @EqualsAndHashCode(callSuper = true)
-@JsonTypeName("ElasticSearchLoadNode")
+@JsonTypeName("ElasticsearchLoadNode")

Review Comment:
   `ElasticsearchLoadNode` should be `elasticsearchLoadNode`



-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] woofyzhao commented on pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
woofyzhao commented on PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425#issuecomment-1140633792

   > pr is duplulication with [4409](https://github.com/apache/incubator-inlong/pull/4409/commits)
   
   4409 can be rebased on this pr. @EMsnap @gong @healchow 


-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] yunqingmoswu commented on a diff in pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
yunqingmoswu commented on code in PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425#discussion_r884863282


##########
inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/ElasticsearchLoadNode.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.inlong.sort.protocol.node.load;
+
+import com.google.common.base.Preconditions;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.NoArgsConstructor;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.enums.FilterStrategy;
+import org.apache.inlong.sort.protocol.node.LoadNode;
+import org.apache.inlong.sort.protocol.transformation.FieldRelation;
+import org.apache.inlong.sort.protocol.transformation.FilterFunction;
+
+/**
+ * elasticSearch load node
+ */
+@EqualsAndHashCode(callSuper = true)
+@JsonTypeName("elasticsearchLoadNode")
+@Data
+@NoArgsConstructor
+public class ElasticsearchLoadNode extends LoadNode implements Serializable {
+
+    private static final long serialVersionUID = -1L;
+
+    @JsonProperty("index")
+    @Nonnull
+    private String index;
+
+    @JsonProperty("hosts")
+    @Nonnull
+    private String hosts;
+
+    @JsonProperty("username")
+    @Nonnull
+    private String username;
+
+    @JsonProperty("password")
+    @Nonnull
+    private String password;
+
+    @JsonProperty("documentType")
+    private String documentType;
+
+    @JsonCreator
+    public ElasticsearchLoadNode(@JsonProperty("id") String id,
+        @JsonProperty("name") String name,
+        @JsonProperty("fields") List<FieldInfo> fields,
+        @JsonProperty("fieldRelationShips") List<FieldRelation> fieldRelationShips,
+        @JsonProperty("filters") List<FilterFunction> filters,
+        @JsonProperty("filterStrategy") FilterStrategy filterStrategy,
+        @Nullable @JsonProperty("sinkParallelism") Integer sinkParallelism,
+        @JsonProperty("properties") Map<String, String> properties,
+        @Nonnull @JsonProperty("index") String index,
+        @Nonnull @JsonProperty("hosts") String hosts,
+        @Nonnull @JsonProperty("username") String username,
+        @Nonnull @JsonProperty("password") String password,
+        @Nonnull @JsonProperty("documentType") String documentType) {
+        super(id, name, fields, fieldRelationShips, filters, filterStrategy, sinkParallelism, properties);
+        this.password = Preconditions.checkNotNull(password, "password is null");
+        this.username = Preconditions.checkNotNull(username, "username is null");
+        this.hosts = Preconditions.checkNotNull(hosts, "hosts is null");
+        this.index = Preconditions.checkNotNull(index, "index is null");
+        this.documentType = documentType;
+    }
+
+    @Override
+    public Map<String, String> tableOptions() {
+        Map<String, String> options = super.tableOptions();
+        options.put("connector", "elasticsearch-6");
+        options.put("hosts", hosts);
+        options.put("index", index);
+        options.put("password", password);
+        options.put("username", username);
+        if (documentType == null) {
+            documentType = "_doc";
+        }
+        options.put("document-type", documentType);
+        return options;
+    }
+
+    @Override
+    public String genTableName() {
+        return "node_" + super.getId() + "_" + index;
+    }
+
+    @Override
+    public String getPrimaryKey() {
+        return super.getPrimaryKey();

Review Comment:
   Make sure it is right?



##########
inlong-sort/sort-connectors/elasticsearch/pom.xml:
##########
@@ -0,0 +1,152 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~  Licensed to the Apache Software Foundation (ASF) under one
+  ~  or more contributor license agreements.  See the NOTICE file
+  ~  distributed with this work for additional information
+  ~  regarding copyright ownership.  The ASF licenses this file
+  ~  to you under the Apache License, Version 2.0 (the
+  ~  "License"); you may not use this file except in compliance
+  ~  with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+
+<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">
+    <parent>
+        <artifactId>sort-connectors</artifactId>
+        <groupId>org.apache.inlong</groupId>
+        <version>1.2.0-incubating-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>sort-connector-elasticsearch</artifactId>
+    <name>Apache InLong - Sort-connector-elasticsearch</name>
+    <packaging>jar</packaging>
+
+    <properties>
+        <flink.connector.elasticsearch6.version>1.16-SNAPSHOT</flink.connector.elasticsearch6.version>

Review Comment:
   Are there any stability issues with the latest version of Flink here? And is it better to be consistent with the Flink version?



##########
licenses/inlong-sort/LICENSE:
##########
@@ -567,6 +567,7 @@ The text of each license is the standard Apache 2.0 license.
   com.ververica:flink-connector-sqlserver-cdc:2.2.1 - flink-connector-sqlserver-cdc (https://github.com/ververica/flink-cdc-connectors/tree/release-2.2.1/flink-connector-sqlserver-cdc),  (The Apache Software License, Version 2.0)
   org.apache.flink:flink-core:1.13.5 - Flink : Core (https://github.com/apache/flink/tree/release-1.13.5/flink-core), (The Apache Software License, Version 2.0)
   org.apache.flink:flink-csv:1.13.5 - Flink : Formats : Csv (https://github.com/apache/flink/tree/release-1.13.5/flink-formats/flink-csv), (The Apache Software License, Version 2.0)
+  org.apache.flink:flink-connector-elasticsearch6_1.16 - Flink : Connectors : Elasticsearch 6 (https://github.com/apache/flink/blob/release-1.13.5/flink-connectors/flink-connector-elasticsearch6), (The Apache Software License, Version 2.0)

Review Comment:
   Are you sure the version of flink-connector-elasticsearch6_1.16 and release-1.13.5 are the same? 



-- 
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: commits-unsubscribe@inlong.apache.org

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


[GitHub] [incubator-inlong] EMsnap merged pull request #4425: [INLONG-4250][Sort] Add Elasticsearch load node

Posted by GitBox <gi...@apache.org>.
EMsnap merged PR #4425:
URL: https://github.com/apache/incubator-inlong/pull/4425


-- 
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: commits-unsubscribe@inlong.apache.org

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