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



##########
File path: 
flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.connector.base.DeliveryGuarantee;
+
+import java.time.Duration;
+import java.util.List;
+
+/** Options for the Elasticsearch connector. */
+public class Elasticsearch7ConnectorOptions {
+
+    private Elasticsearch7ConnectorOptions() {}
+
+    public static final ConfigOption<List<String>> HOSTS_OPTION =
+            ConfigOptions.key("hosts")
+                    .stringType()
+                    .asList()
+                    .noDefaultValue()
+                    .withDescription("Elasticsearch 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> PASSWORD_OPTION =
+            ConfigOptions.key("password")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Password used to connect to 
Elasticsearch instance.");
+
+    public static final ConfigOption<String> USERNAME_OPTION =
+            ConfigOptions.key("username")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Username used to connect to 
Elasticsearch instance.");
+
+    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<Integer> BULK_FLUSH_MAX_ACTIONS_OPTION =
+            ConfigOptions.key("sink.bulk-flush.max-actions")
+                    .intType()
+                    .defaultValue(1000)
+                    .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()
+                    .defaultValue(MemorySize.parse("2mb"))
+                    .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()
+                    .defaultValue(Duration.ofSeconds(1))
+                    .withDescription("Bulk flush interval");
+
+    public static final ConfigOption<ElasticsearchConnectorOptions.BackOffType>
+            BULK_FLUSH_BACKOFF_TYPE_OPTION =
+                    ConfigOptions.key("sink.bulk-flush.backoff.strategy")
+                            
.enumType(ElasticsearchConnectorOptions.BackOffType.class)
+                            
.defaultValue(ElasticsearchConnectorOptions.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")
+                    .durationType()
+                    .noDefaultValue()
+                    .withDescription("Delay between each backoff attempt.");
+
+    public static final ConfigOption<Duration> 
CONNECTION_MAX_RETRY_TIMEOUT_OPTION =

Review comment:
       Is this configuration used?

##########
File path: 
flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.connector.base.DeliveryGuarantee;
+
+import java.time.Duration;
+import java.util.List;
+
+/** Options for the Elasticsearch connector. */
+public class Elasticsearch7ConnectorOptions {

Review comment:
       We have to be careful when making classes public because they are 
visible to all users and they can start depending on them. Can we make the 
class package-private?

##########
File path: 
flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchProcessor.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.connector.elasticsearch.sink.ElasticsearchProcessor;
+import org.apache.flink.connector.elasticsearch.sink.RequestIndexer;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+
+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 java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Sink function for converting upserts into Elasticsearch {@link 
ActionRequest}s. */
+public class RowElasticsearchProcessor implements 
ElasticsearchProcessor<RowData> {

Review comment:
       Same here package-private?




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