You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/16 14:59:30 UTC

[GitHub] [flink] wuchong commented on a change in pull request #12184: [FLINK-17027] Introduce a new Elasticsearch 7 connector with new property keys

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



##########
File path: flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.backoff.type")

Review comment:
       "sink.bulk-flush.back-off.strategy"?

##########
File path: flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.backoff.type")
+			.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")
+			.durationType()
+			.noDefaultValue()
+			.withDescription("Delay between each backoff attempt.");
+	public static final ConfigOption<Duration> CONNECTION_MAX_RETRY_TIMEOUT_OPTION =
+		ConfigOptions.key("connection.max-retry-timeout")
+			.durationType()
+			.noDefaultValue()
+			.withDescription("Maximum timeout between retries.");
+	public static final ConfigOption<String> CONNECTION_PATH_PREFIX =
+		ConfigOptions.key("connection.path-prefix")
+			.stringType()
+			.noDefaultValue()
+			.withDescription("Prefix string to be added to every REST communication.");
+	public static final ConfigOption<String> FORMAT_OPTION =
+		ConfigOptions.key("format")
+			.stringType()
+			.defaultValue("json")

Review comment:
       I think we should add a validation on the format value? Should throw exception to tell users ES only supports json?

##########
File path: flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.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_AFTER:
+				processUpsert(element, indexer);
+				break;
+			case DELETE:

Review comment:
       For now, we have to add `case UPDATE_BEFORE` here, ignore before is just an optimization. There is still cases the before can't be ignored, e.g. multi-sink (one way requires before), filter on agg result (FLINK-9528). 

##########
File path: flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.serialization.SerializationSchema;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.format.SinkFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+
+import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_PATH_PREFIX;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.DOCUMENT_TYPE_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FAILURE_HANDLER_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FORMAT_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.INDEX_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.KEY_DELIMITER_OPTION;
+
+/**
+ * A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}.
+ */
+@Internal
+public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory {
+	private static final Set<ConfigOption<?>> requiredOptions = Stream.of(
+		HOSTS_OPTION,
+		INDEX_OPTION,
+		DOCUMENT_TYPE_OPTION
+	).collect(Collectors.toSet());
+	private static final Set<ConfigOption<?>> optionalOptions = Stream.of(
+		KEY_DELIMITER_OPTION,
+		FAILURE_HANDLER_OPTION,
+		FLUSH_ON_CHECKPOINT_OPTION,
+		BULK_FLASH_MAX_SIZE_OPTION,
+		BULK_FLUSH_MAX_ACTIONS_OPTION,
+		BULK_FLUSH_INTERVAL_OPTION,
+		BULK_FLUSH_BACKOFF_TYPE_OPTION,
+		BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION,
+		BULK_FLUSH_BACKOFF_DELAY_OPTION,
+		CONNECTION_MAX_RETRY_TIMEOUT_OPTION,
+		CONNECTION_PATH_PREFIX,
+		FORMAT_OPTION
+	).collect(Collectors.toSet());
+
+	@Override
+	public DynamicTableSink createDynamicTableSink(Context context) {
+		ElasticsearchValidationUtils.validatePrimaryKey(context.getCatalogTable().getSchema());
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+
+		final SinkFormat<SerializationSchema<RowData>> format = helper.discoverSinkFormat(
+			SerializationFormatFactory.class,
+			FORMAT_OPTION);
+
+		helper.validate();
+		Configuration configuration = new Configuration();
+		context.getCatalogTable()
+			.getOptions()
+			.forEach(configuration::setString);
+		Elasticsearch6Configuration config = new Elasticsearch6Configuration(configuration, context.getClassLoader());
+
+		validate(config, configuration);
+
+		return new Elasticsearch6DynamicSink(
+			format,
+			config,
+			context.getCatalogTable().getSchema());

Review comment:
       It's a little tricky here. We should use `TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema())` here to get rid of computed columns. However `TableSchemaUtils.getPhysicalSchema` will remove primary key information too. Maybe we should keep primary key in `TableSchemaUtils.getPhysicalSchema` and use it here. 

##########
File path: flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot;
+
+/**
+ * Utility methods for validating Elasticsearch properties.
+ */
+@Internal
+class ElasticsearchValidationUtils {
+
+	private static final Set<LogicalTypeRoot> ILLEGAL_PRIMARY_KEY_TYPES = new LinkedHashSet<>();
+
+	static {
+		ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ARRAY);
+		ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MAP);
+		ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MULTISET);
+		ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.STRUCTURED_TYPE);
+		ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ROW);
+		ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.RAW);
+		ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BINARY);
+		ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARBINARY);
+	}
+
+	/**
+	 * Checks that the table does not have primary key defined on illegal types.
+	 * In Elasticsearch the primary key is used to calculate the Elasticsearch document id,
+	 * which is a string of up to 512 bytes. It cannot have whitespaces. As of now it is calculated
+	 * by concatenating the fields. Certain types do not have a good string representation to be used
+	 * in this scenario. The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and
+	 * {@link LogicalTypeRoot#RAW} type.
+	 */
+	public static void validatePrimaryKey(TableSchema schema) {
+		schema.getPrimaryKey().ifPresent(
+			key -> {
+				List<LogicalTypeRoot> illegalTypes = key.getColumns()
+					.stream()
+					.map(fieldName -> {
+						LogicalType logicalType = schema.getFieldDataType(fieldName).get().getLogicalType();
+						if (hasRoot(logicalType, LogicalTypeRoot.DISTINCT_TYPE)) {
+							return ((DistinctType) logicalType).getSourceType().getTypeRoot();
+						} else {
+							return logicalType.getTypeRoot();
+						}
+					})
+					.filter(ILLEGAL_PRIMARY_KEY_TYPES::contains)
+					.collect(Collectors.toList());
+
+				if (!illegalTypes.isEmpty()) {
+					throw new ValidationException(
+						String.format(
+							"The table has a primary key on columns of illegal types: %s.\n" +
+								" Elasticsearch sink does not support primary keys on columns of types: %s.",
+							ILLEGAL_PRIMARY_KEY_TYPES,
+							illegalTypes));

Review comment:
       Switch these two parameter?

##########
File path: flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.configuration.ReadableConfig;
+import org.apache.flink.table.api.ValidationException;
+
+import org.apache.http.HttpHost;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION;
+
+/**
+ * Elasticsearch 7 specific configuration.
+ */
+@Internal
+final class Elasticsearch6Configuration extends ElasticsearchConfiguration {
+	Elasticsearch6Configuration(ReadableConfig config, ClassLoader classLoader) {
+		super(config, classLoader);
+	}
+
+	public List<HttpHost> getHosts() {
+		return config.get(HOSTS_OPTION).stream()
+			.map(Elasticsearch6Configuration::validateAndParseHostsString)
+			.collect(Collectors.toList());
+	}
+
+	/**
+	 * Parse Hosts String to list.
+	 *
+	 * <p>Hosts String format was given as following:
+	 *
+	 * <pre>
+	 *     connector.hosts = http://host_name:9092;http://host_name:9093

Review comment:
       ```suggestion
   	 *     hosts = http://host_name:9092;http://host_name:9093
   ```

##########
File path: flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.backoff.type")
+			.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")
+			.durationType()
+			.noDefaultValue()
+			.withDescription("Delay between each backoff attempt.");
+	public static final ConfigOption<Duration> CONNECTION_MAX_RETRY_TIMEOUT_OPTION =
+		ConfigOptions.key("connection.max-retry-timeout")
+			.durationType()
+			.noDefaultValue()
+			.withDescription("Maximum timeout between retries.");
+	public static final ConfigOption<String> CONNECTION_PATH_PREFIX =
+		ConfigOptions.key("connection.path-prefix")
+			.stringType()
+			.noDefaultValue()
+			.withDescription("Prefix string to be added to every REST communication.");
+	public static final ConfigOption<String> FORMAT_OPTION =
+		ConfigOptions.key("format")
+			.stringType()
+			.defaultValue("json")
+			.withDescription("Prefix string to be added to every REST communication.");

Review comment:
       Wrong description?

##########
File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.configuration.ReadableConfig;
+import org.apache.flink.table.api.ValidationException;
+
+import org.apache.http.HttpHost;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION;
+
+/**
+ * Elasticsearch 7 specific configuration.
+ */
+@Internal
+final class Elasticsearch7Configuration extends ElasticsearchConfiguration {
+	Elasticsearch7Configuration(ReadableConfig config, ClassLoader classLoader) {
+		super(config, classLoader);
+	}
+
+	public List<HttpHost> getHosts() {
+		return config.get(HOSTS_OPTION).stream()
+			.map(Elasticsearch7Configuration::validateAndParseHostsString)
+			.collect(Collectors.toList());
+	}
+
+	/**
+	 * Parse Hosts String to list.
+	 *
+	 * <p>Hosts String format was given as following:
+	 *
+	 * <pre>
+	 *     connector.hosts = http://host_name:9092;http://host_name:9093

Review comment:
       ```suggestion
   	 *     hosts = http://host_name:9092;http://host_name:9093
   ```

##########
File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.serialization.SerializationSchema;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.format.SinkFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+
+import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_PATH_PREFIX;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FAILURE_HANDLER_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FORMAT_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.INDEX_OPTION;
+import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.KEY_DELIMITER_OPTION;
+
+/**
+ * A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}.
+ */
+@Internal
+public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory {
+	private static final Set<ConfigOption<?>> requiredOptions = Stream.of(
+		HOSTS_OPTION,
+		INDEX_OPTION
+	).collect(Collectors.toSet());
+	private static final Set<ConfigOption<?>> optionalOptions = Stream.of(

Review comment:
       Do we need to add `DOCUMENT_TYPE_OPTION` to the ES7 factory? I think it's still valid?




----------------------------------------------------------------
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:
users@infra.apache.org