wuchong commented on a change in pull request #12184:
URL: https://github.com/apache/flink/pull/12184#discussion_r426209541



##########
File path: 
flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.elasticsearch.table;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.description.Description;
+import 
org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase;
+
+import java.time.Duration;
+import java.util.List;
+
+import static org.apache.flink.configuration.description.TextElement.text;
+
+/**
+ * Options for {@link 
org.apache.flink.table.factories.DynamicTableSinkFactory} for Elasticsearch.
+ */
+public class ElasticsearchOptions {
+       /**
+        * Backoff strategy. Extends {@link 
ElasticsearchSinkBase.FlushBackoffType} with
+        * {@code DISABLED} option.
+        */
+       public enum BackOffType {
+               DISABLED,
+               CONSTANT,
+               EXPONENTIAL
+       }
+
+       public static final ConfigOption<List<String>> HOSTS_OPTION =
+               ConfigOptions.key("hosts")
+                       .stringType()
+                       .asList()
+                       .noDefaultValue()
+                       .withDescription("Elasticseatch hosts to connect to.");
+       public static final ConfigOption<String> INDEX_OPTION =
+               ConfigOptions.key("index")
+                       .stringType()
+                       .noDefaultValue()
+                       .withDescription("Elasticsearch index for every 
record.");
+       public static final ConfigOption<String> DOCUMENT_TYPE_OPTION =
+               ConfigOptions.key("document-type")
+                       .stringType()
+                       .noDefaultValue()
+                       .withDescription("Elasticsearch document type.");
+       public static final ConfigOption<String> KEY_DELIMITER_OPTION =
+               ConfigOptions.key("document-id.key-delimiter")
+                       .stringType()
+                       .defaultValue("_")
+                       .withDescription("Delimiter for composite keys e.g., 
\"$\" would result in IDs \"KEY1$KEY2$KEY3\".");
+       public static final ConfigOption<String> FAILURE_HANDLER_OPTION =
+               ConfigOptions.key("failure-handler")
+                       .stringType()
+                       .defaultValue("fail")
+                       .withDescription(Description.builder()
+                               .text("Failure handling strategy in case a 
request to Elasticsearch fails")
+                               .list(
+                                       text("\"fail\" (throws an exception if 
a request fails and thus causes a job failure),"),
+                                       text("\"ignore\" (ignores failures and 
drops the request),"),
+                                       text("\"retry_rejected\" (re-adds 
requests that have failed due to queue capacity saturation),"),
+                                       text("\"class name\" for failure 
handling with a ActionRequestFailureHandler subclass"))
+                               .build());
+       public static final ConfigOption<Boolean> FLUSH_ON_CHECKPOINT_OPTION =
+               ConfigOptions.key("sink.flush-on-checkpoint")
+                       .booleanType()
+                       .defaultValue(true)
+                       .withDescription("Disables flushing on checkpoint");
+       public static final ConfigOption<Integer> BULK_FLUSH_MAX_ACTIONS_OPTION 
=
+               ConfigOptions.key("sink.bulk-flush.max-actions")
+                       .intType()
+                       .noDefaultValue()
+                       .withDescription("Maximum number of actions to buffer 
for each bulk request.");
+       public static final ConfigOption<MemorySize> BULK_FLASH_MAX_SIZE_OPTION 
=
+               ConfigOptions.key("sink.bulk-flush.max-size")
+                       .memoryType()
+                       .noDefaultValue()
+                       .withDescription("Maximum size of buffered actions per 
bulk request");
+       public static final ConfigOption<Duration> BULK_FLUSH_INTERVAL_OPTION =
+               ConfigOptions.key("sink.bulk-flush.interval")
+                       .durationType()
+                       .noDefaultValue()
+                       .withDescription("Bulk flush interval");
+       public static final ConfigOption<BackOffType> 
BULK_FLUSH_BACKOFF_TYPE_OPTION =
+               ConfigOptions.key("sink.bulk-flush.back-off.strategy")
+                       .enumType(BackOffType.class)
+                       .defaultValue(BackOffType.DISABLED)
+                       .withDescription("Backoff strategy");
+       public static final ConfigOption<Integer> 
BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION =
+               ConfigOptions.key("sink.bulk-flush.backoff.max-retries")

Review comment:
       sink.bulk-flush.back-off.max-retries

##########
File path: 
flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.elasticsearch.table;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.description.Description;
+import 
org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase;
+
+import java.time.Duration;
+import java.util.List;
+
+import static org.apache.flink.configuration.description.TextElement.text;
+
+/**
+ * Options for {@link 
org.apache.flink.table.factories.DynamicTableSinkFactory} for Elasticsearch.
+ */
+public class ElasticsearchOptions {
+       /**
+        * Backoff strategy. Extends {@link 
ElasticsearchSinkBase.FlushBackoffType} with
+        * {@code DISABLED} option.
+        */
+       public enum BackOffType {
+               DISABLED,
+               CONSTANT,
+               EXPONENTIAL
+       }
+
+       public static final ConfigOption<List<String>> HOSTS_OPTION =
+               ConfigOptions.key("hosts")
+                       .stringType()
+                       .asList()
+                       .noDefaultValue()
+                       .withDescription("Elasticseatch hosts to connect to.");
+       public static final ConfigOption<String> INDEX_OPTION =
+               ConfigOptions.key("index")
+                       .stringType()
+                       .noDefaultValue()
+                       .withDescription("Elasticsearch index for every 
record.");
+       public static final ConfigOption<String> DOCUMENT_TYPE_OPTION =
+               ConfigOptions.key("document-type")
+                       .stringType()
+                       .noDefaultValue()
+                       .withDescription("Elasticsearch document type.");
+       public static final ConfigOption<String> KEY_DELIMITER_OPTION =
+               ConfigOptions.key("document-id.key-delimiter")
+                       .stringType()
+                       .defaultValue("_")
+                       .withDescription("Delimiter for composite keys e.g., 
\"$\" would result in IDs \"KEY1$KEY2$KEY3\".");
+       public static final ConfigOption<String> FAILURE_HANDLER_OPTION =
+               ConfigOptions.key("failure-handler")
+                       .stringType()
+                       .defaultValue("fail")
+                       .withDescription(Description.builder()
+                               .text("Failure handling strategy in case a 
request to Elasticsearch fails")
+                               .list(
+                                       text("\"fail\" (throws an exception if 
a request fails and thus causes a job failure),"),
+                                       text("\"ignore\" (ignores failures and 
drops the request),"),
+                                       text("\"retry_rejected\" (re-adds 
requests that have failed due to queue capacity saturation),"),
+                                       text("\"class name\" for failure 
handling with a ActionRequestFailureHandler subclass"))
+                               .build());
+       public static final ConfigOption<Boolean> FLUSH_ON_CHECKPOINT_OPTION =
+               ConfigOptions.key("sink.flush-on-checkpoint")
+                       .booleanType()
+                       .defaultValue(true)
+                       .withDescription("Disables flushing on checkpoint");
+       public static final ConfigOption<Integer> BULK_FLUSH_MAX_ACTIONS_OPTION 
=
+               ConfigOptions.key("sink.bulk-flush.max-actions")
+                       .intType()
+                       .noDefaultValue()
+                       .withDescription("Maximum number of actions to buffer 
for each bulk request.");
+       public static final ConfigOption<MemorySize> BULK_FLASH_MAX_SIZE_OPTION 
=
+               ConfigOptions.key("sink.bulk-flush.max-size")
+                       .memoryType()
+                       .noDefaultValue()
+                       .withDescription("Maximum size of buffered actions per 
bulk request");
+       public static final ConfigOption<Duration> BULK_FLUSH_INTERVAL_OPTION =
+               ConfigOptions.key("sink.bulk-flush.interval")
+                       .durationType()
+                       .noDefaultValue()
+                       .withDescription("Bulk flush interval");
+       public static final ConfigOption<BackOffType> 
BULK_FLUSH_BACKOFF_TYPE_OPTION =
+               ConfigOptions.key("sink.bulk-flush.back-off.strategy")
+                       .enumType(BackOffType.class)
+                       .defaultValue(BackOffType.DISABLED)
+                       .withDescription("Backoff strategy");
+       public static final ConfigOption<Integer> 
BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION =
+               ConfigOptions.key("sink.bulk-flush.backoff.max-retries")
+                       .intType()
+                       .noDefaultValue()
+                       .withDescription("Maximum number of retries.");
+       public static final ConfigOption<Duration> 
BULK_FLUSH_BACKOFF_DELAY_OPTION =
+               ConfigOptions.key("sink.bulk-flush.backoff.delay")

Review comment:
       sink.bulk-flush.back-off.delay

##########
File path: 
flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import 
org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction;
+import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+
+import org.elasticsearch.action.ActionRequest;
+import org.elasticsearch.action.delete.DeleteRequest;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.update.UpdateRequest;
+import org.elasticsearch.common.xcontent.XContentType;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.function.Function;
+
+/**
+ * Sink function for converting upserts into Elasticsearch {@link 
ActionRequest}s.
+ */
+@Internal
+class RowElasticsearchSinkFunction implements 
ElasticsearchSinkFunction<RowData> {
+
+       private static final long serialVersionUID = 1L;
+
+       private final IndexGenerator indexGenerator;
+       private final String docType;
+       private final SerializationSchema<RowData> serializationSchema;
+       private final XContentType contentType;
+       private final RequestFactory requestFactory;
+       private final Function<RowData, String> createKey;
+
+       public RowElasticsearchSinkFunction(
+                       IndexGenerator indexGenerator,
+                       @Nullable String docType, // this is deprecated in es 7+
+                       SerializationSchema<RowData> serializationSchema,
+                       XContentType contentType,
+                       RequestFactory requestFactory,
+                       Function<RowData, String> createKey) {
+               this.indexGenerator = 
Preconditions.checkNotNull(indexGenerator);
+               this.docType = docType;
+               this.serializationSchema = 
Preconditions.checkNotNull(serializationSchema);
+               this.contentType = Preconditions.checkNotNull(contentType);
+               this.requestFactory = 
Preconditions.checkNotNull(requestFactory);
+               this.createKey = Preconditions.checkNotNull(createKey);
+       }
+
+       @Override
+       public void process(
+                       RowData element,
+                       RuntimeContext ctx,
+                       RequestIndexer indexer) {
+               switch (element.getRowKind()) {
+                       case INSERT:
+                       case UPDATE_BEFORE:

Review comment:
       An `UPDATE_BEFORE` means it's a retraction of previous row, so we should 
`processDelete` for 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.

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


Reply via email to