fapaul commented on a change in pull request #17374: URL: https://github.com/apache/flink/pull/17374#discussion_r724926736
########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.java ########## @@ -0,0 +1,122 @@ +/* + * 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 org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; + +import java.time.Duration; +import java.util.List; + +/** Options for the Elasticsearch connector. */ +class Elasticsearch7ConnectorOptions { Review comment: This class needs to be public that the `TableDescriptor` can access it. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java ########## @@ -20,20 +20,82 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; import org.apache.flink.table.api.ValidationException; import org.apache.http.HttpHost; +import java.time.Duration; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.util.Preconditions.checkNotNull; /** Elasticsearch 7 specific configuration. */ @Internal -final class Elasticsearch7Configuration extends ElasticsearchConfiguration { - Elasticsearch7Configuration(ReadableConfig config, ClassLoader classLoader) { - super(config, classLoader); +final class Elasticsearch7Configuration { + protected final ReadableConfig config; + + Elasticsearch7Configuration(ReadableConfig config) { + this.config = checkNotNull(config); + } + + public int getBulkFlushMaxActions() { + return config.get(ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); + } + + public long getBulkFlushMaxByteSize() { + return config.get(ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION).getBytes(); + } Review comment: They still refer to the old ConnectorOptions. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java ########## @@ -63,25 +66,46 @@ 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_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, + CONNECTION_PATH_PREFIX_OPTION, FORMAT_OPTION, + DELIVERY_GUARANTEE_OPTION, PASSWORD_OPTION, USERNAME_OPTION) .collect(Collectors.toSet()); @Override public DynamicTableSink createDynamicTableSink(Context context) { - TableSchema tableSchema = context.getCatalogTable().getSchema(); - ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + DataType physicalRowDataType = context.getPhysicalRowDataType(); + int[] primaryKeyIndexes = context.getPrimaryKeyIndexes(); + if (primaryKeyIndexes.length != 0) { + DataType pkDataType = DataType.projectFields(physicalRowDataType, primaryKeyIndexes); + + ElasticsearchValidationUtils.validatePrimaryKey(pkDataType); + } + + ResolvedSchema resolvedSchema = context.getCatalogTable().getResolvedSchema(); + List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex = + Arrays.stream(primaryKeyIndexes) + .mapToObj( + index -> { + Optional<Column> column = resolvedSchema.getColumn(index); + if (!column.isPresent()) { + throw new ValidationException( Review comment: Nit: If I understand this correctly it is rather an `IllegalStateException` because we have extracted the primary keys with the given index and it should theoretically never happen that the column for this index does not exist ########## File path: flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java ########## @@ -93,5 +95,31 @@ public static void validatePrimaryKey(TableSchema schema) { }); } + public static void validatePrimaryKey(DataType primaryKeyDataType) { Review comment: Nit: Duplicate the doc string from the deprecated method? -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
