alpreu commented on a change in pull request #17538:
URL: https://github.com/apache/flink/pull/17538#discussion_r741694795
##########
File path:
flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSink.java
##########
@@ -45,22 +48,44 @@
* logical description.
*/
@Internal
-final class Elasticsearch7DynamicSink implements DynamicTableSink {
+class ElasticsearchDynamicSink implements DynamicTableSink {
Review comment:
Yes :)
##########
File path:
flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorTest.java
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.streaming.connectors.elasticsearch.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.temporal.UnsupportedTemporalTypeException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/** Suite tests for {@link IndexGenerator}. */
+public class IndexGeneratorTest {
+
+ private List<RowData> rows;
+ private List<String> fieldNames;
+ private List<DataType> dataTypes;
+
+ @BeforeEach
+ public void prepareData() {
+ fieldNames =
+ Stream.of(
+ "id",
+ "item",
+ "log_ts",
+ "log_date",
+ "order_timestamp",
+ "log_time",
+ "local_datetime",
+ "local_date",
+ "local_time",
+ "note",
+ "status")
+ .collect(Collectors.toList());
Review comment:
👍
##########
File path:
flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorTest.java
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.streaming.connectors.elasticsearch.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.temporal.UnsupportedTemporalTypeException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/** Suite tests for {@link IndexGenerator}. */
+public class IndexGeneratorTest {
+
+ private List<RowData> rows;
+ private List<String> fieldNames;
+ private List<DataType> dataTypes;
+
+ @BeforeEach
+ public void prepareData() {
Review comment:
👍
##########
File path:
flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.connector.elasticsearch.sink;
+
+import org.apache.flink.annotation.Internal;
+
+import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.bulk.BulkRequest;
+import org.elasticsearch.action.bulk.BulkResponse;
+import org.elasticsearch.client.RestHighLevelClient;
+
+import java.io.Serializable;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * {@link BulkRequestConsumerFactory} is used to bridge incompatible
Elasticsearch Java API calls
+ * across different Elasticsearch versions.
+ */
+@Internal
+interface BulkRequestConsumerFactory
+ extends Function<
+ RestHighLevelClient, BiConsumer<BulkRequest,
ActionListener<BulkResponse>>>,
Review comment:
I'm not quite sure I understand, SerializableFunction also extends
Function
##########
File path:
flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java
##########
@@ -23,31 +24,21 @@
import org.apache.http.HttpHost;
+import java.io.Serializable;
import java.util.Arrays;
import java.util.List;
+import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkState;
/**
- * Builder to construct a {@link ElasticsearchSink}.
- *
- * <p>The following example shows the minimal setup to create a
ElasticsearchSink that submits
- * actions on checkpoint or the default number of actions was buffered (1000).
- *
- * <pre>{@code
- * Elasticsearch<String> sink = Elasticsearch
- * .builder()
- * .setHosts(MY_ELASTICSEARCH_HOSTS)
- * .setEmitter(MY_ELASTICSEARCH_EMITTER)
- * .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
- * .build();
- * }</pre>
+ * Base builder to construct a {@link ElasticsearchSink}.
*
* @param <IN> type of the records converted to Elasticsearch actions
*/
@PublicEvolving
-public class ElasticsearchSinkBuilder<IN> {
+public abstract class ElasticsearchSinkBuilderBase<IN> implements Serializable
{
Review comment:
Regarding the base builder I initially tried the type capture but could
not get `setEmitter` to work with in this setup. If you could help me out there
we can try to change it
The serializable was an accident, I'll remove it.
##########
File path:
flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java
##########
@@ -165,8 +161,8 @@ private void runTest(
@Nullable MapFunction<Long, Long> additionalMapper)
throws Exception {
final ElasticsearchSink<Tuple2<Integer, String>> sink =
- ElasticsearchSink.builder()
-
.setHosts(HttpHost.create(ES_CONTAINER.getHttpHostAddress()))
+ getSinkBuilder()
+
.setHosts(HttpHost.create(getElasticsearchHttpHostAddress()))
Review comment:
No because we still need it for creating the RestHighlevelClient
##########
File path:
flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.connector.elasticsearch.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * Builder to construct an Elasticsearch 6 compatible {@link
ElasticsearchSink}.
+ *
+ * <p>The following example shows the minimal setup to create a
ElasticsearchSink that submits
+ * actions on checkpoint or the default number of actions was buffered (1000).
+ *
+ * <pre>{@code
+ * ElasticsearchSink<String> sink = new Elasticsearch6SinkBuilder<String>()
Review comment:
There is no `Elasticsearch6Sink` anymore, just `ElasticsearchSink` in
the base module so I'm not sure how we can keep this pattern in this case
--
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]