fapaul commented on a change in pull request #17363:
URL: https://github.com/apache/flink/pull/17363#discussion_r718296315



##########
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.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

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


Reply via email to