[
https://issues.apache.org/jira/browse/FLINK-5487?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15879029#comment-15879029
]
ASF GitHub Bot commented on FLINK-5487:
---------------------------------------
Github user rmetzger commented on a diff in the pull request:
https://github.com/apache/flink/pull/3358#discussion_r102544239
--- Diff:
flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java
---
@@ -92,40 +152,59 @@
/** Call bridge for different version-specfic */
private final ElasticsearchApiCallBridge callBridge;
+ /**
+ * Number of pending action requests not yet acknowledged by
Elasticsearch.
+ * This value is maintained only if {@link
ElasticsearchSinkBase#flushOnCheckpoint} is {@code true}.
+ *
+ * This is incremented whenever the user adds (or re-adds through the
{@link ActionRequestFailureHandler}) requests
+ * to the {@link RequestIndexer}. It is decremented for each completed
request of a bulk request, in
+ * {@link BulkProcessor.Listener#afterBulk(long, BulkRequest,
BulkResponse)} and
+ * {@link BulkProcessor.Listener#afterBulk(long, BulkRequest,
Throwable)}.
+ */
+ private AtomicLong numPendingRequests = new AtomicLong(0);
+
/** Elasticsearch client created using the call bridge. */
private transient Client client;
/** Bulk processor to buffer and send requests to Elasticsearch,
created using the client. */
private transient BulkProcessor bulkProcessor;
/**
- * This is set from inside the {@link BulkProcessor.Listener} if a
{@link Throwable} was thrown in callbacks.
+ * This is set from inside the {@link BulkProcessor.Listener} if a
{@link Throwable} was thrown in callbacks and
+ * the user considered it should fail the sink via the
+ * {@link ActionRequestFailureHandler#onFailure(ActionRequest,
Throwable, int, RequestIndexer)} method.
+ *
+ * Errors will be checked and rethrown before processing each input
element, and when the sink is closed.
*/
private final AtomicReference<Throwable> failureThrowable = new
AtomicReference<>();
public ElasticsearchSinkBase(
ElasticsearchApiCallBridge callBridge,
Map<String, String> userConfig,
- ElasticsearchSinkFunction<T> elasticsearchSinkFunction) {
+ ElasticsearchSinkFunction<T> elasticsearchSinkFunction,
+ ActionRequestFailureHandler failureHandler) {
this.callBridge = checkNotNull(callBridge);
this.elasticsearchSinkFunction =
checkNotNull(elasticsearchSinkFunction);
+ this.failureHandler = checkNotNull(failureHandler);
- // we eagerly check if the user-provided sink function is
serializable;
- // otherwise, if it isn't serializable, users will merely get a
non-informative error message
+ // we eagerly check if the user-provided sink function and
failure handler is serializable;
+ // otherwise, if they aren't serializable, users will merely
get a non-informative error message
// "ElasticsearchSinkBase is not serializable"
- try {
-
InstantiationUtil.serializeObject(elasticsearchSinkFunction);
- } catch (Exception e) {
- throw new IllegalArgumentException(
- "The implementation of the provided
ElasticsearchSinkFunction is not serializable. " +
- "The object probably contains or references non
serializable fields.");
- }
- checkNotNull(userConfig);
+
checkArgument(InstantiationUtil.isSerializable(elasticsearchSinkFunction),
+ "The implementation of the provided
ElasticsearchSinkFunction is not serializable. " +
+ "The object probably contains or references
non-serializable fields.");
+
+ checkArgument(InstantiationUtil.isSerializable(failureHandler),
+ "The implementation of the provided
ActionRequestFailureHandler is not serializable. " +
+ "The object probably contains or references
non-serializable fields.");
--- End diff --
That's so much nicer now :)
> Proper at-least-once support for ElasticsearchSink
> --------------------------------------------------
>
> Key: FLINK-5487
> URL: https://issues.apache.org/jira/browse/FLINK-5487
> Project: Flink
> Issue Type: Bug
> Components: Streaming Connectors
> Reporter: Tzu-Li (Gordon) Tai
> Assignee: Tzu-Li (Gordon) Tai
> Priority: Critical
>
> Discussion in ML:
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Fault-tolerance-guarantees-of-Elasticsearch-sink-in-flink-elasticsearch2-td10982.html
> Currently, the Elasticsearch Sink actually doesn't offer any guarantees for
> message delivery.
> For proper support of at-least-once, the sink will need to participate in
> Flink's checkpointing: when snapshotting is triggered at the
> {{ElasticsearchSink}}, we need to synchronize on the pending ES requests by
> flushing the internal bulk processor. For temporary ES failures (see
> FLINK-5122) that may happen on the flush, we should retry them before
> returning from snapshotting and acking the checkpoint. If there are
> non-temporary ES failures on the flush, the current snapshot should fail.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)