fapaul commented on a change in pull request #17363: URL: https://github.com/apache/flink/pull/17363#discussion_r718199624
########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java ########## @@ -0,0 +1,276 @@ +/* + * 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; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.http.HttpHost; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Builder to construct {@link ElasticsearchSink}. + * + * <p>The following example shows the minimum setup to create a ElasticsearchSink that submits + * actions on checkpoint. + * + * <pre>{@code + * Elasticsearch<String> sink = Elasticsearch + * .builder() + * .setHosts(MY_ELASTICSEARCH_HOSTS) + * .setProcessor(MY_ELASTICSEARCH_PROCESSOR) + * .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + * .build(); + * }</pre> + * + * @param <IN> type of the records converted to Elasticsearch actions + */ +@PublicEvolving +public class ElasticsearchSinkBuilder<IN> { + + private int bulkFlushMaxActions = -1; + private int bulkFlushMaxMb = -1; + private long bulkFlushInterval = -1; + private ElasticsearchSinkBase.FlushBackoffType bulkFlushBackoffType; + private int bulkFlushBackoffRetries = -1; + private long bulkFlushBackOffDelay = -1; + private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE; + private List<HttpHost> hosts; + private ElasticsearchProcessor<? extends IN> processor; + private String username; + private String password; + private String connectionPathPrefix; + + ElasticsearchSinkBuilder() {} + + /** + * Sets the hosts where the Elasticsearch cluster nodes are reachable. + * + * @param hosts http addresses describing the node locations + */ + public ElasticsearchSinkBuilder<IN> setHosts(HttpHost... hosts) { + checkNotNull(hosts); + checkState(hosts.length > 0, "Hosts cannot be empty."); + this.hosts = Arrays.asList(hosts); + return this; + } + + /** + * Sets the processor which is invoked on every record to convert it to Elasticsearch actions. + * + * @param processor to process records into Elasticsearch actions. + * @return {@link ElasticsearchSinkBuilder} + */ + public <T extends IN> ElasticsearchSinkBuilder<T> setProcessor( + ElasticsearchProcessor<? extends T> processor) { + checkNotNull(processor); + checkState( + InstantiationUtil.isSerializable(processor), + "The elasticsearch processor must be serializable."); + final ElasticsearchSinkBuilder<T> self = self(); + self.processor = processor; + return self; + } + + /** + * Sets the wanted the {@link DeliveryGuarantee}. The default delivery guarantee is {@link + * #deliveryGuarantee}. Review comment: Good that we have done the same thing in the Kafka docs :) ########## File path: flink-connectors/flink-connector-elasticsearch7/pom.xml ########## @@ -92,6 +92,12 @@ under the License. <scope>test</scope> </dependency> + <dependency> Review comment: Isn't it already part of the root pom by specifying the bom https://github.com/apache/flink/blob/4fe9f525a92319acc1e3434bebed601306f7a16f/pom.xml#L780 ? ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java ########## @@ -0,0 +1,78 @@ +/* + * 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.streaming.connectors.elasticsearch.ElasticsearchSinkBase; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +class BulkProcessorConfig implements Serializable { Review comment: Yes, do we need `@Internal` for package-private classes? ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java ########## @@ -0,0 +1,78 @@ +/* + * 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.streaming.connectors.elasticsearch.ElasticsearchSinkBase; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +class BulkProcessorConfig implements Serializable { + + private final int bulkFlushMaxActions; + private final int bulkFlushMaxMb; + private final long bulkFlushInterval; + @Nullable private final ElasticsearchSinkBase.FlushBackoffType flushBackoffType; Review comment: What exactly do you mean here? ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchProcessor.java ########## @@ -0,0 +1,79 @@ +/* + * 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; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.connector.sink.SinkWriter; + +import org.elasticsearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * Creates multiple {@link ActionRequest ActionRequests} from the incoming elements. + * + * <p>This is used by sinks to prepare elements for sending them to Elasticsearch. + * + * <p>Example: + * + * <pre>{@code + * private static class TestElasticsearchProcessor implements ElasticsearchProcessor<Tuple2<Integer, String>> { + * + * public IndexRequest createIndexRequest(Tuple2<Integer, String> element) { + * Map<String, Object> json = new HashMap<>(); + * json.put("data", element.f1); + * + * return Requests.indexRequest() + * .index("my-index") + * .type("my-type") + * .id(element.f0.toString()) + * .source(json); + * } + * + * public void process(Tuple2<Integer, String> element, RequestIndexer indexer) { + * indexer.add(createIndexRequest(element)); + * } + * } + * + * }</pre> + * + * @param <T> The type of the element handled by this {@link ElasticsearchProcessor} + */ +@PublicEvolving +public interface ElasticsearchProcessor<T> extends Function, Serializable { Review comment: In the old sink, it was called `ElasticsearchFunction` which I found even worse. We can go with emitter. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchProcessor.java ########## @@ -0,0 +1,79 @@ +/* + * 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; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.connector.sink.SinkWriter; + +import org.elasticsearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * Creates multiple {@link ActionRequest ActionRequests} from the incoming elements. + * + * <p>This is used by sinks to prepare elements for sending them to Elasticsearch. + * + * <p>Example: + * + * <pre>{@code + * private static class TestElasticsearchProcessor implements ElasticsearchProcessor<Tuple2<Integer, String>> { + * + * public IndexRequest createIndexRequest(Tuple2<Integer, String> element) { + * Map<String, Object> json = new HashMap<>(); + * json.put("data", element.f1); + * + * return Requests.indexRequest() + * .index("my-index") + * .type("my-type") + * .id(element.f0.toString()) + * .source(json); + * } + * + * public void process(Tuple2<Integer, String> element, RequestIndexer indexer) { + * indexer.add(createIndexRequest(element)); + * } + * } + * + * }</pre> + * + * @param <T> The type of the element handled by this {@link ElasticsearchProcessor} + */ +@PublicEvolving +public interface ElasticsearchProcessor<T> extends Function, Serializable { + + /** + * Initialization method for the function. It is called once before the actual working process + * methods. + */ + default void open() throws Exception {} + + /** Tear-down method for the function. It is called when the sink closes. */ + default void close() throws Exception {} + + /** + * Process the incoming element to produce multiple {@link ActionRequest ActionRequests}. The + * produced requests should be added to the provided {@link RequestIndexer}. + * + * @param element incoming element to process + * @param context to access additional information about the record + * @param indexer request indexer that {@code ActionRequest} should be added to + */ + void process(T element, SinkWriter.Context context, RequestIndexer indexer); Review comment: Looking at Elasticsearch's APIs in the past I think it is a good idea to not make any ES API part of Flink's public API to easier support multiple versions. I also feel that `Requests` or `BatchRequests` are not really correct here because it implies that you always add to a single request. The class was called `RequestIndexer` before so I have no hard feeling about finding a good replacement. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java ########## @@ -0,0 +1,276 @@ +/* + * 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; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.http.HttpHost; + +import java.util.Arrays; +import java.util.List; + +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 minimum setup to create a ElasticsearchSink that submits + * actions on checkpoint. Review comment: If you do not specify anything for the bulk processor it will not emit anything except on a checkpoint. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchProcessor.java ########## @@ -0,0 +1,79 @@ +/* + * 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; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.connector.sink.SinkWriter; + +import org.elasticsearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * Creates multiple {@link ActionRequest ActionRequests} from the incoming elements. + * + * <p>This is used by sinks to prepare elements for sending them to Elasticsearch. + * + * <p>Example: + * + * <pre>{@code + * private static class TestElasticsearchProcessor implements ElasticsearchProcessor<Tuple2<Integer, String>> { + * + * public IndexRequest createIndexRequest(Tuple2<Integer, String> element) { + * Map<String, Object> json = new HashMap<>(); Review comment: AFAIK you always need to pass in some kind of json to the request. Either you do it via a map or you can implicitly use a string and specify a key. This way the request builds the json internally `{your_key: your_string}`. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java ########## @@ -0,0 +1,124 @@ +/* + * 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; +import org.apache.flink.api.connector.sink.Committer; +import org.apache.flink.api.connector.sink.GlobalCommitter; +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import org.apache.http.HttpHost; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Flink Sink to produce or update data in an Elasticsearch index. The sink supports the following + * delivery guarantees. + * + * <ul> + * <li>{@link DeliveryGuarantee#NONE} does not provide any guarantees: actions are flushed to + * Elasticsearch only depending on the configurations of the bulk processor. In case of a + * failure it might happen that actions are lost if the bulk processor still has buffered + * actions. + * <li>{@link DeliveryGuarantee#AT_LEAST_ONCE} on a checkpoint the sink will wait until all + * buffered actions are flushed to and acknowledged by Elasticsearch. No actions will be lost + * but actions might be sent to Elasticsearch multiple times when Flink restarts. + * </ul> + * + * @param <IN> type of the records converted to Elasticsearch actions + * @see ElasticsearchSinkBuilder on how to construct a ElasticsearchSink + */ +@PublicEvolving +public class ElasticsearchSink<IN> implements Sink<IN, Void, Void, Void> { + + private final List<HttpHost> hosts; + private final ElasticsearchProcessor<IN> processor; + private final BulkProcessorConfig buildBulkProcessorConfig; + private final NetworkClientConfig networkClientConfig; + private final DeliveryGuarantee deliveryGuarantee; + + ElasticsearchSink( + List<HttpHost> hosts, + ElasticsearchProcessor<IN> processor, + DeliveryGuarantee deliveryGuarantee, + BulkProcessorConfig buildBulkProcessorConfig, + NetworkClientConfig networkClientConfig) { + this.hosts = checkNotNull(hosts); + checkState(!hosts.isEmpty(), "Hosts cannot be empty."); Review comment: I am still surprised when to use which precondition ... :) ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java ########## @@ -0,0 +1,276 @@ +/* + * 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; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.http.HttpHost; + +import java.util.Arrays; +import java.util.List; + +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 minimum setup to create a ElasticsearchSink that submits + * actions on checkpoint. + * + * <pre>{@code + * Elasticsearch<String> sink = Elasticsearch + * .builder() + * .setHosts(MY_ELASTICSEARCH_HOSTS) + * .setProcessor(MY_ELASTICSEARCH_PROCESSOR) + * .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + * .build(); + * }</pre> + * + * @param <IN> type of the records converted to Elasticsearch actions + */ +@PublicEvolving +public class ElasticsearchSinkBuilder<IN> { + + private int bulkFlushMaxActions = -1; + private int bulkFlushMaxMb = -1; + private long bulkFlushInterval = -1; + private ElasticsearchSinkBase.FlushBackoffType bulkFlushBackoffType; + private int bulkFlushBackoffRetries = -1; + private long bulkFlushBackOffDelay = -1; + private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE; + private List<HttpHost> hosts; + private ElasticsearchProcessor<? extends IN> processor; + private String username; + private String password; + private String connectionPathPrefix; + + ElasticsearchSinkBuilder() {} + + /** + * Sets the hosts where the Elasticsearch cluster nodes are reachable. + * + * @param hosts http addresses describing the node locations + */ + public ElasticsearchSinkBuilder<IN> setHosts(HttpHost... hosts) { + checkNotNull(hosts); + checkState(hosts.length > 0, "Hosts cannot be empty."); + this.hosts = Arrays.asList(hosts); + return this; + } + + /** + * Sets the processor which is invoked on every record to convert it to Elasticsearch actions. + * + * @param processor to process records into Elasticsearch actions. + * @return {@link ElasticsearchSinkBuilder} + */ + public <T extends IN> ElasticsearchSinkBuilder<T> setProcessor( + ElasticsearchProcessor<? extends T> processor) { Review comment: > In a way the processor is like the `SerializationSchema`. How do they relate? Should we add some Processor builder similar to `KafkaSerializationSchema`? You are right, I was also wondering about the relation between the usually used `SerializationSchema`s and this elasticsearch sink way. So far I do not see a comfortable way to offer a nice build. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java ########## @@ -0,0 +1,276 @@ +/* + * 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; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.http.HttpHost; + +import java.util.Arrays; +import java.util.List; + +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 minimum setup to create a ElasticsearchSink that submits + * actions on checkpoint. + * + * <pre>{@code + * Elasticsearch<String> sink = Elasticsearch + * .builder() + * .setHosts(MY_ELASTICSEARCH_HOSTS) + * .setProcessor(MY_ELASTICSEARCH_PROCESSOR) + * .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + * .build(); + * }</pre> + * + * @param <IN> type of the records converted to Elasticsearch actions + */ +@PublicEvolving +public class ElasticsearchSinkBuilder<IN> { + + private int bulkFlushMaxActions = -1; + private int bulkFlushMaxMb = -1; + private long bulkFlushInterval = -1; + private ElasticsearchSinkBase.FlushBackoffType bulkFlushBackoffType; + private int bulkFlushBackoffRetries = -1; + private long bulkFlushBackOffDelay = -1; + private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE; + private List<HttpHost> hosts; + private ElasticsearchProcessor<? extends IN> processor; + private String username; + private String password; + private String connectionPathPrefix; + + ElasticsearchSinkBuilder() {} + + /** + * Sets the hosts where the Elasticsearch cluster nodes are reachable. + * + * @param hosts http addresses describing the node locations + */ + public ElasticsearchSinkBuilder<IN> setHosts(HttpHost... hosts) { + checkNotNull(hosts); + checkState(hosts.length > 0, "Hosts cannot be empty."); + this.hosts = Arrays.asList(hosts); + return this; + } + + /** + * Sets the processor which is invoked on every record to convert it to Elasticsearch actions. + * + * @param processor to process records into Elasticsearch actions. + * @return {@link ElasticsearchSinkBuilder} + */ + public <T extends IN> ElasticsearchSinkBuilder<T> setProcessor( + ElasticsearchProcessor<? extends T> processor) { + checkNotNull(processor); + checkState( + InstantiationUtil.isSerializable(processor), + "The elasticsearch processor must be serializable."); + final ElasticsearchSinkBuilder<T> self = self(); + self.processor = processor; + return self; + } + + /** + * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link + * #deliveryGuarantee}. + * + * @param deliveryGuarantee which describes the record emission behaviour + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder<IN> setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) { + checkState( + deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE, + "Elasticsearch sink does not support the EXACTLY_ONCE guarantee."); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + return this; + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to + * disable it. + * + * @param numMaxActions the maximum number of actions to buffer per bulk request. + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder<IN> setBulkFlushMaxActions(int numMaxActions) { + checkState( + numMaxActions == -1 || numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + this.bulkFlushMaxActions = numMaxActions; + return this; + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. You can pass -1 to + * disable it. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder<IN> setBulkFlushMaxSizeMb(int maxSizeMb) { + checkState( + maxSizeMb == -1 || maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + this.bulkFlushMaxMb = maxSizeMb; + return this; + } + + /** + * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder<IN> setBulkFlushInterval(long intervalMillis) { + checkState( + intervalMillis == -1 || intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than " + + "or equal to 0."); + this.bulkFlushInterval = intervalMillis; + return this; + } + + /** + * Sets the type of back of to use when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder<IN> setBulkFlushBackoffType( + ElasticsearchSinkBase.FlushBackoffType flushBackoffType) { + this.bulkFlushBackoffType = checkNotNull(flushBackoffType); + return this; + } + + /** + * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk + * requests + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder<IN> setBulkFlushBackoffRetries(int maxRetries) { + checkState(maxRetries > 0, "Max number of backoff attempts must be larger than 0."); + this.bulkFlushBackoffRetries = maxRetries; + return this; + } + + /** + * Sets the amount of delay between each backoff attempt when flushing bulk requests, in + * milliseconds. + * + * @param delayMillis the amount of delay between each backoff attempt when flushing bulk + * requests, in milliseconds. + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder<IN> setBulkFlushBackoffDelay(long delayMillis) { + checkState( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger " + + "than or equal to 0."); + this.bulkFlushBackOffDelay = delayMillis; + return this; + } Review comment: Definitely 👍 ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java ########## @@ -0,0 +1,326 @@ +/* + * 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.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.rest.RestStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class ElasticsearchWriter<IN> implements SinkWriter<IN, Void, Void> { Review comment: Can do but the class is already package-private, so is it necessary? ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java ########## @@ -0,0 +1,326 @@ +/* + * 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.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.rest.RestStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class ElasticsearchWriter<IN> implements SinkWriter<IN, Void, Void> { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriter.class); + + private final ElasticsearchProcessor<IN> processor; + private final MailboxExecutor mailboxExecutor; + private final boolean flushOnCheckpoint; + private final BulkProcessor bulkProcessor; + private final RestHighLevelClient client; + private final RequestIndexer requestIndexer; + private final Counter numBytesOutCounter; + + private long pendingActions = 0; + private boolean checkpointInProgress = false; + private volatile long lastSendTime = 0; + private volatile long receiveTime = Long.MAX_VALUE; + private volatile boolean closed = false; + + /** + * Constructor creating an elasticsearch writer. + * + * <p>It will throw a {@link RuntimeException} if {@link ElasticsearchProcessor#open()} fails. + * + * @param hosts the reachable elasticsearch cluster nodes + * @param processor converting incoming records to elasticsearch actions + * @param flushOnCheckpoint if true all until now received records are flushed after every + * checkpoint + * @param bulkProcessorConfig describing the flushing and failure handling of the used {@link + * BulkProcessor} + * @param networkClientConfig describing properties of the network connection used to connect to + * the elasticsearch cluster + * @param metricGroup for the sink writer + * @param mailboxExecutor Flink's mailbox executor + */ + public ElasticsearchWriter( + List<HttpHost> hosts, + ElasticsearchProcessor<IN> processor, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + NetworkClientConfig networkClientConfig, + SinkWriterMetricGroup metricGroup, + MailboxExecutor mailboxExecutor) { + this.processor = checkNotNull(processor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.mailboxExecutor = checkNotNull(mailboxExecutor); + this.client = + new RestHighLevelClient( + configureRestClientBuilder( + RestClient.builder(hosts.toArray(new HttpHost[0])), + networkClientConfig)); + this.bulkProcessor = + configureBulkProcessor( + BulkProcessor.builder( + (bulkRequest, bulkResponseActionListener) -> + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener), + new BulkListener()), + bulkProcessorConfig); + this.requestIndexer = new DefaultRequestIndexer(); + checkNotNull(metricGroup); + metricGroup.setCurrentSendTimeGauge(() -> receiveTime - lastSendTime); + this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); + try { + processor.open(); + } catch (Exception e) { + throw new RuntimeException("Failed to open the ElasticsearchProcessor", e); + } + } + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // do not allow new bulk writes until all actions are flushed + while (checkpointInProgress) { + mailboxExecutor.yield(); + } + processor.process(element, context, requestIndexer); + } + + @Override + public List<Void> prepareCommit(boolean flush) throws IOException, InterruptedException { + checkpointInProgress = true; + while (pendingActions != 0) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + checkpointInProgress = false; + return Collections.emptyList(); + } + + @Override + public void close() throws Exception { + closed = true; + processor.close(); + client.close(); + } + + private static RestClientBuilder configureRestClientBuilder( + RestClientBuilder builder, NetworkClientConfig networkClientConfig) { + if (networkClientConfig.getConnectionPathPrefix() != null) { + builder.setPathPrefix(networkClientConfig.getConnectionPathPrefix()); + } + if (networkClientConfig.getPassword() != null + && networkClientConfig.getUsername() != null) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, + new UsernamePasswordCredentials( + networkClientConfig.getUsername(), networkClientConfig.getPassword())); + builder.setHttpClientConfigCallback( + httpClientBuilder -> + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); + } + return builder; + } + + private static BulkProcessor configureBulkProcessor( + BulkProcessor.Builder builder, BulkProcessorConfig bulkProcessorConfig) { + // This makes flush() blocking + builder.setConcurrentRequests(0); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue(bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval(new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + if (bulkProcessorConfig.getFlushBackoffType() != null) { + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + default: + backoffPolicy = BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + builder.setBackoffPolicy(backoffPolicy); + Review comment: WDYT about making the `FlushBackoffType` similar to `DeliveryGuarantee` and configuration enum accessible to all connectors? I can imagine there a lot of connectors wanting to implement a similar thing. This would mean we cannot put elastic-specific code into it. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java ########## @@ -0,0 +1,326 @@ +/* + * 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.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.rest.RestStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class ElasticsearchWriter<IN> implements SinkWriter<IN, Void, Void> { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriter.class); + + private final ElasticsearchProcessor<IN> processor; + private final MailboxExecutor mailboxExecutor; + private final boolean flushOnCheckpoint; + private final BulkProcessor bulkProcessor; + private final RestHighLevelClient client; + private final RequestIndexer requestIndexer; + private final Counter numBytesOutCounter; + + private long pendingActions = 0; + private boolean checkpointInProgress = false; + private volatile long lastSendTime = 0; + private volatile long receiveTime = Long.MAX_VALUE; + private volatile boolean closed = false; + + /** + * Constructor creating an elasticsearch writer. + * + * <p>It will throw a {@link RuntimeException} if {@link ElasticsearchProcessor#open()} fails. + * + * @param hosts the reachable elasticsearch cluster nodes + * @param processor converting incoming records to elasticsearch actions + * @param flushOnCheckpoint if true all until now received records are flushed after every + * checkpoint + * @param bulkProcessorConfig describing the flushing and failure handling of the used {@link + * BulkProcessor} + * @param networkClientConfig describing properties of the network connection used to connect to + * the elasticsearch cluster + * @param metricGroup for the sink writer + * @param mailboxExecutor Flink's mailbox executor + */ + public ElasticsearchWriter( + List<HttpHost> hosts, + ElasticsearchProcessor<IN> processor, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + NetworkClientConfig networkClientConfig, + SinkWriterMetricGroup metricGroup, + MailboxExecutor mailboxExecutor) { + this.processor = checkNotNull(processor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.mailboxExecutor = checkNotNull(mailboxExecutor); + this.client = + new RestHighLevelClient( + configureRestClientBuilder( + RestClient.builder(hosts.toArray(new HttpHost[0])), + networkClientConfig)); + this.bulkProcessor = + configureBulkProcessor( + BulkProcessor.builder( + (bulkRequest, bulkResponseActionListener) -> + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener), + new BulkListener()), + bulkProcessorConfig); + this.requestIndexer = new DefaultRequestIndexer(); + checkNotNull(metricGroup); + metricGroup.setCurrentSendTimeGauge(() -> receiveTime - lastSendTime); + this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); + try { + processor.open(); + } catch (Exception e) { + throw new RuntimeException("Failed to open the ElasticsearchProcessor", e); + } + } + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // do not allow new bulk writes until all actions are flushed + while (checkpointInProgress) { + mailboxExecutor.yield(); + } + processor.process(element, context, requestIndexer); + } + + @Override + public List<Void> prepareCommit(boolean flush) throws IOException, InterruptedException { + checkpointInProgress = true; + while (pendingActions != 0) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + checkpointInProgress = false; + return Collections.emptyList(); + } + + @Override + public void close() throws Exception { + closed = true; + processor.close(); + client.close(); + } + + private static RestClientBuilder configureRestClientBuilder( + RestClientBuilder builder, NetworkClientConfig networkClientConfig) { + if (networkClientConfig.getConnectionPathPrefix() != null) { + builder.setPathPrefix(networkClientConfig.getConnectionPathPrefix()); + } + if (networkClientConfig.getPassword() != null + && networkClientConfig.getUsername() != null) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, + new UsernamePasswordCredentials( + networkClientConfig.getUsername(), networkClientConfig.getPassword())); + builder.setHttpClientConfigCallback( + httpClientBuilder -> + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); + } + return builder; + } + + private static BulkProcessor configureBulkProcessor( + BulkProcessor.Builder builder, BulkProcessorConfig bulkProcessorConfig) { + // This makes flush() blocking + builder.setConcurrentRequests(0); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue(bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval(new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + if (bulkProcessorConfig.getFlushBackoffType() != null) { + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + default: + backoffPolicy = BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + builder.setBackoffPolicy(backoffPolicy); + Review comment: WDYT about making the `FlushBackoffType` similar to `DeliveryGuarantee` an configuration enum accessible to all connectors? I can imagine there a lot of connectors wanting to implement a similar thing. This would mean we cannot put elastic-specific code into it. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java ########## @@ -0,0 +1,326 @@ +/* + * 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.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.rest.RestStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class ElasticsearchWriter<IN> implements SinkWriter<IN, Void, Void> { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriter.class); + + private final ElasticsearchProcessor<IN> processor; + private final MailboxExecutor mailboxExecutor; + private final boolean flushOnCheckpoint; + private final BulkProcessor bulkProcessor; + private final RestHighLevelClient client; + private final RequestIndexer requestIndexer; + private final Counter numBytesOutCounter; + + private long pendingActions = 0; + private boolean checkpointInProgress = false; + private volatile long lastSendTime = 0; + private volatile long receiveTime = Long.MAX_VALUE; + private volatile boolean closed = false; + + /** + * Constructor creating an elasticsearch writer. + * + * <p>It will throw a {@link RuntimeException} if {@link ElasticsearchProcessor#open()} fails. + * + * @param hosts the reachable elasticsearch cluster nodes + * @param processor converting incoming records to elasticsearch actions + * @param flushOnCheckpoint if true all until now received records are flushed after every + * checkpoint + * @param bulkProcessorConfig describing the flushing and failure handling of the used {@link + * BulkProcessor} + * @param networkClientConfig describing properties of the network connection used to connect to + * the elasticsearch cluster + * @param metricGroup for the sink writer + * @param mailboxExecutor Flink's mailbox executor + */ + public ElasticsearchWriter( + List<HttpHost> hosts, + ElasticsearchProcessor<IN> processor, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + NetworkClientConfig networkClientConfig, + SinkWriterMetricGroup metricGroup, + MailboxExecutor mailboxExecutor) { + this.processor = checkNotNull(processor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.mailboxExecutor = checkNotNull(mailboxExecutor); + this.client = + new RestHighLevelClient( + configureRestClientBuilder( + RestClient.builder(hosts.toArray(new HttpHost[0])), + networkClientConfig)); + this.bulkProcessor = + configureBulkProcessor( + BulkProcessor.builder( + (bulkRequest, bulkResponseActionListener) -> + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener), + new BulkListener()), + bulkProcessorConfig); + this.requestIndexer = new DefaultRequestIndexer(); + checkNotNull(metricGroup); + metricGroup.setCurrentSendTimeGauge(() -> receiveTime - lastSendTime); + this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); + try { + processor.open(); + } catch (Exception e) { + throw new RuntimeException("Failed to open the ElasticsearchProcessor", e); + } + } + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // do not allow new bulk writes until all actions are flushed + while (checkpointInProgress) { + mailboxExecutor.yield(); + } + processor.process(element, context, requestIndexer); + } + + @Override + public List<Void> prepareCommit(boolean flush) throws IOException, InterruptedException { + checkpointInProgress = true; + while (pendingActions != 0) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + checkpointInProgress = false; + return Collections.emptyList(); + } + + @Override + public void close() throws Exception { + closed = true; + processor.close(); + client.close(); + } + + private static RestClientBuilder configureRestClientBuilder( + RestClientBuilder builder, NetworkClientConfig networkClientConfig) { + if (networkClientConfig.getConnectionPathPrefix() != null) { + builder.setPathPrefix(networkClientConfig.getConnectionPathPrefix()); + } + if (networkClientConfig.getPassword() != null + && networkClientConfig.getUsername() != null) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, + new UsernamePasswordCredentials( + networkClientConfig.getUsername(), networkClientConfig.getPassword())); + builder.setHttpClientConfigCallback( + httpClientBuilder -> + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); + } + return builder; + } + + private static BulkProcessor configureBulkProcessor( + BulkProcessor.Builder builder, BulkProcessorConfig bulkProcessorConfig) { + // This makes flush() blocking + builder.setConcurrentRequests(0); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue(bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval(new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + if (bulkProcessorConfig.getFlushBackoffType() != null) { + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + default: + backoffPolicy = BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + builder.setBackoffPolicy(backoffPolicy); + Review comment: WDYT about making the `FlushBackoffType` similar to `DeliveryGuarantee` a configuration enum accessible to all connectors? I can imagine there a lot of connectors wanting to implement a similar thing. This would mean we cannot put elastic-specific code into it. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java ########## @@ -0,0 +1,326 @@ +/* + * 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.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.rest.RestStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class ElasticsearchWriter<IN> implements SinkWriter<IN, Void, Void> { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriter.class); + + private final ElasticsearchProcessor<IN> processor; + private final MailboxExecutor mailboxExecutor; + private final boolean flushOnCheckpoint; + private final BulkProcessor bulkProcessor; + private final RestHighLevelClient client; + private final RequestIndexer requestIndexer; + private final Counter numBytesOutCounter; + + private long pendingActions = 0; + private boolean checkpointInProgress = false; + private volatile long lastSendTime = 0; + private volatile long receiveTime = Long.MAX_VALUE; + private volatile boolean closed = false; + + /** + * Constructor creating an elasticsearch writer. + * + * <p>It will throw a {@link RuntimeException} if {@link ElasticsearchProcessor#open()} fails. + * + * @param hosts the reachable elasticsearch cluster nodes + * @param processor converting incoming records to elasticsearch actions + * @param flushOnCheckpoint if true all until now received records are flushed after every + * checkpoint + * @param bulkProcessorConfig describing the flushing and failure handling of the used {@link + * BulkProcessor} + * @param networkClientConfig describing properties of the network connection used to connect to + * the elasticsearch cluster + * @param metricGroup for the sink writer + * @param mailboxExecutor Flink's mailbox executor + */ + public ElasticsearchWriter( + List<HttpHost> hosts, + ElasticsearchProcessor<IN> processor, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + NetworkClientConfig networkClientConfig, + SinkWriterMetricGroup metricGroup, + MailboxExecutor mailboxExecutor) { + this.processor = checkNotNull(processor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.mailboxExecutor = checkNotNull(mailboxExecutor); + this.client = + new RestHighLevelClient( + configureRestClientBuilder( + RestClient.builder(hosts.toArray(new HttpHost[0])), + networkClientConfig)); + this.bulkProcessor = + configureBulkProcessor( + BulkProcessor.builder( + (bulkRequest, bulkResponseActionListener) -> + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener), + new BulkListener()), + bulkProcessorConfig); + this.requestIndexer = new DefaultRequestIndexer(); + checkNotNull(metricGroup); + metricGroup.setCurrentSendTimeGauge(() -> receiveTime - lastSendTime); + this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); + try { + processor.open(); + } catch (Exception e) { + throw new RuntimeException("Failed to open the ElasticsearchProcessor", e); + } + } + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // do not allow new bulk writes until all actions are flushed + while (checkpointInProgress) { + mailboxExecutor.yield(); + } + processor.process(element, context, requestIndexer); + } + + @Override + public List<Void> prepareCommit(boolean flush) throws IOException, InterruptedException { + checkpointInProgress = true; + while (pendingActions != 0) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + checkpointInProgress = false; + return Collections.emptyList(); + } + + @Override + public void close() throws Exception { + closed = true; + processor.close(); + client.close(); + } + + private static RestClientBuilder configureRestClientBuilder( + RestClientBuilder builder, NetworkClientConfig networkClientConfig) { + if (networkClientConfig.getConnectionPathPrefix() != null) { + builder.setPathPrefix(networkClientConfig.getConnectionPathPrefix()); + } + if (networkClientConfig.getPassword() != null + && networkClientConfig.getUsername() != null) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, + new UsernamePasswordCredentials( + networkClientConfig.getUsername(), networkClientConfig.getPassword())); + builder.setHttpClientConfigCallback( + httpClientBuilder -> + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); + } + return builder; + } + + private static BulkProcessor configureBulkProcessor( + BulkProcessor.Builder builder, BulkProcessorConfig bulkProcessorConfig) { + // This makes flush() blocking + builder.setConcurrentRequests(0); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue(bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval(new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + if (bulkProcessorConfig.getFlushBackoffType() != null) { + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + default: + backoffPolicy = BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + builder.setBackoffPolicy(backoffPolicy); + + return builder.build(); + } + + private class BulkListener implements BulkProcessor.Listener { + + @Override + public void beforeBulk(long executionId, BulkRequest request) { + LOG.info("Sending bulk of {} actions to Elasticsearch.", request.numberOfActions()); + lastSendTime = System.currentTimeMillis(); + numBytesOutCounter.inc(request.estimatedSizeInBytes()); + } + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + receiveTime = System.currentTimeMillis(); + // If the writer is cancelled before the last bulk response (i.e. no flush on checkpoint + // configured or shutdown without a final + // checkpoint) the mailbox might already be shutdown, so we should not enqueue any + // actions. + if (isClosed()) { + return; + } + mailboxExecutor.execute( + () -> { + if (!response.hasFailures() && flushOnCheckpoint) { + pendingActions -= request.numberOfActions(); + return; + } + + for (int i = 0; i < response.getItems().length; i++) { + final BulkItemResponse itemResponse = response.getItems()[i]; + if (!itemResponse.isFailed()) { + continue; + } + final Throwable failure = itemResponse.getFailure().getCause(); + if (failure == null) { + continue; + } + final RestStatus restStatus = itemResponse.getFailure().getStatus(); + final DocWriteRequest<?> actionRequest = request.requests().get(i); + + if (restStatus == null) { + throw new FlinkRuntimeException( Review comment: Good idea 👍 ########## File path: flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderTest.java ########## @@ -0,0 +1,120 @@ +/* + * 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.connector.base.DeliveryGuarantee; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import org.apache.http.HttpHost; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.List; +import java.util.function.Consumer; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +/** Tests for {@link ElasticsearchSinkBuilder}. */ +class ElasticsearchSinkBuilderTest extends TestLogger { + + @ParameterizedTest + @MethodSource("validBuilders") + void testBuildElasticsearchSink(ElasticsearchSinkBuilder<?> builder) { + builder.build(); + } + + @Test + void testThrowIfExactlyOnceConfigured() { + assertThrows( + IllegalStateException.class, + () -> + new ElasticsearchSinkBuilder<>() + .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE)); + } + + @Test + void testThrowIfHostsNotSet() { + assertThrows( + NullPointerException.class, + () -> + new ElasticsearchSinkBuilder<>() + .setProcessor((element, indexer, context) -> {}) + .build()); + } + + @Test + void testThrowIfProcessorNotSet() { + assertThrows( + NullPointerException.class, + () -> + new ElasticsearchSinkBuilder<>() + .setHosts(new HttpHost("localhost:3000")) + .build()); + } + + @ParameterizedTest + @MethodSource("backoffConfigurations") + void testThrowIfBackoffNotEnabledButConfigured( + Consumer<ElasticsearchSinkBuilder<?>> configure) { + final ElasticsearchSinkBuilder<?> builder = + new ElasticsearchSinkBuilder<>() + .setHosts(new HttpHost("localhost:3000")) + .setProcessor((element, indexer, context) -> {}); + configure.accept(builder); + assertThrows(IllegalStateException.class, builder::build); + } + + @ParameterizedTest + @MethodSource("backoffConfigurations") Review comment: Some of the tests became unnecessary and only one simple method source is still used. I would say it is not worth the extra effort to remove it. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkITCase.java ########## @@ -0,0 +1,229 @@ +/* + * 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.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.utility.DockerImageName; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.function.BiFunction; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Tests for {@link ElasticsearchSink}. */ +@Testcontainers +class ElasticsearchSinkITCase extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkITCase.class); + private static final String ELASTICSEARCH_PASSWORD = "test-password"; + private static final String ELASTICSEARCH_USER = "elastic"; + + @Container + private static final ElasticsearchContainer ES_CONTAINER = + new ElasticsearchContainer( + DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_COMMERCIAL_7)) + .withPassword(ELASTICSEARCH_PASSWORD) Review comment: Do not ask me why bu the `ElasticsearchContainer` does only support setting the password and the user cannot be configured ... ########## File path: flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java ########## @@ -27,6 +27,9 @@ public static final String ELASTICSEARCH_7 = "docker.elastic.co/elasticsearch/elasticsearch-oss:7.5.1"; + public static final String ELASTICSEARCH_COMMERCIAL_7 = Review comment: Only the commercial image allows using the password authentication. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestProcessor.java ########## @@ -0,0 +1,75 @@ +/* + * 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.api.connector.sink.SinkWriter; +import org.apache.flink.api.java.tuple.Tuple2; + +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +class TestProcessor implements ElasticsearchProcessor<Tuple2<Integer, String>> { + + private final String index; + private final XContentBuilderProvider xContentBuilderProvider; + private final String dataFieldName; + + public static TestProcessor jsonProcessor(String index, String dataFieldName) { + return new TestProcessor(index, dataFieldName, XContentFactory::jsonBuilder); + } + + public static TestProcessor smileProcessor(String index, String dataFieldName) { + return new TestProcessor(index, dataFieldName, XContentFactory::smileBuilder); + } + Review comment: Unfortunately, I do not see an easy way to provide a builder because users can produce multiple types of actions `Delete`, `Index` or `Update`, and the parameters are not compatible across elastic versions. Any idea? ########## File path: flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java ########## @@ -0,0 +1,266 @@ +/* + * 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.api.common.operators.MailboxExecutor; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.apache.http.HttpHost; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +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.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.Optional; + +import static org.apache.flink.connector.elasticsearch.sink.TestContext.buildMessage; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Tests for {@link ElasticsearchWriter}. */ +@Testcontainers +class ElasticsearchWriterITCase extends TestLogger { Review comment: Hmm, I like that the WriterITCase covers most of the actual flush behavior which seemed hard to test when Flink is calling the flushing methods and the overall sink it cases are to see how it interacts in common scenarios. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestContext.java ########## @@ -0,0 +1,73 @@ +/* + * 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.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; + +class TestContext { Review comment: It holds the client used for test thus I named it context. To me utils are not instantiable and only provide static methods. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java ########## @@ -0,0 +1,326 @@ +/* + * 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.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.rest.RestStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class ElasticsearchWriter<IN> implements SinkWriter<IN, Void, Void> { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriter.class); + + private final ElasticsearchProcessor<IN> processor; + private final MailboxExecutor mailboxExecutor; + private final boolean flushOnCheckpoint; + private final BulkProcessor bulkProcessor; + private final RestHighLevelClient client; + private final RequestIndexer requestIndexer; + private final Counter numBytesOutCounter; + + private long pendingActions = 0; + private boolean checkpointInProgress = false; + private volatile long lastSendTime = 0; + private volatile long receiveTime = Long.MAX_VALUE; + private volatile boolean closed = false; + + /** + * Constructor creating an elasticsearch writer. + * + * <p>It will throw a {@link RuntimeException} if {@link ElasticsearchProcessor#open()} fails. + * + * @param hosts the reachable elasticsearch cluster nodes + * @param processor converting incoming records to elasticsearch actions + * @param flushOnCheckpoint if true all until now received records are flushed after every + * checkpoint + * @param bulkProcessorConfig describing the flushing and failure handling of the used {@link + * BulkProcessor} + * @param networkClientConfig describing properties of the network connection used to connect to + * the elasticsearch cluster + * @param metricGroup for the sink writer + * @param mailboxExecutor Flink's mailbox executor + */ + public ElasticsearchWriter( + List<HttpHost> hosts, + ElasticsearchProcessor<IN> processor, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + NetworkClientConfig networkClientConfig, + SinkWriterMetricGroup metricGroup, + MailboxExecutor mailboxExecutor) { + this.processor = checkNotNull(processor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.mailboxExecutor = checkNotNull(mailboxExecutor); + this.client = + new RestHighLevelClient( + configureRestClientBuilder( + RestClient.builder(hosts.toArray(new HttpHost[0])), + networkClientConfig)); + this.bulkProcessor = + configureBulkProcessor( + BulkProcessor.builder( + (bulkRequest, bulkResponseActionListener) -> + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener), + new BulkListener()), + bulkProcessorConfig); + this.requestIndexer = new DefaultRequestIndexer(); + checkNotNull(metricGroup); + metricGroup.setCurrentSendTimeGauge(() -> receiveTime - lastSendTime); + this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); + try { + processor.open(); + } catch (Exception e) { + throw new RuntimeException("Failed to open the ElasticsearchProcessor", e); + } + } + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // do not allow new bulk writes until all actions are flushed + while (checkpointInProgress) { + mailboxExecutor.yield(); + } + processor.process(element, context, requestIndexer); + } + + @Override + public List<Void> prepareCommit(boolean flush) throws IOException, InterruptedException { + checkpointInProgress = true; + while (pendingActions != 0) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + checkpointInProgress = false; + return Collections.emptyList(); + } + + @Override + public void close() throws Exception { + closed = true; + processor.close(); + client.close(); + } + + private static RestClientBuilder configureRestClientBuilder( + RestClientBuilder builder, NetworkClientConfig networkClientConfig) { + if (networkClientConfig.getConnectionPathPrefix() != null) { + builder.setPathPrefix(networkClientConfig.getConnectionPathPrefix()); + } + if (networkClientConfig.getPassword() != null + && networkClientConfig.getUsername() != null) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, + new UsernamePasswordCredentials( + networkClientConfig.getUsername(), networkClientConfig.getPassword())); + builder.setHttpClientConfigCallback( + httpClientBuilder -> + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); + } Review comment: We have to find a balance here all additional UDFs are making the usage in Table API more difficult. The previous Table API connector only supported Basic auth and no one ever complained. In general, I understand your idea but we can easily extend the authentication feature set if needed. -- 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