zentol commented on code in PR #1: URL: https://github.com/apache/flink-connector-opensearch/pull/1#discussion_r1028176163
########## pom.xml: ########## @@ -0,0 +1,463 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- +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. +--> +<project + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" + xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> + + <parent> + <groupId>io.github.zentol.flink</groupId> + <artifactId>flink-connector-parent</artifactId> + <version>1.0</version> + </parent> + + <modelVersion>4.0.0</modelVersion> + + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-opensearch-parent</artifactId> + <version>1.0.0-SNAPSHOT</version> + <name>Flink : Connectors : Opensearch : Parent</name> + <packaging>pom</packaging> + <url>https://flink.apache.org</url> + <inceptionYear>2022</inceptionYear> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>https://www.apache.org/licenses/LICENSE-2.0.txt</url> + <distribution>repo</distribution> + </license> + </licenses> + + <scm> + <url>https://github.com/apache/flink-connector-opensearch</url> + <connection>[email protected]:apache/flink-connector-opensearch.git</connection> + <developerConnection> + scm:git:https://gitbox.apache.org/repos/asf/flink-connector-opensearch.git + </developerConnection> + </scm> + + <modules> + <module>flink-connector-opensearch</module> + <module>flink-connector-opensearch-e2e-tests</module> Review Comment: missing the sql connector module ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSink.java: ########## @@ -0,0 +1,91 @@ +/* + * 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.connector.opensearch.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.connector.base.DeliveryGuarantee; + +import org.apache.http.HttpHost; + +import java.io.IOException; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Flink Sink to insert or update data in an Opensearch index. The sink supports the following + * delivery guarantees. + * + * <ul> + * <li>{@link DeliveryGuarantee#NONE} does not provide any guarantees: actions are flushed to + * Opensearch only depending on the configurations of the bulk processor. In case of a + * failure, it might happen that actions are lost if the bulk processor still has buffered + * actions. + * <li>{@link DeliveryGuarantee#AT_LEAST_ONCE} on a checkpoint the sink will wait until all + * buffered actions are flushed to and acknowledged by Opensearch. No actions will be lost but + * actions might be sent to Opensearch multiple times when Flink restarts. These additional + * requests may cause inconsistent data in Opensearch right after the restart, but eventually + * everything will be consistent again. + * </ul> + * + * @param <IN> type of the records converted to Opensearch actions + * @see OpensearchSinkBuilder on how to construct a OpensearchSink + */ +@PublicEvolving +public class OpensearchSink<IN> implements Sink<IN> { Review Comment: Curious why this class lost the `getDeliveryGuarantee()` ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactory.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.connector.opensearch.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.connector.opensearch.sink.OpensearchSinkBuilder; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.StringUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.ALLOW_INSECURE; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.SOCKET_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.opensearch.common.Strings.capitalize; + +/** A {@link DynamicTableSinkFactory} for discovering OpensearchDynamicSink. */ +@Internal +public class OpensearchDynamicSinkFactory implements DynamicTableSinkFactory { + private static final String FACTORY_IDENTIFIER = "opensearch"; + + private final OpensearchSinkBuilderSupplier<RowData> sinkBuilderSupplier; + + public OpensearchDynamicSinkFactory() { + this.sinkBuilderSupplier = OpensearchSinkBuilder<RowData>::new; + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex = + getPrimaryKeyLogicalTypesWithIndex(context); + EncodingFormat<SerializationSchema<RowData>> format = + getValidatedEncodingFormat(this, context); + + OpensearchConfiguration config = getConfiguration(context); + validateConfiguration(config); + + return new OpensearchDynamicSink( + format, + config, + primaryKeyLogicalTypesWithIndex, + context.getPhysicalRowDataType(), + capitalize(FACTORY_IDENTIFIER), + sinkBuilderSupplier); + } + + OpensearchConfiguration getConfiguration(Context context) { Review Comment: can be private, probably also applies to other methods in here. ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactory.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.connector.opensearch.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.connector.opensearch.sink.OpensearchSinkBuilder; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.StringUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.ALLOW_INSECURE; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.SOCKET_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.opensearch.common.Strings.capitalize; + +/** A {@link DynamicTableSinkFactory} for discovering OpensearchDynamicSink. */ +@Internal +public class OpensearchDynamicSinkFactory implements DynamicTableSinkFactory { + private static final String FACTORY_IDENTIFIER = "opensearch"; + + private final OpensearchSinkBuilderSupplier<RowData> sinkBuilderSupplier; + + public OpensearchDynamicSinkFactory() { + this.sinkBuilderSupplier = OpensearchSinkBuilder<RowData>::new; + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex = + getPrimaryKeyLogicalTypesWithIndex(context); + EncodingFormat<SerializationSchema<RowData>> format = + getValidatedEncodingFormat(this, context); + + OpensearchConfiguration config = getConfiguration(context); + validateConfiguration(config); + + return new OpensearchDynamicSink( + format, + config, + primaryKeyLogicalTypesWithIndex, + context.getPhysicalRowDataType(), + capitalize(FACTORY_IDENTIFIER), + sinkBuilderSupplier); + } + + OpensearchConfiguration getConfiguration(Context context) { + return new OpensearchConfiguration( + Configuration.fromMap(context.getCatalogTable().getOptions())); + } + + void validateConfiguration(OpensearchConfiguration config) { + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize().getBytes(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLUSH_MAX_SIZE_OPTION.key(), + config.getBulkFlushMaxByteSize().toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + static void validate(boolean condition, Supplier<String> message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + EncodingFormat<SerializationSchema<RowData>> getValidatedEncodingFormat( + DynamicTableFactory factory, DynamicTableFactory.Context context) { + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(factory, context); + final EncodingFormat<SerializationSchema<RowData>> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + helper.validate(); + return format; + } + + List<LogicalTypeWithIndex> getPrimaryKeyLogicalTypesWithIndex(Context context) { + DataType physicalRowDataType = context.getPhysicalRowDataType(); + int[] primaryKeyIndexes = context.getPrimaryKeyIndexes(); + if (primaryKeyIndexes.length != 0) { + DataType pkDataType = Projection.of(primaryKeyIndexes).project(physicalRowDataType); + + OpensearchValidationUtils.validatePrimaryKey(pkDataType); + } + + ResolvedSchema resolvedSchema = context.getCatalogTable().getResolvedSchema(); + return Arrays.stream(primaryKeyIndexes) + .mapToObj( + index -> { + Optional<Column> column = resolvedSchema.getColumn(index); + if (!column.isPresent()) { + throw new IllegalStateException( + String.format( + "No primary key column found with index '%s'.", + index)); + } + LogicalType logicalType = column.get().getDataType().getLogicalType(); + return new LogicalTypeWithIndex(index, logicalType); + }) + .collect(Collectors.toList()); + } + + @Override + public Set<ConfigOption<?>> requiredOptions() { + return Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); + } + + @Override + public Set<ConfigOption<?>> optionalOptions() { Review Comment: Why is this missing `forwardOptions()` (FLINK-25391)? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSink.java: ########## @@ -0,0 +1,201 @@ +/* + * 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.connector.opensearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.opensearch.sink.FlushBackoffType; +import org.apache.flink.connector.opensearch.sink.OpensearchSink; +import org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; + +import org.apache.http.HttpHost; +import org.opensearch.common.xcontent.XContentType; + +import java.util.List; +import java.util.Objects; +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link DynamicTableSink} that describes how to create a {@link OpensearchSink} from a logical + * description. + */ +@Internal +class OpensearchDynamicSink implements DynamicTableSink { + + final EncodingFormat<SerializationSchema<RowData>> format; + final DataType physicalRowDataType; + final List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex; + final OpensearchConfiguration config; Review Comment: The lack of the localTimeZoneId field implies that FLINK-24571 isn't contained in this PR. I'm wondering why that is. ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactory.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.connector.opensearch.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.connector.opensearch.sink.OpensearchSinkBuilder; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.StringUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.ALLOW_INSECURE; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.SOCKET_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.opensearch.common.Strings.capitalize; + +/** A {@link DynamicTableSinkFactory} for discovering OpensearchDynamicSink. */ +@Internal +public class OpensearchDynamicSinkFactory implements DynamicTableSinkFactory { + private static final String FACTORY_IDENTIFIER = "opensearch"; + + private final OpensearchSinkBuilderSupplier<RowData> sinkBuilderSupplier; + + public OpensearchDynamicSinkFactory() { + this.sinkBuilderSupplier = OpensearchSinkBuilder<RowData>::new; + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex = + getPrimaryKeyLogicalTypesWithIndex(context); + EncodingFormat<SerializationSchema<RowData>> format = + getValidatedEncodingFormat(this, context); + + OpensearchConfiguration config = getConfiguration(context); + validateConfiguration(config); + + return new OpensearchDynamicSink( + format, + config, + primaryKeyLogicalTypesWithIndex, + context.getPhysicalRowDataType(), + capitalize(FACTORY_IDENTIFIER), + sinkBuilderSupplier); + } + + OpensearchConfiguration getConfiguration(Context context) { + return new OpensearchConfiguration( + Configuration.fromMap(context.getCatalogTable().getOptions())); + } + + void validateConfiguration(OpensearchConfiguration config) { + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize().getBytes(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLUSH_MAX_SIZE_OPTION.key(), + config.getBulkFlushMaxByteSize().toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + static void validate(boolean condition, Supplier<String> message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + EncodingFormat<SerializationSchema<RowData>> getValidatedEncodingFormat( Review Comment: Where does this come from? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.java: ########## @@ -0,0 +1,161 @@ +/* + * 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.connector.opensearch.table; + +import org.apache.flink.annotation.PublicEvolving; +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.opensearch.sink.FlushBackoffType; + +import java.time.Duration; +import java.util.List; + +/** + * Base options for the Opensearch connector. Needs to be public so that the {@link + * org.apache.flink.table.api.TableDescriptor} can access it. + */ +@PublicEvolving +public class OpensearchConnectorOptions { + + OpensearchConnectorOptions() {} + + public static final ConfigOption<List<String>> HOSTS_OPTION = + ConfigOptions.key("hosts") + .stringType() + .asList() + .noDefaultValue() + .withDescription("Opensearch hosts to connect to."); + + public static final ConfigOption<String> INDEX_OPTION = + ConfigOptions.key("index") + .stringType() + .noDefaultValue() + .withDescription("Opensearch index for every record."); + + public static final ConfigOption<String> PASSWORD_OPTION = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("Password used to connect to Opensearch instance."); + + public static final ConfigOption<String> USERNAME_OPTION = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("Username used to connect to Opensearch 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_FLUSH_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<FlushBackoffType> BULK_FLUSH_BACKOFF_TYPE_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.strategy") + .enumType(FlushBackoffType.class) + .noDefaultValue() + .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<Boolean> FLUSH_ON_CHECKPOINT_OPTION = Review Comment: effectively unused ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java: ########## @@ -0,0 +1,168 @@ +/* + * 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.connector.opensearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.opensearch.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.connector.opensearch.table.OpensearchConnectorOptions.ALLOW_INSECURE; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.SOCKET_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Opensearch base configuration. */ +@Internal +class OpensearchConfiguration { + protected final ReadableConfig config; + + OpensearchConfiguration(ReadableConfig config) { + this.config = checkNotNull(config); + } + + public int getBulkFlushMaxActions() { + return config.get(BULK_FLUSH_MAX_ACTIONS_OPTION); + } + + public MemorySize getBulkFlushMaxByteSize() { + return config.get(BULK_FLUSH_MAX_SIZE_OPTION); + } + + public long getBulkFlushInterval() { + return config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); + } + + public DeliveryGuarantee getDeliveryGuarantee() { + return config.get(DELIVERY_GUARANTEE_OPTION); + } + + public Optional<String> getUsername() { + return config.getOptional(USERNAME_OPTION); + } + + public Optional<String> getPassword() { + return config.getOptional(PASSWORD_OPTION); + } + + public Optional<FlushBackoffType> getBulkFlushBackoffType() { + return config.getOptional(BULK_FLUSH_BACKOFF_TYPE_OPTION); + } + + public Optional<Integer> getBulkFlushBackoffRetries() { + return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); + } + + public Optional<Long> getBulkFlushBackoffDelay() { + return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); + } + + public boolean isDisableFlushOnCheckpoint() { Review Comment: unused. I also cant find the corresponding option in the ES connector; where does it come from? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/NetworkClientConfig.java: ########## @@ -0,0 +1,86 @@ +/* + * 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.connector.opensearch.sink; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +class NetworkClientConfig implements Serializable { + + @Nullable private final String username; + @Nullable private final String password; + @Nullable private final String connectionPathPrefix; + @Nullable private final Integer connectionRequestTimeout; + @Nullable private final Integer connectionTimeout; + @Nullable private final Integer socketTimeout; + @Nullable private final Boolean allowInsecure; + + NetworkClientConfig( + @Nullable String username, + @Nullable String password, + @Nullable String connectionPathPrefix, + @Nullable Integer connectionRequestTimeout, + @Nullable Integer connectionTimeout, + @Nullable Integer socketTimeout, + @Nullable Boolean allowInsecure) { + this.username = username; + this.password = password; + this.connectionPathPrefix = connectionPathPrefix; + this.connectionRequestTimeout = connectionRequestTimeout; + this.connectionTimeout = connectionTimeout; + this.socketTimeout = socketTimeout; + this.allowInsecure = allowInsecure; + } + + @Nullable + public String getUsername() { + return username; + } + + @Nullable + public String getPassword() { + return password; + } + + @Nullable + public Integer getConnectionRequestTimeout() { + return connectionRequestTimeout; + } + + @Nullable + public Integer getConnectionTimeout() { + return connectionTimeout; + } + + @Nullable + public Integer getSocketTimeout() { + return socketTimeout; + } + + @Nullable + public String getConnectionPathPrefix() { + return connectionPathPrefix; + } + + @Nullable + public Boolean isAllowInsecure() { + return allowInsecure; + } Review Comment: Consider improving on the existing code and returning an Optional instead. ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilder.java: ########## @@ -0,0 +1,449 @@ +/* + * 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.connector.opensearch.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.ClosureCleaner; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.http.HttpHost; +import org.opensearch.action.ActionListener; +import org.opensearch.action.bulk.BackoffPolicy; +import org.opensearch.action.bulk.BulkProcessor; +import org.opensearch.action.bulk.BulkRequest; +import org.opensearch.action.bulk.BulkResponse; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.RestHighLevelClient; +import org.opensearch.common.unit.ByteSizeUnit; +import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.common.unit.TimeValue; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Builder to construct an Opensearch compatible {@link OpensearchSink}. + * + * <p>The following example shows the minimal setup to create a OpensearchSink that submits actions + * on checkpoint or the default number of actions was buffered (1000). + * + * <pre>{@code + * OpensearchSink<String> sink = new OpensearchSinkBuilder<String>() + * .setHosts(new HttpHost("localhost:9200") + * .setEmitter((element, context, indexer) -> { + * indexer.add( + * new IndexRequest("my-index") + * .id(element.f0.toString()) + * .source(element.f1) + * ); + * }) + * .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + * .build(); + * }</pre> + * + * @param <IN> type of the records converted to Opensearch actions + */ +@PublicEvolving +public class OpensearchSinkBuilder<IN> { + + private int bulkFlushMaxActions = 1000; + private int bulkFlushMaxMb = -1; + private long bulkFlushInterval = -1; + private FlushBackoffType bulkFlushBackoffType = FlushBackoffType.NONE; + private int bulkFlushBackoffRetries = -1; + private long bulkFlushBackOffDelay = -1; + private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE; + private List<HttpHost> hosts; + protected OpensearchEmitter<? super IN> emitter; + private String username; + private String password; + private String connectionPathPrefix; + private Integer connectionTimeout; + private Integer connectionRequestTimeout; + private Integer socketTimeout; + private Boolean allowInsecure; + + public OpensearchSinkBuilder() {} + + @SuppressWarnings("unchecked") + protected <S extends OpensearchSinkBuilder<?>> S self() { + return (S) this; + } + + /** + * Sets the emitter which is invoked on every record to convert it to Opensearch actions. + * + * @param emitter to process records into Opensearch actions. + * @return this builder + */ + public <T extends IN> OpensearchSinkBuilder<T> setEmitter( + OpensearchEmitter<? super T> emitter) { + checkNotNull(emitter); + checkState( + InstantiationUtil.isSerializable(emitter), + "The Opensearch emitter must be serializable."); + + final OpensearchSinkBuilder<T> self = self(); + self.emitter = emitter; + return self; + } + + /** + * Sets the hosts where the Opensearch cluster nodes are reachable. + * + * @param hosts http addresses describing the node locations + * @return this builder + */ + public OpensearchSinkBuilder<IN> setHosts(HttpHost... hosts) { + checkNotNull(hosts); + checkState(hosts.length > 0, "Hosts cannot be empty."); + this.hosts = Arrays.asList(hosts); + return self(); + } + + /** + * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link + * DeliveryGuarantee#NONE} + * + * @param deliveryGuarantee which describes the record emission behaviour + * @return this builder + */ + public OpensearchSinkBuilder<IN> setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) { + checkState( + deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE, + "Opensearch sink does not support the EXACTLY_ONCE guarantee."); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + return self(); + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to + * disable it. The default flush size 1000. + * + * @param numMaxActions the maximum number of actions to buffer per bulk request. + * @return this builder + */ + public OpensearchSinkBuilder<IN> setBulkFlushMaxActions(int numMaxActions) { + checkState( + numMaxActions == -1 || numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + this.bulkFlushMaxActions = numMaxActions; + return self(); + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. You can pass -1 to + * disable it. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + * @return this builder + */ + public OpensearchSinkBuilder<IN> setBulkFlushMaxSizeMb(int maxSizeMb) { + checkState( + maxSizeMb == -1 || maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + this.bulkFlushMaxMb = maxSizeMb; + return self(); + } + + /** + * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + * @return this builder + */ + public OpensearchSinkBuilder<IN> setBulkFlushInterval(long intervalMillis) { + checkState( + intervalMillis == -1 || intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than " + + "or equal to 0."); + this.bulkFlushInterval = intervalMillis; + return self(); + } + + /** + * Sets the type of back off to use when flushing bulk requests. The default bulk flush back off + * type is {@link FlushBackoffType#NONE}. + * + * <p>Sets the amount of delay between each backoff attempt when flushing bulk requests, in + * milliseconds. + * + * <p>Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + * @return this builder + */ + public OpensearchSinkBuilder<IN> setBulkFlushBackoffStrategy( + FlushBackoffType flushBackoffType, int maxRetries, long delayMillis) { + this.bulkFlushBackoffType = checkNotNull(flushBackoffType); + checkState( + flushBackoffType != FlushBackoffType.NONE, + "FlushBackoffType#NONE does not require a configuration it is the default, retries and delay are ignored."); + checkState(maxRetries > 0, "Max number of backoff attempts must be larger than 0."); + this.bulkFlushBackoffRetries = maxRetries; + checkState( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger " + + "than or equal to 0."); + this.bulkFlushBackOffDelay = delayMillis; + return self(); + } + + /** + * Sets the username used to authenticate the connection with the Opensearch cluster. + * + * @param username of the Opensearch cluster user + * @return this builder + */ + public OpensearchSinkBuilder<IN> setConnectionUsername(String username) { + checkNotNull(username); + this.username = username; + return self(); + } + + /** + * Sets the password used to authenticate the conection with the Opensearch cluster. + * + * @param password of the Opensearch cluster user + * @return this builder + */ + public OpensearchSinkBuilder<IN> setConnectionPassword(String password) { + checkNotNull(password); + this.password = password; + return self(); + } + + /** + * Sets a prefix which used for every REST communication to the Opensearch cluster. + * + * @param prefix for the communication + * @return this builder + */ + public OpensearchSinkBuilder<IN> setConnectionPathPrefix(String prefix) { + checkNotNull(prefix); + this.connectionPathPrefix = prefix; + return self(); + } + + /** + * Sets the timeout for requesting the connection of the Opensearch cluster from the connection + * manager. + * + * @param timeout for the connection request + * @return this builder + */ + public OpensearchSinkBuilder<IN> setConnectionRequestTimeout(int timeout) { + checkState(timeout >= 0, "Connection request timeout must be larger than or equal to 0."); + this.connectionRequestTimeout = timeout; + return self(); + } + + /** + * Sets the timeout for establishing a connection of the Opensearch cluster. + * + * @param timeout for the connection + * @return this builder + */ + public OpensearchSinkBuilder<IN> setConnectionTimeout(int timeout) { + checkState(timeout >= 0, "Connection timeout must be larger than or equal to 0."); + this.connectionTimeout = timeout; + return self(); + } + + /** + * Sets the timeout for waiting for data or, put differently, a maximum period inactivity + * between two consecutive data packets. + * + * @param timeout for the socket + * @return this builder + */ + public OpensearchSinkBuilder<IN> setSocketTimeout(int timeout) { + checkState(timeout >= 0, "Socket timeout must be larger than or equal to 0."); + this.socketTimeout = timeout; + return self(); + } + + /** + * Allows to bypass the certificates chain validation and connect to insecure network endpoints + * (for example, servers which use self-signed certificates). + * + * @param allowInsecure allow or not to insecure network endpoints + * @return this builder + */ + public OpensearchSinkBuilder<IN> setAllowInsecure(boolean allowInsecure) { + this.allowInsecure = allowInsecure; + return self(); + } + + protected BulkProcessorBuilderFactory getBulkProcessorBuilderFactory() { Review Comment: There's no reason for this to be protected since BulkProcessorBuilderFactory is an internal interface. Maybe that entire class shouldn't exist in it's current form, as it only existed to supported different elasticsearch versions (unless we intend to do the same in the near future for opensearch). ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactory.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.connector.opensearch.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.connector.opensearch.sink.OpensearchSinkBuilder; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.StringUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.ALLOW_INSECURE; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.SOCKET_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.opensearch.common.Strings.capitalize; + +/** A {@link DynamicTableSinkFactory} for discovering OpensearchDynamicSink. */ +@Internal +public class OpensearchDynamicSinkFactory implements DynamicTableSinkFactory { + private static final String FACTORY_IDENTIFIER = "opensearch"; + + private final OpensearchSinkBuilderSupplier<RowData> sinkBuilderSupplier; + + public OpensearchDynamicSinkFactory() { + this.sinkBuilderSupplier = OpensearchSinkBuilder<RowData>::new; Review Comment: ```suggestion this.sinkBuilderSupplier = OpensearchSinkBuilder::new; ``` not required ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactory.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.connector.opensearch.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.connector.opensearch.sink.OpensearchSinkBuilder; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.StringUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.ALLOW_INSECURE; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.SOCKET_TIMEOUT; +import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.opensearch.common.Strings.capitalize; + +/** A {@link DynamicTableSinkFactory} for discovering OpensearchDynamicSink. */ +@Internal +public class OpensearchDynamicSinkFactory implements DynamicTableSinkFactory { + private static final String FACTORY_IDENTIFIER = "opensearch"; + + private final OpensearchSinkBuilderSupplier<RowData> sinkBuilderSupplier; + + public OpensearchDynamicSinkFactory() { + this.sinkBuilderSupplier = OpensearchSinkBuilder<RowData>::new; Review Comment: This field doesn't seem necessary in the first place; we could handle that within the sink. -- 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]
