diff --git a/flink-connectors/flink-connector-opensearch/pom.xml b/flink-connectors/flink-connector-opensearch/pom.xml new file mode 100644 index 00000000000000..f889db69412ffe --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/pom.xml @@ -0,0 +1,186 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + .. + + + flink-connector-opensearch + Flink : Connectors : Opensearch + + jar + + + + 1.3.0 + + + + + + + + org.apache.flink + flink-connector-base + ${project.version} + + + org.apache.flink + flink-streaming-java + ${project.version} + provided + + + + + + + org.apache.flink + flink-table-api-java-bridge + ${project.version} + provided + true + + + + + + org.opensearch + opensearch + ${opensearch.version} + + + + org.opensearch.client + opensearch-rest-high-level-client + ${opensearch.version} + + + org.apache.httpcomponents + httpcore-nio + + + + + + + org.apache.httpcomponents + httpcore-nio + 4.4.12 + + + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + + org.apache.flink + flink-runtime + ${project.version} + test-jar + test + + + + org.apache.flink + flink-streaming-java + ${project.version} + test + test-jar + + + + + org.apache.flink + flink-table-common + ${project.version} + test-jar + test + + + + + org.apache.flink + flink-json + ${project.version} + test + + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test + + + + + + org.apache.logging.log4j + log4j-api + provided + + + + org.apache.logging.log4j + log4j-core + test + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + 1 + + + + + + diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorBuilderFactory.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorBuilderFactory.java new file mode 100644 index 00000000000000..ff284a64e3ded2 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorBuilderFactory.java @@ -0,0 +1,36 @@ +/* + * 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.Internal; +import org.apache.flink.util.function.TriFunction; + +import org.opensearch.action.bulk.BulkProcessor; +import org.opensearch.client.RestHighLevelClient; + +import java.io.Serializable; + +@Internal +interface BulkProcessorBuilderFactory + extends Serializable, + TriFunction< + RestHighLevelClient, + BulkProcessorConfig, + BulkProcessor.Listener, + BulkProcessor.Builder> {} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorConfig.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorConfig.java new file mode 100644 index 00000000000000..07ed9a4537d7fe --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorConfig.java @@ -0,0 +1,72 @@ +/* + * 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 java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class BulkProcessorConfig implements Serializable { + + private final int bulkFlushMaxActions; + private final int bulkFlushMaxMb; + private final long bulkFlushInterval; + private final FlushBackoffType flushBackoffType; + private final int bulkFlushBackoffRetries; + private final long bulkFlushBackOffDelay; + + BulkProcessorConfig( + int bulkFlushMaxActions, + int bulkFlushMaxMb, + long bulkFlushInterval, + FlushBackoffType flushBackoffType, + int bulkFlushBackoffRetries, + long bulkFlushBackOffDelay) { + this.bulkFlushMaxActions = bulkFlushMaxActions; + this.bulkFlushMaxMb = bulkFlushMaxMb; + this.bulkFlushInterval = bulkFlushInterval; + this.flushBackoffType = checkNotNull(flushBackoffType); + this.bulkFlushBackoffRetries = bulkFlushBackoffRetries; + this.bulkFlushBackOffDelay = bulkFlushBackOffDelay; + } + + public int getBulkFlushMaxActions() { + return bulkFlushMaxActions; + } + + public int getBulkFlushMaxMb() { + return bulkFlushMaxMb; + } + + public long getBulkFlushInterval() { + return bulkFlushInterval; + } + + public FlushBackoffType getFlushBackoffType() { + return flushBackoffType; + } + + public int getBulkFlushBackoffRetries() { + return bulkFlushBackoffRetries; + } + + public long getBulkFlushBackOffDelay() { + return bulkFlushBackOffDelay; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkRequestConsumerFactory.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkRequestConsumerFactory.java new file mode 100644 index 00000000000000..0eb3f6872e2151 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkRequestConsumerFactory.java @@ -0,0 +1,35 @@ +/* + * 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.Internal; + +import org.opensearch.action.ActionListener; +import org.opensearch.action.bulk.BulkRequest; +import org.opensearch.action.bulk.BulkResponse; + +import java.util.function.BiConsumer; + +/** + * {@link BulkRequestConsumerFactory} is used to bridge incompatible Opensearch Java API calls + * across different Opensearch versions. + */ +@Internal +interface BulkRequestConsumerFactory + extends BiConsumer> {} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/FlushBackoffType.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/FlushBackoffType.java new file mode 100644 index 00000000000000..5b015ec5eaabf6 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/FlushBackoffType.java @@ -0,0 +1,38 @@ +/* + * 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; + +/** + * Used to control whether the sink should retry failed requests at all or with which kind back off + * strategy. + */ +@PublicEvolving +public enum FlushBackoffType { + /** After every failure, it waits a configured time until the retries are exhausted. */ + CONSTANT, + /** + * After every failure, it waits initially the configured time and increases the waiting time + * exponentially until the retries are exhausted. + */ + EXPONENTIAL, + /** The failure is not retried. */ + NONE, +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/NetworkClientConfig.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/NetworkClientConfig.java new file mode 100644 index 00000000000000..cb9993d54a887a --- /dev/null +++ b/flink-connectors/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; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchEmitter.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchEmitter.java new file mode 100644 index 00000000000000..cf2e0c461d45e4 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchEmitter.java @@ -0,0 +1,77 @@ +/* + * 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.functions.Function; +import org.apache.flink.api.connector.sink.SinkWriter; + +import org.opensearch.action.ActionRequest; + +/** + * Creates none or multiple {@link ActionRequest ActionRequests} from the incoming elements. + * + *

This is used by sinks to prepare elements for sending them to Opensearch. + * + *

Example: + * + *

{@code
+ * private static class TestOpensearchEmitter implements OpensearchEmitter> {
+ *
+ *     public IndexRequest createIndexRequest(Tuple2 element) {
+ *         Map document = new HashMap<>();
+ * 		   document.put("data", element.f1);
+ *
+ * 	       return Requests.indexRequest()
+ * 		       .index("my-index")
+ * 		       .id(element.f0.toString())
+ * 		       .source(document);
+ *     }
+ *
+ * 	   public void emit(Tuple2 element, RequestIndexer indexer) {
+ * 	       indexer.add(createIndexRequest(element));
+ *     }
+ * }
+ *
+ * }
+ * + * @param The type of the element handled by this {@link OpensearchEmitter} + */ +@PublicEvolving +public interface OpensearchEmitter extends Function { + + /** + * Initialization method for the function. It is called once before the actual working process + * methods. + */ + default void open() throws Exception {} + + /** Tear-down method for the function. It is called when the sink closes. */ + default void close() throws Exception {} + + /** + * Process the incoming element to produce multiple {@link ActionRequest ActionRequests}. The + * produced requests should be added to the provided {@link RequestIndexer}. + * + * @param element incoming element to process + * @param context to access additional information about the record + * @param indexer request indexer that {@code ActionRequest} should be added to + */ + void emit(T element, SinkWriter.Context context, RequestIndexer indexer); +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSink.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSink.java new file mode 100644 index 00000000000000..6f4d28296c9c8f --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSink.java @@ -0,0 +1,121 @@ +/* + * 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.sink.Committer; +import org.apache.flink.api.connector.sink.GlobalCommitter; +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import org.apache.http.HttpHost; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; + +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. + * + *
    + *
  • {@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. + *
  • {@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. + *
+ * + * @param type of the records converted to Opensearch actions + * @see OpensearchSinkBuilder on how to construct a OpensearchSink + */ +@PublicEvolving +public class OpensearchSink implements Sink { + + private final List hosts; + private final OpensearchEmitter emitter; + private final BulkProcessorConfig buildBulkProcessorConfig; + private final BulkProcessorBuilderFactory bulkProcessorBuilderFactory; + private final NetworkClientConfig networkClientConfig; + private final DeliveryGuarantee deliveryGuarantee; + + OpensearchSink( + List hosts, + OpensearchEmitter emitter, + DeliveryGuarantee deliveryGuarantee, + BulkProcessorBuilderFactory bulkProcessorBuilderFactory, + BulkProcessorConfig buildBulkProcessorConfig, + NetworkClientConfig networkClientConfig) { + this.hosts = checkNotNull(hosts); + this.bulkProcessorBuilderFactory = checkNotNull(bulkProcessorBuilderFactory); + checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); + this.emitter = checkNotNull(emitter); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + this.buildBulkProcessorConfig = checkNotNull(buildBulkProcessorConfig); + this.networkClientConfig = checkNotNull(networkClientConfig); + } + + @Override + public SinkWriter createWriter(InitContext context, List states) + throws IOException { + return new OpensearchWriter<>( + hosts, + emitter, + deliveryGuarantee == DeliveryGuarantee.AT_LEAST_ONCE, + buildBulkProcessorConfig, + bulkProcessorBuilderFactory, + networkClientConfig, + context.metricGroup(), + context.getMailboxExecutor()); + } + + @Override + public Optional> getWriterStateSerializer() { + return Optional.empty(); + } + + @Override + public Optional> createCommitter() throws IOException { + return Optional.empty(); + } + + @Override + public Optional> createGlobalCommitter() throws IOException { + return Optional.empty(); + } + + @Override + public Optional> getCommittableSerializer() { + return Optional.empty(); + } + + @Override + public Optional> getGlobalCommittableSerializer() { + return Optional.empty(); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilder.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilder.java new file mode 100644 index 00000000000000..04ffbf7fa96669 --- /dev/null +++ b/flink-connectors/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}. + * + *

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). + * + *

{@code
+ * OpensearchSink sink = new OpensearchSinkBuilder()
+ *     .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();
+ * }
+ * + * @param type of the records converted to Opensearch actions + */ +@PublicEvolving +public class OpensearchSinkBuilder { + + 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.NONE; + private List hosts; + protected OpensearchEmitter 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 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 OpensearchSinkBuilder setEmitter( + OpensearchEmitter emitter) { + checkNotNull(emitter); + checkState( + InstantiationUtil.isSerializable(emitter), + "The Opensearch emitter must be serializable."); + + final OpensearchSinkBuilder 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 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 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 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 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 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}. + * + *

Sets the amount of delay between each backoff attempt when flushing bulk requests, in + * milliseconds. + * + *

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 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 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 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 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 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 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 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 setAllowInsecure(boolean allowInsecure) { + this.allowInsecure = allowInsecure; + return self(); + } + + protected BulkProcessorBuilderFactory getBulkProcessorBuilderFactory() { + return new BulkProcessorBuilderFactory() { + @Override + public BulkProcessor.Builder apply( + RestHighLevelClient client, + BulkProcessorConfig bulkProcessorConfig, + BulkProcessor.Listener listener) { + BulkProcessor.Builder builder = + BulkProcessor.builder( + new BulkRequestConsumerFactory() { // This cannot be inlined as a + // lambda because then + // deserialization fails + @Override + public void accept( + BulkRequest bulkRequest, + ActionListener + bulkResponseActionListener) { + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener); + } + }, + listener); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue( + bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval( + new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + backoffPolicy = + BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + break; + case NONE: + backoffPolicy = BackoffPolicy.noBackoff(); + break; + default: + throw new IllegalArgumentException( + "Received unknown backoff policy type " + + bulkProcessorConfig.getFlushBackoffType()); + } + builder.setBackoffPolicy(backoffPolicy); + return builder; + } + }; + } + + /** + * Constructs the {@link OpensearchSink} with the properties configured this builder. + * + * @return {@link OpensearchSink} + */ + public OpensearchSink build() { + checkNotNull(emitter); + checkNotNull(hosts); + + NetworkClientConfig networkClientConfig = buildNetworkClientConfig(); + BulkProcessorConfig bulkProcessorConfig = buildBulkProcessorConfig(); + + BulkProcessorBuilderFactory bulkProcessorBuilderFactory = getBulkProcessorBuilderFactory(); + ClosureCleaner.clean( + bulkProcessorBuilderFactory, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true); + + return new OpensearchSink<>( + hosts, + emitter, + deliveryGuarantee, + bulkProcessorBuilderFactory, + bulkProcessorConfig, + networkClientConfig); + } + + private NetworkClientConfig buildNetworkClientConfig() { + checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); + + return new NetworkClientConfig( + username, + password, + connectionPathPrefix, + connectionRequestTimeout, + connectionTimeout, + socketTimeout, + allowInsecure); + } + + private BulkProcessorConfig buildBulkProcessorConfig() { + return new BulkProcessorConfig( + bulkFlushMaxActions, + bulkFlushMaxMb, + bulkFlushInterval, + bulkFlushBackoffType, + bulkFlushBackoffRetries, + bulkFlushBackOffDelay); + } + + @Override + public String toString() { + return "OpensearchSinkBuilder{" + + "bulkFlushMaxActions=" + + bulkFlushMaxActions + + ", bulkFlushMaxMb=" + + bulkFlushMaxMb + + ", bulkFlushInterval=" + + bulkFlushInterval + + ", bulkFlushBackoffType=" + + bulkFlushBackoffType + + ", bulkFlushBackoffRetries=" + + bulkFlushBackoffRetries + + ", bulkFlushBackOffDelay=" + + bulkFlushBackOffDelay + + ", deliveryGuarantee=" + + deliveryGuarantee + + ", hosts=" + + hosts + + ", emitter=" + + emitter + + ", username='" + + username + + '\'' + + ", password='" + + password + + '\'' + + ", connectionPathPrefix='" + + connectionPathPrefix + + '\'' + + ", allowInsecure='" + + allowInsecure + + '\'' + + '}'; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchWriter.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchWriter.java new file mode 100644 index 00000000000000..0935634163ee08 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchWriter.java @@ -0,0 +1,350 @@ +/* + * 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.VisibleForTesting; +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.conn.ssl.TrustAllStrategy; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.http.ssl.SSLContexts; +import org.opensearch.action.DocWriteRequest; +import org.opensearch.action.bulk.BulkItemResponse; +import org.opensearch.action.bulk.BulkProcessor; +import org.opensearch.action.bulk.BulkRequest; +import org.opensearch.action.bulk.BulkResponse; +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; +import org.opensearch.client.RestClient; +import org.opensearch.client.RestClientBuilder; +import org.opensearch.client.RestHighLevelClient; +import org.opensearch.rest.RestStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.KeyManagementException; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed; +import static org.apache.flink.util.Preconditions.checkNotNull; + +class OpensearchWriter implements SinkWriter { + + private static final Logger LOG = LoggerFactory.getLogger(OpensearchWriter.class); + + private final OpensearchEmitter emitter; + private final MailboxExecutor mailboxExecutor; + private final boolean flushOnCheckpoint; + private final BulkProcessor bulkProcessor; + private final RestHighLevelClient client; + private final RequestIndexer requestIndexer; + private final Counter numBytesOutCounter; + + private long pendingActions = 0; + private boolean checkpointInProgress = false; + private volatile long lastSendTime = 0; + private volatile long ackTime = Long.MAX_VALUE; + private volatile boolean closed = false; + + /** + * Constructor creating an Opensearch writer. + * + * @param hosts the reachable Opensearch cluster nodes + * @param emitter converting incoming records to Opensearch actions + * @param flushOnCheckpoint if true all until now received records are flushed after every + * checkpoint + * @param bulkProcessorConfig describing the flushing and failure handling of the used {@link + * BulkProcessor} + * @param bulkProcessorBuilderFactory configuring the {@link BulkProcessor}'s builder + * @param networkClientConfig describing properties of the network connection used to connect to + * the Opensearch cluster + * @param metricGroup for the sink writer + * @param mailboxExecutor Flink's mailbox executor + */ + OpensearchWriter( + List hosts, + OpensearchEmitter emitter, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + BulkProcessorBuilderFactory bulkProcessorBuilderFactory, + NetworkClientConfig networkClientConfig, + SinkWriterMetricGroup metricGroup, + MailboxExecutor mailboxExecutor) { + this.emitter = checkNotNull(emitter); + this.flushOnCheckpoint = flushOnCheckpoint; + this.mailboxExecutor = checkNotNull(mailboxExecutor); + this.client = + new RestHighLevelClient( + configureRestClientBuilder( + RestClient.builder(hosts.toArray(new HttpHost[0])), + networkClientConfig)); + this.bulkProcessor = createBulkProcessor(bulkProcessorBuilderFactory, bulkProcessorConfig); + this.requestIndexer = new DefaultRequestIndexer(); + checkNotNull(metricGroup); + metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime); + this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); + try { + emitter.open(); + } catch (Exception e) { + throw new FlinkRuntimeException("Failed to open the OpensearchEmitter", e); + } + } + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // do not allow new bulk writes until all actions are flushed + while (checkpointInProgress) { + mailboxExecutor.yield(); + } + emitter.emit(element, context, requestIndexer); + } + + @Override + public List prepareCommit(boolean flush) throws IOException, InterruptedException { + checkpointInProgress = true; + while (pendingActions != 0 && (flushOnCheckpoint || flush)) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + checkpointInProgress = false; + return Collections.emptyList(); + } + + @VisibleForTesting + void blockingFlushAllActions() throws InterruptedException { + while (pendingActions != 0) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + } + + @Override + public void close() throws Exception { + closed = true; + emitter.close(); + bulkProcessor.close(); + client.close(); + } + + private static RestClientBuilder configureRestClientBuilder( + RestClientBuilder builder, NetworkClientConfig networkClientConfig) { + if (networkClientConfig.getConnectionPathPrefix() != null) { + builder.setPathPrefix(networkClientConfig.getConnectionPathPrefix()); + } + + builder.setHttpClientConfigCallback( + httpClientBuilder -> { + if (networkClientConfig.getPassword() != null + && networkClientConfig.getUsername() != null) { + final CredentialsProvider credentialsProvider = + new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, + new UsernamePasswordCredentials( + networkClientConfig.getUsername(), + networkClientConfig.getPassword())); + + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + } + + if (networkClientConfig.isAllowInsecure() != null + && networkClientConfig.isAllowInsecure()) { + try { + httpClientBuilder.setSSLContext( + SSLContexts.custom() + .loadTrustMaterial(new TrustAllStrategy()) + .build()); + } catch (final NoSuchAlgorithmException + | KeyStoreException + | KeyManagementException ex) { + throw new IllegalStateException( + "Unable to create custom SSL context", ex); + } + } + + return httpClientBuilder; + }); + if (networkClientConfig.getConnectionRequestTimeout() != null + || networkClientConfig.getConnectionTimeout() != null + || networkClientConfig.getSocketTimeout() != null) { + builder.setRequestConfigCallback( + requestConfigBuilder -> { + if (networkClientConfig.getConnectionRequestTimeout() != null) { + requestConfigBuilder.setConnectionRequestTimeout( + networkClientConfig.getConnectionRequestTimeout()); + } + if (networkClientConfig.getConnectionTimeout() != null) { + requestConfigBuilder.setConnectTimeout( + networkClientConfig.getConnectionTimeout()); + } + if (networkClientConfig.getSocketTimeout() != null) { + requestConfigBuilder.setSocketTimeout( + networkClientConfig.getSocketTimeout()); + } + return requestConfigBuilder; + }); + } + return builder; + } + + private BulkProcessor createBulkProcessor( + BulkProcessorBuilderFactory bulkProcessorBuilderFactory, + BulkProcessorConfig bulkProcessorConfig) { + + BulkProcessor.Builder builder = + bulkProcessorBuilderFactory.apply(client, bulkProcessorConfig, new BulkListener()); + + // This makes flush() blocking + builder.setConcurrentRequests(0); + + return builder.build(); + } + + private class BulkListener implements BulkProcessor.Listener { + + @Override + public void beforeBulk(long executionId, BulkRequest request) { + LOG.info("Sending bulk of {} actions to Opensearch.", request.numberOfActions()); + lastSendTime = System.currentTimeMillis(); + numBytesOutCounter.inc(request.estimatedSizeInBytes()); + } + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + ackTime = System.currentTimeMillis(); + enqueueActionInMailbox( + () -> extractFailures(request, response), "opensearchSuccessCallback"); + } + + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + enqueueActionInMailbox( + () -> { + throw new FlinkRuntimeException("Complete bulk has failed.", failure); + }, + "opensearchErrorCallback"); + } + } + + private void enqueueActionInMailbox( + ThrowingRunnable action, String actionName) { + // If the writer is cancelled before the last bulk response (i.e. no flush on checkpoint + // configured or shutdown without a final + // checkpoint) the mailbox might already be shutdown, so we should not enqueue any + // actions. + if (isClosed()) { + return; + } + mailboxExecutor.execute(action, actionName); + } + + private void extractFailures(BulkRequest request, BulkResponse response) { + if (!response.hasFailures()) { + pendingActions -= request.numberOfActions(); + return; + } + + Throwable chainedFailures = null; + for (int i = 0; i < response.getItems().length; i++) { + final BulkItemResponse itemResponse = response.getItems()[i]; + if (!itemResponse.isFailed()) { + continue; + } + final Throwable failure = itemResponse.getFailure().getCause(); + if (failure == null) { + continue; + } + final RestStatus restStatus = itemResponse.getFailure().getStatus(); + final DocWriteRequest actionRequest = request.requests().get(i); + + chainedFailures = + firstOrSuppressed( + wrapException(restStatus, failure, actionRequest), chainedFailures); + } + if (chainedFailures == null) { + return; + } + throw new FlinkRuntimeException(chainedFailures); + } + + private static Throwable wrapException( + RestStatus restStatus, Throwable rootFailure, DocWriteRequest actionRequest) { + if (restStatus == null) { + return new FlinkRuntimeException( + String.format("Single action %s of bulk request failed.", actionRequest), + rootFailure); + } else { + return new FlinkRuntimeException( + String.format( + "Single action %s of bulk request failed with status %s.", + actionRequest, restStatus.getStatus()), + rootFailure); + } + } + + private boolean isClosed() { + if (closed) { + LOG.warn("Writer was closed before all records were acknowledged by Opensearch."); + } + return closed; + } + + private class DefaultRequestIndexer implements RequestIndexer { + + @Override + public void add(DeleteRequest... deleteRequests) { + for (final DeleteRequest deleteRequest : deleteRequests) { + pendingActions++; + bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (final IndexRequest indexRequest : indexRequests) { + pendingActions++; + bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (final UpdateRequest updateRequest : updateRequests) { + pendingActions++; + bulkProcessor.add(updateRequest); + } + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/RequestIndexer.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/RequestIndexer.java new file mode 100644 index 00000000000000..227a526e81b1fe --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/RequestIndexer.java @@ -0,0 +1,56 @@ +/* + * 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.Internal; + +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; + +/** + * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare them + * for sending to an Opensearch cluster. + */ +@Internal +public interface RequestIndexer { + /** + * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to + * Opensearch. + * + * @param deleteRequests The multiple {@link DeleteRequest} to add. + */ + void add(DeleteRequest... deleteRequests); + + /** + * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to + * Opensearch. + * + * @param indexRequests The multiple {@link IndexRequest} to add. + */ + void add(IndexRequest... indexRequests); + + /** + * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to + * Opensearch. + * + * @param updateRequests The multiple {@link UpdateRequest} to add. + */ + void add(UpdateRequest... updateRequests); +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/AbstractTimeIndexGenerator.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/AbstractTimeIndexGenerator.java new file mode 100644 index 00000000000000..80a7a8c2a6537e --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/AbstractTimeIndexGenerator.java @@ -0,0 +1,41 @@ +/* + * 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 java.time.format.DateTimeFormatter; + +/** Abstract class for time related {@link IndexGenerator}. */ +@Internal +abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase { + + private final String dateTimeFormat; + protected transient DateTimeFormatter dateTimeFormatter; + + public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { + super(index); + this.dateTimeFormat = dateTimeFormat; + } + + @Override + public void open() { + this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGenerator.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGenerator.java new file mode 100644 index 00000000000000..cd821fc59a8953 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGenerator.java @@ -0,0 +1,39 @@ +/* + * 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.table.data.RowData; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +/** This interface is responsible to generate index name from given {@link Row} record. */ +@Internal +interface IndexGenerator extends Serializable { + + /** + * Initialize the index generator, this will be called only once before {@link + * #generate(RowData)} is called. + */ + default void open() {} + + /** Generate index name according to the given row. */ + String generate(RowData row); +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorBase.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorBase.java new file mode 100644 index 00000000000000..e5298c1a31324e --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorBase.java @@ -0,0 +1,52 @@ +/* + * 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 java.util.Objects; + +/** Base class for {@link IndexGenerator}. */ +@Internal +public abstract class IndexGeneratorBase implements IndexGenerator { + + private static final long serialVersionUID = 1L; + protected final String index; + + public IndexGeneratorBase(String index) { + this.index = index; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IndexGeneratorBase)) { + return false; + } + IndexGeneratorBase that = (IndexGeneratorBase) o; + return index.equals(that.index); + } + + @Override + public int hashCode() { + return Objects.hash(index); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorFactory.java new file mode 100644 index 00000000000000..e87c41f77fdaea --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorFactory.java @@ -0,0 +1,278 @@ +/* + * 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.table.api.TableException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Factory of {@link IndexGenerator}. + * + *

Flink supports both static index and dynamic index. + * + *

If you want to have a static index, this option value should be a plain string, e.g. + * 'myusers', all the records will be consistently written into "myusers" index. + * + *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in + * the record to dynamically generate a target index. You can also use + * '{field_name|date_format_string}' to convert a field value of TIMESTAMP/DATE/TIME type into the + * format specified by date_format_string. The date_format_string is compatible with {@link + * java.text.SimpleDateFormat}. For example, if the option value is 'myusers_{log_ts|yyyy-MM-dd}', + * then a record with log_ts field value 2020-03-27 12:25:55 will be written into + * "myusers_2020-03-27" index. + */ +@Internal +final class IndexGeneratorFactory { + + private IndexGeneratorFactory() {} + + public static IndexGenerator createIndexGenerator( + String index, List fieldNames, List dataTypes) { + final IndexHelper indexHelper = new IndexHelper(); + if (indexHelper.checkIsDynamicIndex(index)) { + return createRuntimeIndexGenerator( + index, + fieldNames.toArray(new String[0]), + dataTypes.toArray(new DataType[0]), + indexHelper); + } else { + return new StaticIndexGenerator(index); + } + } + + interface DynamicFormatter extends Serializable { + String format(@Nonnull Object fieldValue, DateTimeFormatter formatter); + } + + private static IndexGenerator createRuntimeIndexGenerator( + String index, String[] fieldNames, DataType[] fieldTypes, IndexHelper indexHelper) { + final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); + final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); + final String indexSuffix = + index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); + + final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); + final int indexFieldPos = + indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); + final LogicalType indexFieldType = fieldTypes[indexFieldPos].getLogicalType(); + final LogicalTypeRoot indexFieldLogicalTypeRoot = indexFieldType.getTypeRoot(); + + // validate index field type + indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot); + + // time extract dynamic index pattern + final RowData.FieldGetter fieldGetter = + RowData.createFieldGetter(indexFieldType, indexFieldPos); + + if (isDynamicIndexWithFormat) { + final String dateTimeFormat = + indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); + DynamicFormatter formatFunction = + createFormatFunction(indexFieldType, indexFieldLogicalTypeRoot); + + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + Object fieldOrNull = fieldGetter.getFieldOrNull(row); + final String formattedField; + // TODO we can possibly optimize it to use the nullability of the field + if (fieldOrNull != null) { + formattedField = formatFunction.format(fieldOrNull, dateTimeFormatter); + } else { + formattedField = "null"; + } + return indexPrefix.concat(formattedField).concat(indexSuffix); + } + }; + } + // general dynamic index pattern + return new IndexGeneratorBase(index) { + @Override + public String generate(RowData row) { + Object indexField = fieldGetter.getFieldOrNull(row); + return indexPrefix + .concat(indexField == null ? "null" : indexField.toString()) + .concat(indexSuffix); + } + }; + } + + private static DynamicFormatter createFormatFunction( + LogicalType indexFieldType, LogicalTypeRoot indexFieldLogicalTypeRoot) { + switch (indexFieldLogicalTypeRoot) { + case DATE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalDate.ofEpochDay(indexField).format(dateTimeFormatter); + }; + case TIME_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalTime.ofNanoOfDay(indexField * 1_000_000L).format(dateTimeFormatter); + }; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toLocalDateTime().format(dateTimeFormatter); + }; + case TIMESTAMP_WITH_TIME_ZONE: + throw new UnsupportedOperationException( + "TIMESTAMP_WITH_TIME_ZONE is not supported yet"); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter); + }; + default: + throw new TableException( + String.format( + "Unsupported type '%s' found in Opensearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", + indexFieldType)); + } + } + + /** + * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field + * type ans parse index format from pattern. + */ + private static class IndexHelper { + private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); + private static final Pattern dynamicIndexTimeExtractPattern = + Pattern.compile(".*\\{.+\\|.*\\}.*"); + private static final List supportedTypes = new ArrayList<>(); + private static final Map defaultFormats = new HashMap<>(); + + static { + // time related types + supportedTypes.add(LogicalTypeRoot.DATE); + supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + // general types + supportedTypes.add(LogicalTypeRoot.VARCHAR); + supportedTypes.add(LogicalTypeRoot.CHAR); + supportedTypes.add(LogicalTypeRoot.TINYINT); + supportedTypes.add(LogicalTypeRoot.INTEGER); + supportedTypes.add(LogicalTypeRoot.BIGINT); + } + + static { + defaultFormats.put(LogicalTypeRoot.DATE, "yyyy_MM_dd"); + defaultFormats.put(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE, "HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put( + LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE, "yyyy_MM_dd_HH_mm_ssX"); + } + + /** Validate the index field Type. */ + void validateIndexFieldType(LogicalTypeRoot logicalType) { + if (!supportedTypes.contains(logicalType)) { + throw new IllegalArgumentException( + String.format( + "Unsupported type %s of index field, " + "Supported types are: %s", + logicalType, supportedTypes)); + } + } + + /** Get the default date format. */ + String getDefaultFormat(LogicalTypeRoot logicalType) { + return defaultFormats.get(logicalType); + } + + /** Check general dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndex(String index) { + final Matcher matcher = dynamicIndexPattern.matcher(index); + int count = 0; + while (matcher.find()) { + count++; + } + if (count > 1) { + throw new TableException( + String.format( + "Chaining dynamic index pattern %s is not supported," + + " only support single dynamic index pattern.", + index)); + } + return count == 1; + } + + /** Check time extract dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndexWithFormat(String index) { + return dynamicIndexTimeExtractPattern.matcher(index).matches(); + } + + /** Extract dynamic index pattern string from index pattern string. */ + String extractDynamicIndexPatternStr(String index) { + int start = index.indexOf("{"); + int end = index.lastIndexOf("}"); + return index.substring(start, end + 1); + } + + /** Extract index field position in a fieldNames, return the field position. */ + int extractIndexFieldPos( + String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { + List fieldList = Arrays.asList(fieldNames); + String indexFieldName; + if (isDynamicIndexWithFormat) { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); + } else { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); + } + if (!fieldList.contains(indexFieldName)) { + throw new TableException( + String.format( + "Unknown field '%s' in index pattern '%s', please check the field name.", + indexFieldName, index)); + } + return fieldList.indexOf(indexFieldName); + } + + /** Extract dateTime format by the date format that extracted from index pattern string. */ + private String extractDateFormat(String index, LogicalTypeRoot logicalType) { + String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); + if ("".equals(format)) { + format = getDefaultFormat(logicalType); + } + return format; + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/KeyExtractor.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/KeyExtractor.java new file mode 100644 index 00000000000000..20e1013f53ff52 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/KeyExtractor.java @@ -0,0 +1,97 @@ +/* + * 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.table.data.RowData; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.function.SerializableFunction; + +import java.io.Serializable; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.Period; +import java.util.List; + +/** An extractor for a Opensearch key from a {@link RowData}. */ +@Internal +class KeyExtractor implements SerializableFunction { + private final FieldFormatter[] fieldFormatters; + private final String keyDelimiter; + + private interface FieldFormatter extends Serializable { + String format(RowData rowData); + } + + private KeyExtractor(FieldFormatter[] fieldFormatters, String keyDelimiter) { + this.fieldFormatters = fieldFormatters; + this.keyDelimiter = keyDelimiter; + } + + @Override + public String apply(RowData rowData) { + final StringBuilder builder = new StringBuilder(); + for (int i = 0; i < fieldFormatters.length; i++) { + if (i > 0) { + builder.append(keyDelimiter); + } + final String value = fieldFormatters[i].format(rowData); + builder.append(value); + } + return builder.toString(); + } + + public static SerializableFunction createKeyExtractor( + List primaryKeyTypesWithIndex, String keyDelimiter) { + if (!primaryKeyTypesWithIndex.isEmpty()) { + FieldFormatter[] formatters = + primaryKeyTypesWithIndex.stream() + .map( + logicalTypeWithIndex -> + toFormatter( + logicalTypeWithIndex.index, + logicalTypeWithIndex.logicalType)) + .toArray(FieldFormatter[]::new); + return new KeyExtractor(formatters, keyDelimiter); + } else { + return (row) -> null; + } + } + + private static FieldFormatter toFormatter(int index, LogicalType type) { + switch (type.getTypeRoot()) { + case DATE: + return (row) -> LocalDate.ofEpochDay(row.getInt(index)).toString(); + case TIME_WITHOUT_TIME_ZONE: + return (row) -> + LocalTime.ofNanoOfDay((long) row.getInt(index) * 1_000_000L).toString(); + case INTERVAL_YEAR_MONTH: + return (row) -> Period.ofDays(row.getInt(index)).toString(); + case INTERVAL_DAY_TIME: + return (row) -> Duration.ofMillis(row.getLong(index)).toString(); + case DISTINCT_TYPE: + return toFormatter(index, ((DistinctType) type).getSourceType()); + default: + RowData.FieldGetter fieldGetter = RowData.createFieldGetter(type, index); + return (row) -> fieldGetter.getFieldOrNull(row).toString(); + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/LogicalTypeWithIndex.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/LogicalTypeWithIndex.java new file mode 100644 index 00000000000000..e5fe9a6ebbc4d2 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/LogicalTypeWithIndex.java @@ -0,0 +1,31 @@ +/* + * 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.table.types.logical.LogicalType; + +class LogicalTypeWithIndex { + public final int index; + public final LogicalType logicalType; + + LogicalTypeWithIndex(int index, LogicalType logicalType) { + this.index = index; + this.logicalType = logicalType; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java new file mode 100644 index 00000000000000..a6f6dc60d8051e --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java @@ -0,0 +1,163 @@ +/* + * 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.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 getUsername() { + return config.getOptional(USERNAME_OPTION); + } + + public Optional getPassword() { + return config.getOptional(PASSWORD_OPTION); + } + + public Optional getBulkFlushBackoffType() { + return config.getOptional(BULK_FLUSH_BACKOFF_TYPE_OPTION); + } + + public Optional getBulkFlushBackoffRetries() { + return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); + } + + public Optional getBulkFlushBackoffDelay() { + return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); + } + + public String getIndex() { + return config.get(INDEX_OPTION); + } + + public String getKeyDelimiter() { + return config.get(KEY_DELIMITER_OPTION); + } + + public Optional getPathPrefix() { + return config.getOptional(CONNECTION_PATH_PREFIX_OPTION); + } + + public Optional getConnectionRequestTimeout() { + return config.getOptional(CONNECTION_REQUEST_TIMEOUT); + } + + public Optional getConnectionTimeout() { + return config.getOptional(CONNECTION_TIMEOUT); + } + + public Optional getSocketTimeout() { + return config.getOptional(SOCKET_TIMEOUT); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(OpensearchConfiguration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + public Optional getParallelism() { + return config.getOptional(SINK_PARALLELISM); + } + + public Optional isAllowInsecure() { + return config.getOptional(ALLOW_INSECURE); + } + + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.java new file mode 100644 index 00000000000000..dcd87cb7d232a6 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.java @@ -0,0 +1,155 @@ +/* + * 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> HOSTS_OPTION = + ConfigOptions.key("hosts") + .stringType() + .asList() + .noDefaultValue() + .withDescription("Opensearch hosts to connect to."); + + public static final ConfigOption INDEX_OPTION = + ConfigOptions.key("index") + .stringType() + .noDefaultValue() + .withDescription("Opensearch index for every record."); + + public static final ConfigOption PASSWORD_OPTION = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("Password used to connect to Opensearch instance."); + + public static final ConfigOption USERNAME_OPTION = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("Username used to connect to Opensearch instance."); + + public static final ConfigOption 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 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 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 BULK_FLUSH_INTERVAL_OPTION = + ConfigOptions.key("sink.bulk-flush.interval") + .durationType() + .defaultValue(Duration.ofSeconds(1)) + .withDescription("Bulk flush interval"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.strategy") + .enumType(FlushBackoffType.class) + .noDefaultValue() + .withDescription("Backoff strategy"); + + public static final ConfigOption 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 BULK_FLUSH_BACKOFF_DELAY_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.delay") + .durationType() + .noDefaultValue() + .withDescription("Delay between each backoff attempt."); + + public static final ConfigOption CONNECTION_PATH_PREFIX_OPTION = + ConfigOptions.key("connection.path-prefix") + .stringType() + .noDefaultValue() + .withDescription("Prefix string to be added to every REST communication."); + + public static final ConfigOption CONNECTION_REQUEST_TIMEOUT = + ConfigOptions.key("connection.request-timeout") + .durationType() + .noDefaultValue() + .withDescription( + "The timeout for requesting a connection from the connection manager."); + + public static final ConfigOption CONNECTION_TIMEOUT = + ConfigOptions.key("connection.timeout") + .durationType() + .noDefaultValue() + .withDescription("The timeout for establishing a connection."); + + public static final ConfigOption SOCKET_TIMEOUT = + ConfigOptions.key("socket.timeout") + .durationType() + .noDefaultValue() + .withDescription( + "The socket timeout (SO_TIMEOUT) for waiting for data or, put differently," + + "a maximum period inactivity between two consecutive data packets."); + + public static final ConfigOption FORMAT_OPTION = + ConfigOptions.key("format") + .stringType() + .defaultValue("json") + .withDescription( + "The format must produce a valid JSON document. " + + "Please refer to the documentation on formats for more details."); + + public static final ConfigOption DELIVERY_GUARANTEE_OPTION = + ConfigOptions.key("sink.delivery-guarantee") + .enumType(DeliveryGuarantee.class) + .defaultValue(DeliveryGuarantee.NONE) + .withDescription("Optional delivery guarantee when committing."); + + public static final ConfigOption ALLOW_INSECURE = + ConfigOptions.key("allow-insecure") + .booleanType() + .defaultValue(false) + .withDescription( + "Allow insecure connections to HTTPS endpoints (disable certificates validation)"); +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSink.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSink.java new file mode 100644 index 00000000000000..cc60b167521f6f --- /dev/null +++ b/flink-connectors/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.SinkProvider; +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> format; + final DataType physicalRowDataType; + final List primaryKeyLogicalTypesWithIndex; + final OpensearchConfiguration config; + + final String summaryString; + final OpensearchSinkBuilderSupplier builderSupplier; + + OpensearchDynamicSink( + EncodingFormat> format, + OpensearchConfiguration config, + List primaryKeyLogicalTypesWithIndex, + DataType physicalRowDataType, + String summaryString, + OpensearchSinkBuilderSupplier builderSupplier) { + this.format = checkNotNull(format); + this.physicalRowDataType = checkNotNull(physicalRowDataType); + this.primaryKeyLogicalTypesWithIndex = checkNotNull(primaryKeyLogicalTypesWithIndex); + this.config = checkNotNull(config); + this.summaryString = checkNotNull(summaryString); + this.builderSupplier = checkNotNull(builderSupplier); + } + + Function createKeyExtractor() { + return KeyExtractor.createKeyExtractor( + primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter()); + } + + IndexGenerator createIndexGenerator() { + return IndexGeneratorFactory.createIndexGenerator( + config.getIndex(), + DataType.getFieldNames(physicalRowDataType), + DataType.getFieldDataTypes(physicalRowDataType)); + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + return builder.build(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + SerializationSchema format = + this.format.createRuntimeEncoder(context, physicalRowDataType); + + final RowOpensearchEmitter rowOpensearchEmitter = + new RowOpensearchEmitter( + createIndexGenerator(), format, XContentType.JSON, createKeyExtractor()); + + OpensearchSinkBuilder builder = builderSupplier.get(); + builder.setEmitter(rowOpensearchEmitter); + builder.setHosts(config.getHosts().toArray(new HttpHost[0])); + builder.setDeliveryGuarantee(config.getDeliveryGuarantee()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb(config.getBulkFlushMaxByteSize().getMebiBytes()); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + + if (config.getBulkFlushBackoffType().isPresent()) { + FlushBackoffType backoffType = config.getBulkFlushBackoffType().get(); + int backoffMaxRetries = config.getBulkFlushBackoffRetries().get(); + long backoffDelayMs = config.getBulkFlushBackoffDelay().get(); + + builder.setBulkFlushBackoffStrategy(backoffType, backoffMaxRetries, backoffDelayMs); + } + + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + builder.setConnectionUsername(config.getUsername().get()); + } + + if (config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setConnectionPassword(config.getPassword().get()); + } + + if (config.getPathPrefix().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPathPrefix().get())) { + builder.setConnectionPathPrefix(config.getPathPrefix().get()); + } + + if (config.getConnectionRequestTimeout().isPresent()) { + builder.setConnectionRequestTimeout( + (int) config.getConnectionRequestTimeout().get().getSeconds()); + } + + if (config.getConnectionTimeout().isPresent()) { + builder.setConnectionTimeout((int) config.getConnectionTimeout().get().getSeconds()); + } + + if (config.getSocketTimeout().isPresent()) { + builder.setSocketTimeout((int) config.getSocketTimeout().get().getSeconds()); + } + + if (config.isAllowInsecure().isPresent()) { + builder.setAllowInsecure(config.isAllowInsecure().get()); + } + + return SinkProvider.of(builder.build(), config.getParallelism().orElse(null)); + } + + @Override + public DynamicTableSink copy() { + return new OpensearchDynamicSink( + format, + config, + primaryKeyLogicalTypesWithIndex, + physicalRowDataType, + summaryString, + builderSupplier); + } + + @Override + public String asSummaryString() { + return summaryString; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OpensearchDynamicSink that = (OpensearchDynamicSink) o; + return Objects.equals(format, that.format) + && Objects.equals(physicalRowDataType, that.physicalRowDataType) + && Objects.equals( + primaryKeyLogicalTypesWithIndex, that.primaryKeyLogicalTypesWithIndex) + && Objects.equals(config, that.config) + && Objects.equals(summaryString, that.summaryString) + && Objects.equals(builderSupplier, that.builderSupplier); + } + + @Override + public int hashCode() { + return Objects.hash( + format, + physicalRowDataType, + primaryKeyLogicalTypesWithIndex, + config, + summaryString, + builderSupplier); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactory.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactory.java new file mode 100644 index 00000000000000..d02dce9dfbf77c --- /dev/null +++ b/flink-connectors/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 sinkBuilderSupplier; + + public OpensearchDynamicSinkFactory() { + this.sinkBuilderSupplier = OpensearchSinkBuilder::new; + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + List primaryKeyLogicalTypesWithIndex = + getPrimaryKeyLogicalTypesWithIndex(context); + EncodingFormat> 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 message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + EncodingFormat> getValidatedEncodingFormat( + DynamicTableFactory factory, DynamicTableFactory.Context context) { + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(factory, context); + final EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + helper.validate(); + return format; + } + + List 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 = 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> requiredOptions() { + return Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); + } + + @Override + public Set> optionalOptions() { + return Stream.of( + KEY_DELIMITER_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_PATH_PREFIX_OPTION, + CONNECTION_REQUEST_TIMEOUT, + CONNECTION_TIMEOUT, + SOCKET_TIMEOUT, + FORMAT_OPTION, + DELIVERY_GUARANTEE_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION, + SINK_PARALLELISM, + ALLOW_INSECURE) + .collect(Collectors.toSet()); + } + + @Override + public String factoryIdentifier() { + return FACTORY_IDENTIFIER; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchSinkBuilderSupplier.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchSinkBuilderSupplier.java new file mode 100644 index 00000000000000..80fc30ff28206f --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchSinkBuilderSupplier.java @@ -0,0 +1,25 @@ +/* + * 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.connector.opensearch.sink.OpensearchSinkBuilder; + +import java.util.function.Supplier; + +interface OpensearchSinkBuilderSupplier extends Supplier> {} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchValidationUtils.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchValidationUtils.java new file mode 100644 index 00000000000000..e9ea7e286f2804 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchValidationUtils.java @@ -0,0 +1,92 @@ +/* + * 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.table.api.ValidationException; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.DistinctType; +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; + +/** Utility methods for validating Opensearch properties. */ +@Internal +class OpensearchValidationUtils { + private static final Set ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); + + static { + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DATE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME); + } + + /** + * Checks that the table does not have a primary key defined on illegal types. In Opensearch the + * primary key is used to calculate the Opensearch 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(DataType primaryKeyDataType) { + List fieldDataTypes = DataType.getFieldDataTypes(primaryKeyDataType); + List illegalTypes = + fieldDataTypes.stream() + .map(DataType::getLogicalType) + .map( + logicalType -> { + if (logicalType.is(LogicalTypeRoot.DISTINCT_TYPE)) { + return ((DistinctType) logicalType) + .getSourceType() + .getTypeRoot(); + } else { + return logicalType.getTypeRoot(); + } + }) + .filter(t -> !ALLOWED_PRIMARY_KEY_TYPES.contains(t)) + .collect(Collectors.toList()); + if (!illegalTypes.isEmpty()) { + throw new ValidationException( + String.format( + "The table has a primary key on columns of illegal types: %s.", + illegalTypes)); + } + } + + private OpensearchValidationUtils() {} +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/RowOpensearchEmitter.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/RowOpensearchEmitter.java new file mode 100644 index 00000000000000..f6d14286ee4beb --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/RowOpensearchEmitter.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.opensearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.opensearch.sink.OpensearchEmitter; +import org.apache.flink.connector.opensearch.sink.RequestIndexer; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; + +import org.opensearch.action.ActionRequest; +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; +import org.opensearch.common.xcontent.XContentType; + +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Sink function for converting upserts into Opensearch {@link ActionRequest}s. */ +class RowOpensearchEmitter implements OpensearchEmitter { + + private final IndexGenerator indexGenerator; + private final SerializationSchema serializationSchema; + private final XContentType contentType; + private final Function createKey; + + public RowOpensearchEmitter( + IndexGenerator indexGenerator, + SerializationSchema serializationSchema, + XContentType contentType, + Function createKey) { + this.indexGenerator = checkNotNull(indexGenerator); + this.serializationSchema = checkNotNull(serializationSchema); + this.contentType = checkNotNull(contentType); + this.createKey = checkNotNull(createKey); + } + + @Override + public void open() { + indexGenerator.open(); + } + + @Override + public void emit(RowData element, SinkWriter.Context context, RequestIndexer indexer) { + switch (element.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + processUpsert(element, indexer); + break; + case UPDATE_BEFORE: + case DELETE: + processDelete(element, indexer); + break; + default: + throw new TableException("Unsupported message kind: " + element.getRowKind()); + } + } + + private void processUpsert(RowData row, RequestIndexer indexer) { + final byte[] document = serializationSchema.serialize(row); + final String key = createKey.apply(row); + if (key != null) { + final UpdateRequest updateRequest = + new UpdateRequest(indexGenerator.generate(row), key) + .doc(document, contentType) + .upsert(document, contentType); + indexer.add(updateRequest); + } else { + final IndexRequest indexRequest = + new IndexRequest(indexGenerator.generate(row)) + .id(key) + .source(document, contentType); + indexer.add(indexRequest); + } + } + + private void processDelete(RowData row, RequestIndexer indexer) { + final String key = createKey.apply(row); + final DeleteRequest deleteRequest = new DeleteRequest(indexGenerator.generate(row), key); + indexer.add(deleteRequest); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/StaticIndexGenerator.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/StaticIndexGenerator.java new file mode 100644 index 00000000000000..768dd84cd9368f --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/StaticIndexGenerator.java @@ -0,0 +1,35 @@ +/* + * 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.table.data.RowData; + +/** A static {@link IndexGenerator} which generate fixed index name. */ +@Internal +final class StaticIndexGenerator extends IndexGeneratorBase { + + public StaticIndexGenerator(String index) { + super(index); + } + + public String generate(RowData row) { + return index; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/ActionRequestFailureHandler.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/ActionRequestFailureHandler.java new file mode 100644 index 00000000000000..2473bf94f2bfa0 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/ActionRequestFailureHandler.java @@ -0,0 +1,77 @@ +/* + * 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.opensearch; + +import org.apache.flink.annotation.PublicEvolving; + +import org.opensearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * An implementation of {@link ActionRequestFailureHandler} is provided by the user to define how + * failed {@link ActionRequest ActionRequests} should be handled, e.g. dropping them, reprocessing + * malformed documents, or simply requesting them to be sent to Opensearch again if the failure is + * only temporary. + * + *

Example: + * + *

{@code
+ * private static class ExampleActionRequestFailureHandler implements ActionRequestFailureHandler {
+ *
+ * 	@Override
+ * 	void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable {
+ * 		if (ExceptionUtils.findThrowable(failure, OpenSearchRejectedExecutionException.class).isPresent()) {
+ * 			// full queue; re-add document for indexing
+ * 			indexer.add(action);
+ * 		} else if (ExceptionUtils.findThrowable(failure, OpensearchParseException.class).isPresent()) {
+ * 			// malformed document; simply drop request without failing sink
+ * 		} else {
+ * 			// for all other failures, fail the sink;
+ * 			// here the failure is simply rethrown, but users can also choose to throw custom exceptions
+ * 			throw failure;
+ * 		}
+ * 	}
+ * }
+ *
+ * }
+ * + *

The above example will let the sink re-add requests that failed due to queue capacity + * saturation and drop requests with malformed documents, without failing the sink. For all other + * failures, the sink will fail. + * + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public interface ActionRequestFailureHandler extends Serializable { + + /** + * Handle a failed {@link ActionRequest}. + * + * @param action the {@link ActionRequest} that failed due to the failure + * @param failure the cause of failure + * @param restStatusCode the REST status code of the failure (-1 if none can be retrieved) + * @param indexer request indexer to re-add the failed action, if intended to do so + * @throws Throwable if the sink should fail on this failure, the implementation should rethrow + * the exception or a custom one + */ + void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable; +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/BufferingNoOpRequestIndexer.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/BufferingNoOpRequestIndexer.java new file mode 100644 index 00000000000000..c0d93c1bd7d4c2 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/BufferingNoOpRequestIndexer.java @@ -0,0 +1,75 @@ +/* + * 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.opensearch; + +import org.apache.flink.annotation.Internal; + +import org.opensearch.action.ActionRequest; +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.util.Collections; +import java.util.concurrent.ConcurrentLinkedQueue; + +/** + * Implementation of a {@link RequestIndexer} that buffers {@link ActionRequest ActionRequests} + * before re-sending them to the Opensearch cluster upon request. + */ +@Internal +@NotThreadSafe +class BufferingNoOpRequestIndexer implements RequestIndexer { + + private ConcurrentLinkedQueue bufferedRequests; + + BufferingNoOpRequestIndexer() { + this.bufferedRequests = new ConcurrentLinkedQueue(); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + Collections.addAll(bufferedRequests, deleteRequests); + } + + @Override + public void add(IndexRequest... indexRequests) { + Collections.addAll(bufferedRequests, indexRequests); + } + + @Override + public void add(UpdateRequest... updateRequests) { + Collections.addAll(bufferedRequests, updateRequests); + } + + void processBufferedRequests(RequestIndexer actualIndexer) { + for (ActionRequest request : bufferedRequests) { + if (request instanceof IndexRequest) { + actualIndexer.add((IndexRequest) request); + } else if (request instanceof DeleteRequest) { + actualIndexer.add((DeleteRequest) request); + } else if (request instanceof UpdateRequest) { + actualIndexer.add((UpdateRequest) request); + } + } + + bufferedRequests.clear(); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchBulkProcessorIndexer.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchBulkProcessorIndexer.java new file mode 100644 index 00000000000000..f60eb5967d4506 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchBulkProcessorIndexer.java @@ -0,0 +1,84 @@ +/* + * 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.opensearch; + +import org.apache.flink.annotation.Internal; + +import org.opensearch.action.ActionRequest; +import org.opensearch.action.bulk.BulkProcessor; +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; + +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. {@link ActionRequest + * ActionRequests} will be buffered before sending a bulk request to the Opensearch cluster. + * + *

Note: This class is binary compatible to Opensearch 1.x. + */ +@Internal +class OpensearchBulkProcessorIndexer implements RequestIndexer { + + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; + + OpensearchBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSink.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSink.java new file mode 100644 index 00000000000000..aae3623b6cae6f --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSink.java @@ -0,0 +1,807 @@ +/* + * 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.opensearch; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.connectors.opensearch.util.NoOpFailureHandler; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.opensearch.action.ActionRequest; +import org.opensearch.action.DocWriteRequest; +import org.opensearch.action.bulk.BackoffPolicy; +import org.opensearch.action.bulk.BulkItemResponse; +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.RestClient; +import org.opensearch.client.RestClientBuilder; +import org.opensearch.client.RestHighLevelClient; +import org.opensearch.common.unit.ByteSizeUnit; +import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.rest.RestStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Base class for all Flink Opensearch Sinks. + * + *

This class implements the common behaviour across Opensearch versions, such as the use of an + * internal {@link BulkProcessor} to buffer multiple {@link ActionRequest}s before sending the + * requests to the cluster, as well as passing input records to the user provided {@link + * OpensearchSinkFunction} for processing. + * + * @param Type of the elements handled by this sink + */ +@Internal +public class OpensearchSink extends RichSinkFunction implements CheckpointedFunction { + private static final long serialVersionUID = -1007596293618451942L; + private static final Logger LOG = LoggerFactory.getLogger(OpensearchSink.class); + + // ------------------------------------------------------------------------ + // Internal bulk processor configuration + // ------------------------------------------------------------------------ + + public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = "bulk.flush.max.actions"; + public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = "bulk.flush.max.size.mb"; + public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = "bulk.flush.interval.ms"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE = "bulk.flush.backoff.enable"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE = "bulk.flush.backoff.type"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES = "bulk.flush.backoff.retries"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY = "bulk.flush.backoff.delay"; + + /** Used to control whether the retry delay should increase exponentially or remain constant. */ + @PublicEvolving + public enum FlushBackoffType { + CONSTANT, + EXPONENTIAL + } + + /** + * Provides a backoff policy for bulk requests. Whenever a bulk request is rejected due to + * resource constraints (i.e. the client's internal thread pool is full), the backoff policy + * decides how long the bulk processor will wait before the operation is retried internally. + * + *

This is a proxy for version specific backoff policies. + */ + public static class BulkFlushBackoffPolicy implements Serializable { + + private static final long serialVersionUID = -6022851996101826049L; + + // the default values follow the Opensearch default settings for BulkProcessor + private FlushBackoffType backoffType = FlushBackoffType.EXPONENTIAL; + private int maxRetryCount = 8; + private long delayMillis = 50; + + public FlushBackoffType getBackoffType() { + return backoffType; + } + + public int getMaxRetryCount() { + return maxRetryCount; + } + + public long getDelayMillis() { + return delayMillis; + } + + public void setBackoffType(FlushBackoffType backoffType) { + this.backoffType = checkNotNull(backoffType); + } + + public void setMaxRetryCount(int maxRetryCount) { + checkArgument(maxRetryCount >= 0); + this.maxRetryCount = maxRetryCount; + } + + public void setDelayMillis(long delayMillis) { + checkArgument(delayMillis >= 0); + this.delayMillis = delayMillis; + } + } + + private final Integer bulkProcessorFlushMaxActions; + private final Integer bulkProcessorFlushMaxSizeMb; + private final Long bulkProcessorFlushIntervalMillis; + private final BulkFlushBackoffPolicy bulkProcessorFlushBackoffPolicy; + + // ------------------------------------------------------------------------ + // User-facing API and configuration + // ------------------------------------------------------------------------ + + /** + * The config map that contains configuration for the bulk flushing behaviours. + * + *

For {@link org.opensearch.client.transport.TransportClient} based implementations, this + * config map would also contain Opensearch-shipped configuration, and therefore this config map + * would also be forwarded when creating the Opensearch client. + */ + private final Map userConfig; + + /** + * The function that is used to construct multiple {@link ActionRequest ActionRequests} from + * each incoming element. + */ + private final OpensearchSinkFunction opensearchSinkFunction; + + /** User-provided handler for failed {@link ActionRequest ActionRequests}. */ + private final ActionRequestFailureHandler failureHandler; + + /** + * If true, the producer will wait until all outstanding action requests have been sent to + * Opensearch. + */ + private boolean flushOnCheckpoint = true; + + /** + * Provided to the user via the {@link OpensearchSinkFunction} to add {@link ActionRequest + * ActionRequests}. + */ + private transient RequestIndexer requestIndexer; + + /** + * Provided to the {@link ActionRequestFailureHandler} to allow users to re-index failed + * requests. + */ + private transient BufferingNoOpRequestIndexer failureRequestIndexer; + + // ------------------------------------------------------------------------ + // Internals for the Flink Opensearch Sink + // ------------------------------------------------------------------------ + + /** Opensearch client instance. */ + private transient RestHighLevelClient client; + + /** + * Number of pending action requests not yet acknowledged by Opensearch. This value is + * maintained only if {@link OpensearchSinkBase#flushOnCheckpoint} is {@code true}. + * + *

This is incremented whenever the user adds (or re-adds through the {@link + * ActionRequestFailureHandler}) requests to the {@link RequestIndexer}. It is decremented for + * each completed request of a bulk request, in {@link BulkProcessor.Listener#afterBulk(long, + * BulkRequest, BulkResponse)} and {@link BulkProcessor.Listener#afterBulk(long, BulkRequest, + * Throwable)}. + */ + private AtomicLong numPendingRequests = new AtomicLong(0); + + /** User-provided HTTP Host. */ + private final List httpHosts; + + /** The factory to configure the rest client. */ + private final RestClientFactory restClientFactory; + + /** Bulk processor to buffer and send requests to Opensearch, created using the client. */ + private transient BulkProcessor bulkProcessor; + + /** + * This is set from inside the {@link BulkProcessor.Listener} if a {@link Throwable} was thrown + * in callbacks and the user considered it should fail the sink via the {@link + * ActionRequestFailureHandler#onFailure(ActionRequest, Throwable, int, RequestIndexer)} method. + * + *

Errors will be checked and rethrown before processing each input element, and when the + * sink is closed. + */ + private final AtomicReference failureThrowable = new AtomicReference<>(); + + private OpensearchSink( + Map userConfig, + List httpHosts, + OpensearchSinkFunction opensearchSinkFunction, + ActionRequestFailureHandler failureHandler, + RestClientFactory restClientFactory) { + checkArgument(httpHosts != null && !httpHosts.isEmpty()); + this.httpHosts = httpHosts; + this.restClientFactory = checkNotNull(restClientFactory); + this.opensearchSinkFunction = checkNotNull(opensearchSinkFunction); + this.failureHandler = checkNotNull(failureHandler); + // we eagerly check if the user-provided sink function and failure handler is serializable; + // otherwise, if they aren't serializable, users will merely get a non-informative error + // message + // "OpensearchSinkBase is not serializable" + + checkArgument( + InstantiationUtil.isSerializable(opensearchSinkFunction), + "The implementation of the provided OpensearchSinkFunction is not serializable. " + + "The object probably contains or references non-serializable fields."); + + checkArgument( + InstantiationUtil.isSerializable(failureHandler), + "The implementation of the provided ActionRequestFailureHandler is not serializable. " + + "The object probably contains or references non-serializable fields."); + + // extract and remove bulk processor related configuration from the user-provided config, + // so that the resulting user config only contains configuration related to the + // Opensearch client. + + checkNotNull(userConfig); + + // copy config so we can remove entries without side-effects + userConfig = new HashMap<>(userConfig); + + ParameterTool params = ParameterTool.fromMap(userConfig); + + if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) { + bulkProcessorFlushMaxActions = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS); + } else { + bulkProcessorFlushMaxActions = null; + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) { + bulkProcessorFlushMaxSizeMb = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB); + } else { + bulkProcessorFlushMaxSizeMb = null; + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) { + bulkProcessorFlushIntervalMillis = params.getLong(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); + } else { + bulkProcessorFlushIntervalMillis = null; + } + + boolean bulkProcessorFlushBackoffEnable = + params.getBoolean(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, true); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE); + + if (bulkProcessorFlushBackoffEnable) { + this.bulkProcessorFlushBackoffPolicy = new BulkFlushBackoffPolicy(); + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE)) { + bulkProcessorFlushBackoffPolicy.setBackoffType( + FlushBackoffType.valueOf(params.get(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE))); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE); + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)) { + bulkProcessorFlushBackoffPolicy.setMaxRetryCount( + params.getInt(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES); + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)) { + bulkProcessorFlushBackoffPolicy.setDelayMillis( + params.getLong(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY); + } + + } else { + bulkProcessorFlushBackoffPolicy = null; + } + + this.userConfig = userConfig; + } + + /** + * Disable flushing on checkpoint. When disabled, the sink will not wait for all pending action + * requests to be acknowledged by Opensearch on checkpoints. + * + *

NOTE: If flushing on checkpoint is disabled, the Flink Opensearch Sink does NOT provide + * any strong guarantees for at-least-once delivery of action requests. + */ + public void disableFlushOnCheckpoint() { + this.flushOnCheckpoint = false; + } + + @Override + public void open(Configuration parameters) throws Exception { + client = createClient(userConfig); + bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); + requestIndexer = + new OpensearchBulkProcessorIndexer( + bulkProcessor, flushOnCheckpoint, numPendingRequests); + failureRequestIndexer = new BufferingNoOpRequestIndexer(); + opensearchSinkFunction.open(); + } + + @Override + public void invoke(T value, Context context) throws Exception { + checkAsyncErrorsAndRequests(); + opensearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + // no initialization needed + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + checkAsyncErrorsAndRequests(); + + if (flushOnCheckpoint) { + while (numPendingRequests.get() != 0) { + bulkProcessor.flush(); + checkAsyncErrorsAndRequests(); + } + } + } + + @Override + public void close() throws Exception { + opensearchSinkFunction.close(); + if (bulkProcessor != null) { + bulkProcessor.close(); + bulkProcessor = null; + } + + if (client != null) { + client.close(); + client = null; + } + + // make sure any errors from callbacks are rethrown + checkErrorAndRethrow(); + } + + /** Build the {@link BulkProcessor}. */ + protected BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) { + checkNotNull(listener); + + BulkProcessor.Builder bulkProcessorBuilder = + BulkProcessor.builder( + (request, bulkListener) -> + client.bulkAsync(request, RequestOptions.DEFAULT, bulkListener), + listener); + + // This makes flush() blocking + bulkProcessorBuilder.setConcurrentRequests(0); + + if (bulkProcessorFlushMaxActions != null) { + bulkProcessorBuilder.setBulkActions(bulkProcessorFlushMaxActions); + } + + if (bulkProcessorFlushMaxSizeMb != null) { + configureBulkSize(bulkProcessorBuilder); + } + + if (bulkProcessorFlushIntervalMillis != null) { + configureFlushInterval(bulkProcessorBuilder); + } + + // if backoff retrying is disabled, bulkProcessorFlushBackoffPolicy will be null + configureBulkProcessorBackoff(bulkProcessorBuilder, bulkProcessorFlushBackoffPolicy); + + return bulkProcessorBuilder.build(); + } + + /** + * Creates an Opensearch client implementing {@link AutoCloseable}. + * + * @param clientConfig The configuration to use when constructing the client. + * @return The created client. + * @throws IOException IOException + */ + private RestHighLevelClient createClient(Map clientConfig) throws IOException { + RestClientBuilder builder = + RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); + restClientFactory.configureRestClientBuilder(builder); + + RestHighLevelClient rhlClient = new RestHighLevelClient(builder); + verifyClientConnection(rhlClient); + + return rhlClient; + } + + /** + * Verify the client connection by making a test request/ping to the Opensearch cluster. + * + *

Called by {@link OpensearchSinkBase#open(org.apache.flink.configuration.Configuration)} + * after creating the client. This makes sure the underlying client is closed if the connection + * is not successful and preventing thread leak. + * + * @param client the Opensearch client. + */ + private void verifyClientConnection(RestHighLevelClient client) throws IOException { + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Opensearch cluster via hosts {} ...", httpHosts); + } + + if (!client.ping(RequestOptions.DEFAULT)) { + throw new RuntimeException("There are no reachable Opensearch nodes!"); + } + + if (LOG.isInfoEnabled()) { + LOG.info("Opensearch RestHighLevelClient is connected to {}", httpHosts.toString()); + } + } + + /** + * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. The builder + * will be later on used to instantiate the actual {@link BulkProcessor}. + * + * @param builder the {@link BulkProcessor.Builder} to configure. + * @param flushBackoffPolicy user-provided backoff retry settings ({@code null} if the user + * disabled backoff retries). + */ + private static void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable OpensearchSink.BulkFlushBackoffPolicy flushBackoffPolicy) { + + BackoffPolicy backoffPolicy; + if (flushBackoffPolicy != null) { + switch (flushBackoffPolicy.getBackoffType()) { + case CONSTANT: + backoffPolicy = + BackoffPolicy.constantBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + break; + case EXPONENTIAL: + default: + backoffPolicy = + BackoffPolicy.exponentialBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + + builder.setBackoffPolicy(backoffPolicy); + } + + private void configureBulkSize(BulkProcessor.Builder bulkProcessorBuilder) { + final ByteSizeUnit sizeUnit; + if (bulkProcessorFlushMaxSizeMb == -1) { + // bulk size can be disabled with -1, however the ByteSizeValue constructor accepts -1 + // only with BYTES as the size unit + sizeUnit = ByteSizeUnit.BYTES; + } else { + sizeUnit = ByteSizeUnit.MB; + } + bulkProcessorBuilder.setBulkSize(new ByteSizeValue(bulkProcessorFlushMaxSizeMb, sizeUnit)); + } + + private void configureFlushInterval(BulkProcessor.Builder bulkProcessorBuilder) { + if (bulkProcessorFlushIntervalMillis == -1) { + bulkProcessorBuilder.setFlushInterval(null); + } else { + bulkProcessorBuilder.setFlushInterval( + TimeValue.timeValueMillis(bulkProcessorFlushIntervalMillis)); + } + } + + private void checkErrorAndRethrow() { + Throwable cause = failureThrowable.get(); + if (cause != null) { + throw new RuntimeException("An error occurred in OpensearchSink.", cause); + } + } + + private void checkAsyncErrorsAndRequests() { + checkErrorAndRethrow(); + failureRequestIndexer.processBufferedRequests(requestIndexer); + } + + private class BulkProcessorListener implements BulkProcessor.Listener { + + @Override + public void beforeBulk(long executionId, BulkRequest request) {} + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + if (response.hasFailures()) { + BulkItemResponse itemResponse; + Throwable failure; + RestStatus restStatus; + DocWriteRequest actionRequest; + + try { + for (int i = 0; i < response.getItems().length; i++) { + itemResponse = response.getItems()[i]; + failure = extractFailureCauseFromBulkItemResponse(itemResponse); + if (failure != null) { + restStatus = itemResponse.getFailure().getStatus(); + actionRequest = request.requests().get(i); + if (restStatus == null) { + if (actionRequest instanceof ActionRequest) { + failureHandler.onFailure( + (ActionRequest) actionRequest, + failure, + -1, + failureRequestIndexer); + } else { + throw new UnsupportedOperationException( + "The sink currently only supports ActionRequests"); + } + } else { + if (actionRequest instanceof ActionRequest) { + failureHandler.onFailure( + (ActionRequest) actionRequest, + failure, + restStatus.getStatus(), + failureRequestIndexer); + } else { + throw new UnsupportedOperationException( + "The sink currently only supports ActionRequests"); + } + } + } + } + } catch (Throwable t) { + // fail the sink and skip the rest of the items + // if the failure handler decides to throw an exception + failureThrowable.compareAndSet(null, t); + } + } + + if (flushOnCheckpoint) { + numPendingRequests.getAndAdd(-request.numberOfActions()); + } + } + + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + try { + for (DocWriteRequest writeRequest : request.requests()) { + if (writeRequest instanceof ActionRequest) { + failureHandler.onFailure( + (ActionRequest) writeRequest, failure, -1, failureRequestIndexer); + } else { + throw new UnsupportedOperationException( + "The sink currently only supports ActionRequests"); + } + } + } catch (Throwable t) { + // fail the sink and skip the rest of the items + // if the failure handler decides to throw an exception + failureThrowable.compareAndSet(null, t); + } + + if (flushOnCheckpoint) { + numPendingRequests.getAndAdd(-request.numberOfActions()); + } + } + } + + /** + * Extracts the cause of failure of a bulk item action. + * + * @param bulkItemResponse the bulk item response to extract cause of failure + * @return the extracted {@link Throwable} from the response ({@code null} is the response is + * successful). + */ + private static Throwable extractFailureCauseFromBulkItemResponse( + BulkItemResponse bulkItemResponse) { + if (!bulkItemResponse.isFailed()) { + return null; + } else { + return bulkItemResponse.getFailure().getCause(); + } + } + + long getNumPendingRequests() { + if (flushOnCheckpoint) { + return numPendingRequests.get(); + } else { + throw new UnsupportedOperationException( + "The number of pending requests is not maintained when flushing on checkpoint is disabled."); + } + } + + /** + * A builder for creating an {@link OpensearchSink}. + * + * @param Type of the elements handled by the sink this builder creates. + * @deprecated This has been deprecated, please use {@link + * org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder}. + */ + @Deprecated + @PublicEvolving + public static class Builder { + + private final List httpHosts; + private final OpensearchSinkFunction opensearchSinkFunction; + + private Map bulkRequestsConfig = new HashMap<>(); + private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); + private RestClientFactory restClientFactory = restClientBuilder -> {}; + + /** + * Creates a new {@code OpensearchSink} that connects to the cluster using a {@link + * RestHighLevelClient}. + * + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} + * connects to. + * @param opensearchSinkFunction This is used to generate multiple {@link ActionRequest} + * from the incoming element. + */ + public Builder(List httpHosts, OpensearchSinkFunction opensearchSinkFunction) { + this.httpHosts = Preconditions.checkNotNull(httpHosts); + this.opensearchSinkFunction = Preconditions.checkNotNull(opensearchSinkFunction); + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to + * disable it. + * + * @param numMaxActions the maximum number of actions to buffer per bulk request. + */ + public void setBulkFlushMaxActions(int numMaxActions) { + Preconditions.checkArgument( + numMaxActions == -1 || numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); + } + + /** + * 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. + */ + public void setBulkFlushMaxSizeMb(int maxSizeMb) { + Preconditions.checkArgument( + maxSizeMb == -1 || maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); + } + + /** + * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + */ + public void setBulkFlushInterval(long intervalMillis) { + Preconditions.checkArgument( + intervalMillis == -1 || intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than or equal to 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); + } + + /** + * Sets whether or not to enable bulk flush backoff behaviour. + * + * @param enabled whether or not to enable backoffs. + */ + public void setBulkFlushBackoff(boolean enabled) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); + } + + /** + * Sets the type of back of to use when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + */ + public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, + Preconditions.checkNotNull(flushBackoffType).toString()); + } + + /** + * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk + * requests + */ + public void setBulkFlushBackoffRetries(int maxRetries) { + Preconditions.checkArgument( + maxRetries > 0, "Max number of backoff attempts must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); + } + + /** + * Sets the amount of delay between each backoff attempt when flushing bulk requests, in + * milliseconds. + * + * @param delayMillis the amount of delay between each backoff attempt when flushing bulk + * requests, in milliseconds. + */ + public void setBulkFlushBackoffDelay(long delayMillis) { + Preconditions.checkArgument( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); + } + + /** + * Sets a failure handler for action requests. + * + * @param failureHandler This is used to handle failed {@link ActionRequest}. + */ + public void setFailureHandler(ActionRequestFailureHandler failureHandler) { + this.failureHandler = Preconditions.checkNotNull(failureHandler); + } + + /** + * Sets a REST client factory for custom client configuration. + * + * @param restClientFactory the factory that configures the rest client. + */ + public void setRestClientFactory(RestClientFactory restClientFactory) { + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + /** + * Creates the Opensearch sink. + * + * @return the created Opensearch sink. + */ + public OpensearchSink build() { + return new OpensearchSink<>( + bulkRequestsConfig, + httpHosts, + opensearchSinkFunction, + failureHandler, + restClientFactory); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Builder builder = (Builder) o; + return Objects.equals(httpHosts, builder.httpHosts) + && Objects.equals(opensearchSinkFunction, builder.opensearchSinkFunction) + && Objects.equals(bulkRequestsConfig, builder.bulkRequestsConfig) + && Objects.equals(failureHandler, builder.failureHandler) + && Objects.equals(restClientFactory, builder.restClientFactory); + } + + @Override + public int hashCode() { + return Objects.hash( + httpHosts, + opensearchSinkFunction, + bulkRequestsConfig, + failureHandler, + restClientFactory); + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkFunction.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkFunction.java new file mode 100644 index 00000000000000..b3c3e941756107 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkFunction.java @@ -0,0 +1,82 @@ +/* + * 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.opensearch; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RuntimeContext; + +import org.opensearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * Creates multiple {@link ActionRequest ActionRequests} from an element in a stream. + * + *

This is used by sinks to prepare elements for sending them to Opensearch. + * + *

Example: + * + *

{@code
+ * 				private static class TestOpensearchSinkFunction implements
+ * 					OpensearchSinkFunction> {
+ *
+ * 				public IndexRequest createIndexRequest(Tuple2 element) {
+ * 					Map json = new HashMap<>();
+ * 					json.put("data", element.f1);
+ *
+ * 					return Requests.indexRequest()
+ * 						.index("my-index")
+ * 						.id(element.f0.toString())
+ * 						.source(json);
+ * 					}
+ *
+ * 			public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) {
+ * 				indexer.add(createIndexRequest(element));
+ * 			}
+ * 	}
+ *
+ * }
+ * + * @param The type of the element handled by this {@code OpensearchSinkFunction} + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public interface OpensearchSinkFunction extends Serializable, Function { + + /** + * Initialization method for the function. It is called once before the actual working process + * methods. + */ + default void open() throws Exception {} + + /** Tear-down method for the function. It is called when the sink closes. */ + default void close() throws Exception {} + + /** + * Process the incoming element to produce multiple {@link ActionRequest ActionsRequests}. The + * produced requests should be added to the provided {@link RequestIndexer}. + * + * @param element incoming element to process + * @param ctx runtime context containing information about the sink instance + * @param indexer request indexer that {@code ActionRequest} should be added to + */ + void process(T element, RuntimeContext ctx, RequestIndexer indexer); +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RequestIndexer.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RequestIndexer.java new file mode 100644 index 00000000000000..78fefed14f29d6 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RequestIndexer.java @@ -0,0 +1,84 @@ +/* + * 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.opensearch; + +import org.apache.flink.annotation.PublicEvolving; + +import org.opensearch.action.ActionRequest; +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; + +/** + * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare them + * for sending to an Opensearch cluster. + * + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public interface RequestIndexer { + + /** + * Add multiple {@link ActionRequest} to the indexer to prepare for sending requests to + * Opensearch. + * + * @param actionRequests The multiple {@link ActionRequest} to add. + * @deprecated use the {@link DeleteRequest}, {@link IndexRequest} or {@link UpdateRequest} + */ + @Deprecated + default void add(ActionRequest... actionRequests) { + for (ActionRequest actionRequest : actionRequests) { + if (actionRequest instanceof IndexRequest) { + add((IndexRequest) actionRequest); + } else if (actionRequest instanceof DeleteRequest) { + add((DeleteRequest) actionRequest); + } else if (actionRequest instanceof UpdateRequest) { + add((UpdateRequest) actionRequest); + } else { + throw new IllegalArgumentException( + "RequestIndexer only supports Index, Delete and Update requests"); + } + } + } + + /** + * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to + * Opensearch. + * + * @param deleteRequests The multiple {@link DeleteRequest} to add. + */ + void add(DeleteRequest... deleteRequests); + + /** + * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to + * Opensearch. + * + * @param indexRequests The multiple {@link IndexRequest} to add. + */ + void add(IndexRequest... indexRequests); + + /** + * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to + * Opensearch. + * + * @param updateRequests The multiple {@link UpdateRequest} to add. + */ + void add(UpdateRequest... updateRequests); +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RestClientFactory.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RestClientFactory.java new file mode 100644 index 00000000000000..c90e94ce454d3d --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RestClientFactory.java @@ -0,0 +1,42 @@ +/* + * 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.opensearch; + +import org.apache.flink.annotation.PublicEvolving; + +import org.opensearch.client.RestClientBuilder; + +import java.io.Serializable; + +/** + * A factory that is used to configure the {@link org.opensearch.client.RestHighLevelClient} + * internally used in the {@link OpensearchSink}. + * + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public interface RestClientFactory extends Serializable { + + /** + * Configures the rest client builder. + * + * @param restClientBuilder the configured rest client builder. + */ + void configureRestClientBuilder(RestClientBuilder restClientBuilder); +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/IgnoringFailureHandler.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/IgnoringFailureHandler.java new file mode 100644 index 00000000000000..a04cd559b1aa3d --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/IgnoringFailureHandler.java @@ -0,0 +1,38 @@ +/* + * 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.opensearch.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.opensearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.opensearch.RequestIndexer; + +import org.opensearch.action.ActionRequest; + +/** Ignores all kinds of failures and drops the affected {@link ActionRequest}. */ +@Internal +public class IgnoringFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 1662846593501L; + + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { + // ignore failure + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/NoOpFailureHandler.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/NoOpFailureHandler.java new file mode 100644 index 00000000000000..a2c60733bd5a1f --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/NoOpFailureHandler.java @@ -0,0 +1,54 @@ +/* + * 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.opensearch.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.opensearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.opensearch.RequestIndexer; + +import org.opensearch.action.ActionRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** An {@link ActionRequestFailureHandler} that simply fails the sink on any failures. */ +@Internal +public class NoOpFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 737941343410827885L; + + private static final Logger LOG = LoggerFactory.getLogger(NoOpFailureHandler.class); + + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable { + LOG.error("Failed Opensearch item request: {}", failure.getMessage(), failure); + // simply fail the sink + throw failure; + } + + @Override + public boolean equals(Object o) { + return o instanceof NoOpFailureHandler; + } + + @Override + public int hashCode() { + return NoOpFailureHandler.class.hashCode(); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/RetryRejectedExecutionFailureHandler.java new file mode 100644 index 00000000000000..922004ebe24dea --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/RetryRejectedExecutionFailureHandler.java @@ -0,0 +1,60 @@ +/* + * 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.opensearch.util; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.connectors.opensearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.opensearch.RequestIndexer; +import org.apache.flink.util.ExceptionUtils; + +import org.opensearch.action.ActionRequest; +import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An {@link ActionRequestFailureHandler} that re-adds requests that failed due to temporary {@link + * OpenSearchRejectedExecutionException}s (which means that Opensearch node queues are currently + * full), and fails for all other failures. + * + * @deprecated This hase been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public class RetryRejectedExecutionFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = -7423562912824511906L; + + private static final Logger LOG = + LoggerFactory.getLogger(RetryRejectedExecutionFailureHandler.class); + + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable { + LOG.error("Failed Opensearch item request: {}", failure.getMessage(), failure); + if (ExceptionUtils.findThrowable(failure, OpenSearchRejectedExecutionException.class) + .isPresent()) { + indexer.add(action); + } else { + // rethrow all other failures + throw failure; + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-opensearch/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000000000..2cbd625e15e6d7 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.connector.opensearch.table.OpensearchDynamicSinkFactory diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/OpensearchUtil.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/OpensearchUtil.java new file mode 100644 index 00000000000000..d550d847f1d458 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/OpensearchUtil.java @@ -0,0 +1,166 @@ +/* + * 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; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.connectors.opensearch.RestClientFactory; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.testcontainers.opensearch.OpensearchContainer; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.conn.ssl.TrustAllStrategy; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.http.ssl.SSLContexts; +import org.opensearch.client.RestClient; +import org.opensearch.client.RestClientBuilder.HttpClientConfigCallback; +import org.opensearch.client.RestHighLevelClient; +import org.slf4j.Logger; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.utility.DockerImageName; + +import java.security.KeyManagementException; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; + +/** Collection of utility methods for Opensearch tests. */ +@Internal +public class OpensearchUtil { + + private OpensearchUtil() {} + + /** + * Creates a preconfigured {@link OpensearchContainer} with limited memory allocation and aligns + * the internal Opensearch log levels with the ones used by the capturing logger. + * + * @param dockerImageVersion describing the Opensearch image + * @param log to derive the log level from + * @return configured Opensearch container + */ + public static OpensearchContainer createOpensearchContainer( + String dockerImageVersion, Logger log) { + String logLevel; + if (log.isTraceEnabled()) { + logLevel = "TRACE"; + } else if (log.isDebugEnabled()) { + logLevel = "DEBUG"; + } else if (log.isInfoEnabled()) { + logLevel = "INFO"; + } else if (log.isWarnEnabled()) { + logLevel = "WARN"; + } else if (log.isErrorEnabled()) { + logLevel = "ERROR"; + } else { + logLevel = "OFF"; + } + + return new OpensearchContainer(DockerImageName.parse(dockerImageVersion)) + .withEnv("OPENSEARCH_JAVA_OPTS", "-Xms2g -Xmx2g") + .withEnv("logger.org.opensearch", logLevel) + .withLogConsumer(new Slf4jLogConsumer(log)); + } + + /** + * Creates a preconfigured {@link RestHighLevelClient} instance for specific {@link + * OpensearchContainer} instance. + * + * @return preconfigured {@link RestHighLevelClient} instance + */ + public static RestHighLevelClient createClient(OpensearchContainer container) { + final String username = container.getUsername(); + final String password = container.getPassword(); + + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(container.getHttpHostAddress())) + .setHttpClientConfigCallback( + createClientConfigCallback(username, password))); + } + + /** + * Creates a preconfigured {@link RestClientFactory} instance for specific {@link + * OpensearchContainer} instance. + * + * @return preconfigured {@link RestClientFactory} instance + */ + public static RestClientFactory createClientFactory(OpensearchContainer container) { + final String username = container.getUsername(); + final String password = container.getPassword(); + + return factory -> + factory.setHttpClientConfigCallback(createClientConfigCallback(username, password)); + } + + /** + * Creates a dedicated {@link HttpClientConfigCallback} instance for specific {@link + * OpensearchContainer} instance. + * + * @return dedicated {@link HttpClientConfigCallback} instance + */ + private static HttpClientConfigCallback createClientConfigCallback( + final String username, final String password) { + + return (httpClientBuilder) -> { + try { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + + return httpClientBuilder + .setDefaultCredentialsProvider(credentialsProvider) + .setSSLContext( + SSLContexts.custom() + .loadTrustMaterial(new TrustAllStrategy()) + .build()); + } catch (final NoSuchAlgorithmException + | KeyStoreException + | KeyManagementException ex) { + throw new RuntimeException(ex); + } + }; + } + + /** A mock {@link DynamicTableSink.Context} for Opensearch tests. */ + public static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilderTest.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilderTest.java new file mode 100644 index 00000000000000..d7fb231bd9277c --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilderTest.java @@ -0,0 +1,102 @@ +/* + * 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.connector.base.DeliveryGuarantee; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.http.HttpHost; +import org.junit.jupiter.api.DynamicTest; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestFactory; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; + +/** Tests for {@link OpensearchSinkBuilder}. */ +@ExtendWith(TestLoggerExtension.class) +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +class OpensearchSinkBuilderTest { + + @TestFactory + Stream testValidBuilders() { + Stream> validBuilders = + Stream.of( + createMinimalBuilder(), + createMinimalBuilder() + .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE), + createMinimalBuilder() + .setBulkFlushBackoffStrategy(FlushBackoffType.CONSTANT, 1, 1), + createMinimalBuilder() + .setConnectionUsername("username") + .setConnectionPassword("password")); + + return DynamicTest.stream( + validBuilders, + OpensearchSinkBuilder::toString, + builder -> assertDoesNotThrow(builder::build)); + } + + @Test + void testThrowIfExactlyOnceConfigured() { + assertThrows( + IllegalStateException.class, + () -> createMinimalBuilder().setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE)); + } + + @Test + void testThrowIfHostsNotSet() { + assertThrows( + NullPointerException.class, + () -> createEmptyBuilder().setEmitter((element, indexer, context) -> {}).build()); + } + + @Test + void testThrowIfEmitterNotSet() { + assertThrows( + NullPointerException.class, + () -> createEmptyBuilder().setHosts(new HttpHost("localhost:3000")).build()); + } + + @Test + void testThrowIfSetInvalidTimeouts() { + assertThrows( + IllegalStateException.class, + () -> createEmptyBuilder().setConnectionRequestTimeout(-1).build()); + assertThrows( + IllegalStateException.class, + () -> createEmptyBuilder().setConnectionTimeout(-1).build()); + assertThrows( + IllegalStateException.class, + () -> createEmptyBuilder().setSocketTimeout(-1).build()); + } + + private OpensearchSinkBuilder createEmptyBuilder() { + return new OpensearchSinkBuilder<>(); + } + + private OpensearchSinkBuilder createMinimalBuilder() { + return new OpensearchSinkBuilder<>() + .setEmitter((element, indexer, context) -> {}) + .setHosts(new HttpHost("localhost:3000")); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkITCase.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkITCase.java new file mode 100644 index 00000000000000..276fa812b11357 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkITCase.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.sink; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.opensearch.OpensearchUtil; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.testcontainers.opensearch.OpensearchContainer; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.http.HttpHost; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.opensearch.client.RestHighLevelClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.UUID; +import java.util.function.BiFunction; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Tests for {@link OpensearchSink}. */ +@Testcontainers +@ExtendWith(TestLoggerExtension.class) +class OpensearchSinkITCase { + protected static final Logger LOG = LoggerFactory.getLogger(OpensearchSinkITCase.class); + private static boolean failed; + + private RestHighLevelClient client; + private OpensearchTestClient context; + + @Container + private static final OpensearchContainer OS_CONTAINER = + OpensearchUtil.createOpensearchContainer(DockerImageVersions.OPENSEARCH_1, LOG); + + @BeforeEach + void setUp() { + failed = false; + client = OpensearchUtil.createClient(OS_CONTAINER); + context = new OpensearchTestClient(client); + } + + @AfterEach + void tearDown() throws IOException { + if (client != null) { + client.close(); + } + } + + @ParameterizedTest + @EnumSource(DeliveryGuarantee.class) + void testWriteToOpensearchWithDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) + throws Exception { + final String index = "test-opensearch-with-delivery-" + deliveryGuarantee; + boolean failure = false; + try { + runTest(index, false, TestEmitter::jsonEmitter, deliveryGuarantee, null); + } catch (IllegalStateException e) { + failure = true; + assertSame(deliveryGuarantee, DeliveryGuarantee.EXACTLY_ONCE); + } finally { + assertEquals(failure, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE); + } + } + + @ParameterizedTest + @MethodSource("opensearchEmitters") + void testWriteJsonToOpensearch( + BiFunction>> emitterProvider) + throws Exception { + final String index = "test-opensearch-sink-" + UUID.randomUUID(); + runTest(index, false, emitterProvider, null); + } + + @Test + void testRecovery() throws Exception { + final String index = "test-recovery-opensearch-sink"; + runTest(index, true, TestEmitter::jsonEmitter, new FailingMapper()); + assertTrue(failed); + } + + private void runTest( + String index, + boolean allowRestarts, + BiFunction>> emitterProvider, + @Nullable MapFunction additionalMapper) + throws Exception { + runTest( + index, + allowRestarts, + emitterProvider, + DeliveryGuarantee.AT_LEAST_ONCE, + additionalMapper); + } + + private void runTest( + String index, + boolean allowRestarts, + BiFunction>> emitterProvider, + DeliveryGuarantee deliveryGuarantee, + @Nullable MapFunction additionalMapper) + throws Exception { + final OpensearchSink> sink = + new OpensearchSinkBuilder<>() + .setHosts(HttpHost.create(OS_CONTAINER.getHttpHostAddress())) + .setEmitter(emitterProvider.apply(index, context.getDataFieldName())) + .setBulkFlushMaxActions(5) + .setConnectionUsername(OS_CONTAINER.getUsername()) + .setConnectionPassword(OS_CONTAINER.getPassword()) + .setDeliveryGuarantee(deliveryGuarantee) + .setAllowInsecure(true) + .build(); + + final StreamExecutionEnvironment env = new LocalStreamEnvironment(); + env.enableCheckpointing(100L); + if (!allowRestarts) { + env.setRestartStrategy(RestartStrategies.noRestart()); + } + DataStream stream = env.fromSequence(1, 5); + + if (additionalMapper != null) { + stream = stream.map(additionalMapper); + } + + stream.map( + new MapFunction>() { + @Override + public Tuple2 map(Long value) throws Exception { + return Tuple2.of( + value.intValue(), + OpensearchTestClient.buildMessage(value.intValue())); + } + }) + .sinkTo(sink); + env.execute(); + context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5); + } + + private static List>>> + opensearchEmitters() { + return Arrays.asList(TestEmitter::jsonEmitter, TestEmitter::smileEmitter); + } + + private static class FailingMapper implements MapFunction, CheckpointListener { + + private int emittedRecords = 0; + + @Override + public Long map(Long value) throws Exception { + Thread.sleep(50); + emittedRecords++; + return value; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (failed || emittedRecords == 0) { + return; + } + failed = true; + throw new Exception("Expected failure"); + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchTestClient.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchTestClient.java new file mode 100644 index 00000000000000..021ad371e69ea3 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchTestClient.java @@ -0,0 +1,74 @@ +/* + * 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.opensearch.OpenSearchStatusException; +import org.opensearch.action.get.GetRequest; +import org.opensearch.action.get.GetResponse; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.RestHighLevelClient; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +class OpensearchTestClient { + private static final String DATA_FIELD_NAME = "data"; + private final RestHighLevelClient client; + + OpensearchTestClient(RestHighLevelClient client) { + this.client = client; + } + + GetResponse getResponse(String index, int id) throws IOException { + return client.get(new GetRequest(index, Integer.toString(id)), RequestOptions.DEFAULT); + } + + void assertThatIdsAreNotWritten(String index, int... ids) throws IOException { + for (final int id : ids) { + try { + final GetResponse response = getResponse(index, id); + assertFalse( + response.isExists(), String.format("Id %s is unexpectedly present.", id)); + } catch (OpenSearchStatusException e) { + assertEquals(404, e.status().getStatus()); + } + } + } + + void assertThatIdsAreWritten(String index, int... ids) + throws IOException, InterruptedException { + for (final int id : ids) { + GetResponse response; + do { + response = getResponse(index, id); + Thread.sleep(10); + } while (response.isSourceEmpty()); + assertEquals(buildMessage(id), response.getSource().get(DATA_FIELD_NAME)); + } + } + + String getDataFieldName() { + return DATA_FIELD_NAME; + } + + static String buildMessage(int id) { + return "test-" + id; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchWriterITCase.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchWriterITCase.java new file mode 100644 index 00000000000000..23135e574e2316 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchWriterITCase.java @@ -0,0 +1,338 @@ +/* + * 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.api.common.operators.MailboxExecutor; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.opensearch.OpensearchUtil; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.testcontainers.opensearch.OpensearchContainer; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.TestLoggerExtension; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.apache.http.HttpHost; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +import java.io.IOException; +import java.util.Collections; +import java.util.Optional; + +import static org.apache.flink.connector.opensearch.sink.OpensearchTestClient.buildMessage; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Tests for {@link OpensearchWriter}. */ +@Testcontainers +@ExtendWith(TestLoggerExtension.class) +class OpensearchWriterITCase { + + private static final Logger LOG = LoggerFactory.getLogger(OpensearchWriterITCase.class); + + @Container + private static final OpensearchContainer OS_CONTAINER = + OpensearchUtil.createOpensearchContainer(DockerImageVersions.OPENSEARCH_1, LOG); + + private RestHighLevelClient client; + private OpensearchTestClient context; + private MetricListener metricListener; + + @BeforeEach + void setUp() { + metricListener = new MetricListener(); + client = OpensearchUtil.createClient(OS_CONTAINER); + context = new OpensearchTestClient(client); + } + + @AfterEach + void tearDown() throws IOException { + if (client != null) { + client.close(); + } + } + + @Test + void testWriteOnBulkFlush() throws Exception { + final String index = "test-bulk-flush-without-checkpoint"; + final int flushAfterNActions = 5; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final OpensearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.write(Tuple2.of(3, buildMessage(3)), null); + writer.write(Tuple2.of(4, buildMessage(4)), null); + + // Ignore flush on checkpoint + writer.prepareCommit(false); + + context.assertThatIdsAreNotWritten(index, 1, 2, 3, 4); + + // Trigger flush + writer.write(Tuple2.of(5, "test-5"), null); + context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5); + + writer.write(Tuple2.of(6, "test-6"), null); + context.assertThatIdsAreNotWritten(index, 6); + + // Force flush + writer.blockingFlushAllActions(); + context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5, 6); + } + } + + @Test + void testWriteOnBulkIntervalFlush() throws Exception { + final String index = "test-bulk-flush-with-interval"; + + // Configure bulk processor to flush every 1s; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(-1, -1, 1000, FlushBackoffType.NONE, 0, 0); + + try (final OpensearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.write(Tuple2.of(3, buildMessage(3)), null); + writer.write(Tuple2.of(4, buildMessage(4)), null); + writer.blockingFlushAllActions(); + } + + context.assertThatIdsAreWritten(index, 1, 2, 3, 4); + } + + @Test + void testWriteOnCheckpoint() throws Exception { + final String index = "test-bulk-flush-with-checkpoint"; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(-1, -1, -1, FlushBackoffType.NONE, 0, 0); + + // Enable flush on checkpoint + try (final OpensearchWriter> writer = + createWriter(index, true, bulkProcessorConfig)) { + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.write(Tuple2.of(3, buildMessage(3)), null); + + context.assertThatIdsAreNotWritten(index, 1, 2, 3); + + // Trigger flush + writer.prepareCommit(false); + + context.assertThatIdsAreWritten(index, 1, 2, 3); + } + } + + @Test + void testIncrementByteOutMetric() throws Exception { + final String index = "test-inc-byte-out"; + final OperatorIOMetricGroup operatorIOMetricGroup = + UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); + final InternalSinkWriterMetricGroup metricGroup = + InternalSinkWriterMetricGroup.mock( + metricListener.getMetricGroup(), operatorIOMetricGroup); + final int flushAfterNActions = 2; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final OpensearchWriter> writer = + createWriter(index, false, bulkProcessorConfig, metricGroup)) { + final Counter numBytesOut = operatorIOMetricGroup.getNumBytesOutCounter(); + assertEquals(numBytesOut.getCount(), 0); + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + + writer.blockingFlushAllActions(); + long first = numBytesOut.getCount(); + + assertTrue(first > 0); + + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + + writer.blockingFlushAllActions(); + assertTrue(numBytesOut.getCount() > first); + } + } + + @Test + void testCurrentSendTime() throws Exception { + final String index = "test-current-send-time"; + final int flushAfterNActions = 2; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final OpensearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + final Optional> currentSendTime = + metricListener.getGauge("currentSendTime"); + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + + writer.blockingFlushAllActions(); + + assertTrue(currentSendTime.isPresent()); + assertThat(currentSendTime.get().getValue(), greaterThan(0L)); + } + } + + private OpensearchWriter> createWriter( + String index, boolean flushOnCheckpoint, BulkProcessorConfig bulkProcessorConfig) { + return createWriter( + index, + flushOnCheckpoint, + bulkProcessorConfig, + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup())); + } + + private OpensearchWriter> createWriter( + String index, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + SinkWriterMetricGroup metricGroup) { + return new OpensearchWriter>( + Collections.singletonList(HttpHost.create(OS_CONTAINER.getHttpHostAddress())), + TestEmitter.jsonEmitter(index, context.getDataFieldName()), + flushOnCheckpoint, + bulkProcessorConfig, + new TestBulkProcessorBuilderFactory(), + new NetworkClientConfig( + OS_CONTAINER.getUsername(), + OS_CONTAINER.getPassword(), + null, + null, + null, + null, + true), + metricGroup, + new TestMailbox()); + } + + private static class TestBulkProcessorBuilderFactory implements BulkProcessorBuilderFactory { + @Override + public BulkProcessor.Builder apply( + RestHighLevelClient client, + BulkProcessorConfig bulkProcessorConfig, + BulkProcessor.Listener listener) { + BulkProcessor.Builder builder = + BulkProcessor.builder( + new BulkRequestConsumerFactory() { // This cannot be inlined as a lambda + // because then deserialization fails + @Override + public void accept( + BulkRequest bulkRequest, + ActionListener bulkResponseActionListener) { + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener); + } + }, + listener); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue( + bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval(new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + backoffPolicy = BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + break; + case NONE: + backoffPolicy = BackoffPolicy.noBackoff(); + break; + default: + throw new IllegalArgumentException( + "Received unknown backoff policy type " + + bulkProcessorConfig.getFlushBackoffType()); + } + builder.setBackoffPolicy(backoffPolicy); + return builder; + } + } + + private static class TestMailbox implements MailboxExecutor { + + @Override + public void execute( + ThrowingRunnable command, + String descriptionFormat, + Object... descriptionArgs) { + try { + command.run(); + } catch (Exception e) { + throw new RuntimeException("Unexpected error", e); + } + } + + @Override + public void yield() throws InterruptedException, FlinkRuntimeException { + Thread.sleep(100); + } + + @Override + public boolean tryYield() throws FlinkRuntimeException { + return false; + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/TestEmitter.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/TestEmitter.java new file mode 100644 index 00000000000000..86814cdf33eadd --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/TestEmitter.java @@ -0,0 +1,75 @@ +/* + * 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.api.connector.sink.SinkWriter; +import org.apache.flink.api.java.tuple.Tuple2; + +import org.opensearch.action.index.IndexRequest; +import org.opensearch.common.xcontent.XContentBuilder; +import org.opensearch.common.xcontent.XContentFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +class TestEmitter implements OpensearchEmitter> { + + private final String index; + private final XContentBuilderProvider xContentBuilderProvider; + private final String dataFieldName; + + public static TestEmitter jsonEmitter(String index, String dataFieldName) { + return new TestEmitter(index, dataFieldName, XContentFactory::jsonBuilder); + } + + public static TestEmitter smileEmitter(String index, String dataFieldName) { + return new TestEmitter(index, dataFieldName, XContentFactory::smileBuilder); + } + + private TestEmitter( + String index, String dataFieldName, XContentBuilderProvider xContentBuilderProvider) { + this.dataFieldName = dataFieldName; + this.index = index; + this.xContentBuilderProvider = xContentBuilderProvider; + } + + @Override + public void emit( + Tuple2 element, SinkWriter.Context context, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } + + public IndexRequest createIndexRequest(Tuple2 element) { + Map document = new HashMap<>(); + document.put(dataFieldName, element.f1); + try { + return new IndexRequest(index) + .id(element.f0.toString()) + .source(xContentBuilderProvider.getBuilder().map(document)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @FunctionalInterface + private interface XContentBuilderProvider extends Serializable { + XContentBuilder getBuilder() throws IOException; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/IndexGeneratorTest.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/IndexGeneratorTest.java new file mode 100644 index 00000000000000..fe61398565ebfe --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/IndexGeneratorTest.java @@ -0,0 +1,264 @@ +/* + * 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.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.temporal.UnsupportedTemporalTypeException; +import java.util.Arrays; +import java.util.List; + +/** Suite tests for {@link IndexGenerator}. */ +class IndexGeneratorTest { + + private static final List fieldNames = + Arrays.asList( + "id", + "item", + "log_ts", + "log_date", + "order_timestamp", + "log_time", + "local_datetime", + "local_date", + "local_time", + "note", + "status"); + + private static final List dataTypes = + Arrays.asList( + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.BIGINT(), + DataTypes.DATE().bridgedTo(Date.class), + DataTypes.TIMESTAMP().bridgedTo(Timestamp.class), + DataTypes.TIME().bridgedTo(Time.class), + DataTypes.TIMESTAMP().bridgedTo(LocalDateTime.class), + DataTypes.DATE().bridgedTo(LocalDate.class), + DataTypes.TIME().bridgedTo(LocalTime.class), + DataTypes.STRING(), + DataTypes.BOOLEAN()); + + private static final List rows = + Arrays.asList( + GenericRowData.of( + 1, + StringData.fromString("apple"), + Timestamp.valueOf("2020-03-18 12:12:14").getTime(), + (int) Date.valueOf("2020-03-18").toLocalDate().toEpochDay(), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-03-18 12:12:14")), + (int) + (Time.valueOf("12:12:14").toLocalTime().toNanoOfDay() + / 1_000_000L), + TimestampData.fromLocalDateTime( + LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000)), + (int) LocalDate.of(2020, 3, 18).toEpochDay(), + (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L), + "test1", + true), + GenericRowData.of( + 2, + StringData.fromString("peanut"), + Timestamp.valueOf("2020-03-19 12:22:14").getTime(), + (int) Date.valueOf("2020-03-19").toLocalDate().toEpochDay(), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-03-19 12:22:21")), + (int) + (Time.valueOf("12:22:21").toLocalTime().toNanoOfDay() + / 1_000_000L), + TimestampData.fromLocalDateTime( + LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000)), + (int) LocalDate.of(2020, 3, 19).toEpochDay(), + (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L), + "test2", + false)); + + @Test + public void testDynamicIndexFromTimestamp() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH-ss}_index", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH_mm}_index", fieldNames, dataTypes); + indexGenerator1.open(); + Assertions.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalDateTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{local_datetime|yyyy_MM_dd_HH-ss}_index", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{local_datetime|yyyy_MM_dd_HH_mm}_index", fieldNames, dataTypes); + indexGenerator1.open(); + Assertions.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date|yyyy/MM/dd}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_time|HH-mm}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_time|HH-mm}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormat() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_time|}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testGeneralDynamicIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("index_{item}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testStaticIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testUnknownField() { + String expectedExceptionMsg = + "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + + " please check the field name."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{unknown_ts|yyyy-MM-dd}", fieldNames, dataTypes); + } catch (TableException e) { + Assertions.assertEquals(e.getMessage(), expectedExceptionMsg); + } + } + + @Test + public void testUnsupportedTimeType() { + String expectedExceptionMsg = + "Unsupported type 'INT' found in Opensearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{id|yyyy-MM-dd}", fieldNames, dataTypes); + } catch (TableException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedMultiParametersType() { + String expectedExceptionMsg = + "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + + " only support single dynamic index pattern."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date}-{local_time}", fieldNames, dataTypes); + } catch (TableException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testDynamicIndexUnsupportedFormat() { + String expectedExceptionMsg = "Unsupported field: HourOfDay"; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date|yyyy/MM/dd HH:mm}", fieldNames, dataTypes); + } catch (UnsupportedTemporalTypeException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedIndexFieldType() { + String expectedExceptionMsg = + "Unsupported type BOOLEAN of index field, Supported types are:" + + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE," + + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]"; + try { + IndexGeneratorFactory.createIndexGenerator("index_{status}", fieldNames, dataTypes); + } catch (IllegalArgumentException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/KeyExtractorTest.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/KeyExtractorTest.java new file mode 100644 index 00000000000000..537fc721cecd06 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/KeyExtractorTest.java @@ -0,0 +1,147 @@ +/* + * 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.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Collections; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** Tests for {@link KeyExtractor}. */ +class KeyExtractorTest { + @Test + public void testSimpleKey() { + List logicalTypesWithIndex = + Stream.of( + new LogicalTypeWithIndex( + 0, DataTypes.BIGINT().notNull().getLogicalType())) + .collect(Collectors.toList()); + + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + Assertions.assertEquals(key, "12"); + } + + @Test + public void testNoPrimaryKey() { + List logicalTypesWithIndex = Collections.emptyList(); + + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + Assertions.assertEquals(key, null); + } + + @Test + public void testTwoFieldsKey() { + List logicalTypesWithIndex = + Stream.of( + new LogicalTypeWithIndex( + 0, DataTypes.BIGINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 2, DataTypes.TIMESTAMP().notNull().getLogicalType())) + .collect(Collectors.toList()); + + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + 12L, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")))); + Assertions.assertEquals(key, "12_2012-12-12T12:12:12"); + } + + @Test + public void testAllTypesKey() { + List logicalTypesWithIndex = + Stream.of( + new LogicalTypeWithIndex( + 0, DataTypes.TINYINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 1, DataTypes.SMALLINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 2, DataTypes.INT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 3, DataTypes.BIGINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 4, DataTypes.BOOLEAN().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 5, DataTypes.FLOAT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 6, DataTypes.DOUBLE().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 7, DataTypes.STRING().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 8, DataTypes.TIMESTAMP().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 9, + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE() + .notNull() + .getLogicalType()), + new LogicalTypeWithIndex( + 10, DataTypes.TIME().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 11, DataTypes.DATE().notNull().getLogicalType())) + .collect(Collectors.toList()); + + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + (byte) 1, + (short) 2, + 3, + (long) 4, + true, + 1.0f, + 2.0d, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")), + TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")), + (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000), + (int) LocalDate.parse("2015-05-15").toEpochDay())); + Assertions.assertEquals( + key, + "1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15"); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactoryTest.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactoryTest.java new file mode 100644 index 00000000000000..0096d919b2f55d --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactoryTest.java @@ -0,0 +1,267 @@ +/* + * 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.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.connector.opensearch.OpensearchUtil; +import org.apache.flink.table.api.DataTypes; +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.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkProvider; +import org.apache.flink.util.TestLoggerExtension; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.function.Executable; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for validation in {@link OpensearchDynamicSinkFactory}. */ +@ExtendWith(TestLoggerExtension.class) +class OpensearchDynamicSinkFactoryTest { + private TestContext createPrefilledTestContext() { + return TestContext.context() + .withOption(OpensearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + OpensearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:12345"); + } + + @Test + public void validateEmptyConfiguration() { + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + + assertValidationException( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "hosts\n" + + "index", + () -> sinkFactory.createDynamicTableSink(TestContext.context().build())); + } + + void assertValidationException(String expectedMessage, Executable executable) { + ValidationException thrown = Assertions.assertThrows(ValidationException.class, executable); + Assertions.assertEquals(expectedMessage, thrown.getMessage()); + } + + @Test + public void validateWrongIndex() { + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + assertValidationException( + "'index' must not be empty", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + OpensearchConnectorOptions.INDEX_OPTION.key(), "") + .build())); + } + + @Test + public void validateWrongHosts() { + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + assertValidationException( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'.", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + OpensearchConnectorOptions.HOSTS_OPTION.key(), + "wrong-host") + .build())); + } + + @Test + public void validateWrongFlushSize() { + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + assertValidationException( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + OpensearchConnectorOptions + .BULK_FLUSH_MAX_SIZE_OPTION + .key(), + "1kb") + .build())); + } + + @Test + public void validateWrongRetries() { + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + + assertValidationException( + "'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + OpensearchConnectorOptions + .BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), + "0") + .build())); + } + + @Test + public void validateWrongMaxActions() { + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + + assertValidationException( + "'sink.bulk-flush.max-actions' must be at least 1. Got: -2", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + OpensearchConnectorOptions + .BULK_FLUSH_MAX_ACTIONS_OPTION + .key(), + "-2") + .build())); + } + + @Test + public void validateWrongBackoffDelay() { + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + + assertValidationException( + "Invalid value for option 'sink.bulk-flush.backoff.delay'.", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + OpensearchConnectorOptions + .BULK_FLUSH_BACKOFF_DELAY_OPTION + .key(), + "-1s") + .build())); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + + assertValidationException( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withSchema( + new ResolvedSchema( + Arrays.asList( + Column.physical( + "a", + DataTypes.BIGINT() + .notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes + .BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes + .BIGINT(), + DataTypes + .STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes + .BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes + .FIELD( + "a", + DataTypes + .BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer + .INSTANCE) + .notNull()), + Column.physical( + "g", + DataTypes.BYTES() + .notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList( + "a", "b", "c", "d", "e", + "f", "g")))) + .build())); + } + + @Test + public void validateWrongCredential() { + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + + assertValidationException( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + OpensearchConnectorOptions.USERNAME_OPTION.key(), + "username") + .withOption( + OpensearchConnectorOptions.PASSWORD_OPTION.key(), + "") + .build())); + } + + @Test + public void testSinkParallelism() { + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + DynamicTableSink sink = + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption(SINK_PARALLELISM.key(), "2") + .build()); + assertThat(sink).isInstanceOf(OpensearchDynamicSink.class); + OpensearchDynamicSink opensearchSink = (OpensearchDynamicSink) sink; + SinkProvider provider = + (SinkProvider) + opensearchSink.getSinkRuntimeProvider(new OpensearchUtil.MockContext()); + assertThat(2).isEqualTo(provider.getParallelism().get()); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkITCase.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkITCase.java new file mode 100644 index 00000000000000..291aef84eed4f5 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkITCase.java @@ -0,0 +1,334 @@ +/* + * 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.api.common.time.Deadline; +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.connector.opensearch.OpensearchUtil; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkProvider; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.testcontainers.opensearch.OpensearchContainer; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLoggerExtension; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.opensearch.action.get.GetRequest; +import org.opensearch.action.search.SearchRequest; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.RestHighLevelClient; +import org.opensearch.search.SearchHits; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.apache.flink.table.api.Expressions.row; + +/** IT tests for {@link OpensearchDynamicSink}. */ +@ExtendWith(TestLoggerExtension.class) +@Testcontainers +class OpensearchDynamicSinkITCase { + private static final Logger LOG = LoggerFactory.getLogger(OpensearchDynamicSinkITCase.class); + + @Container + private static final OpensearchContainer OS_CONTAINER = + OpensearchUtil.createOpensearchContainer(DockerImageVersions.OPENSEARCH_1, LOG); + + private TestContext getPrefilledTestContext(String index) { + return TestContext.context() + .withOption(OpensearchConnectorOptions.INDEX_OPTION.key(), index) + .withOption( + OpensearchConnectorOptions.HOSTS_OPTION.key(), + OS_CONTAINER.getHttpHostAddress()) + .withOption(OpensearchConnectorOptions.ALLOW_INSECURE.key(), "true") + .withOption( + OpensearchConnectorOptions.USERNAME_OPTION.key(), + OS_CONTAINER.getUsername()) + .withOption( + OpensearchConnectorOptions.PASSWORD_OPTION.key(), + OS_CONTAINER.getPassword()); + } + + private String getConnectorSql(String index) { + return String.format("'%s'='%s',\n", "connector", "opensearch") + + String.format( + "'%s'='%s',\n", OpensearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s', \n", + OpensearchConnectorOptions.HOSTS_OPTION.key(), + OS_CONTAINER.getHttpHostAddress()) + + String.format( + "'%s'='%s', \n", + OpensearchConnectorOptions.USERNAME_OPTION.key(), + OS_CONTAINER.getUsername()) + + String.format( + "'%s'='%s', \n", + OpensearchConnectorOptions.PASSWORD_OPTION.key(), + OS_CONTAINER.getPassword()) + + String.format( + "'%s'='%s'\n", OpensearchConnectorOptions.ALLOW_INSECURE.key(), true); + } + + @Test + public void testWritingDocuments() throws Exception { + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + OpensearchDynamicSinkFactory sinkFactory = new OpensearchDynamicSinkFactory(); + + DynamicTableSink.SinkRuntimeProvider runtimeProvider = + sinkFactory + .createDynamicTableSink( + getPrefilledTestContext(index).withSchema(schema).build()) + .getSinkRuntimeProvider(new OpensearchUtil.MockContext()); + + final SinkProvider sinkProvider = (SinkProvider) runtimeProvider; + final Sink sink = sinkProvider.createSink(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).sinkTo(sink); + environment.execute(); + + RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER); + Map response = + client.get(new GetRequest(index, "1_2012-12-12T12:12:12"), RequestOptions.DEFAULT) + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "table-api"; + tableEnvironment.executeSql( + "CREATE TABLE osTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("osTable") + .await(); + + RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER); + Map response = + client.get(new GetRequest(index, "1_2012-12-12T12:12:12"), RequestOptions.DEFAULT) + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "no-primary-key"; + tableEnvironment.executeSql( + "CREATE TABLE osTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("osTable") + .await(); + + RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = client.search(new SearchRequest(index), RequestOptions.DEFAULT).getHits(); + if (hits.getTotalHits().value < 2) { + Thread.sleep(200); + } + } while (hits.getTotalHits().value < 2 && deadline.hasTimeLeft()); + + if (hits.getTotalHits().value < 2) { + throw new AssertionError("Could not retrieve results from Opensearch."); + } + + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + Assertions.assertEquals(resultSet, expectedSet); + } + + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + tableEnvironment.executeSql( + "CREATE TABLE osTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("osTable") + .await(); + + RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER); + Map response = + client.get(new GetRequest("dynamic-index-2012-12-12", "1"), RequestOptions.DEFAULT) + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/TestContext.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/TestContext.java new file mode 100644 index 00000000000000..50da696ac80db6 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/table/TestContext.java @@ -0,0 +1,72 @@ +/* + * 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.configuration.Configuration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** A utility class for mocking {@link DynamicTableFactory.Context}. */ +class TestContext { + + private ResolvedSchema schema = ResolvedSchema.of(Column.physical("a", DataTypes.TIME())); + + private final Map options = new HashMap<>(); + + public static TestContext context() { + return new TestContext(); + } + + public TestContext withSchema(ResolvedSchema schema) { + this.schema = schema; + return this; + } + + DynamicTableFactory.Context build() { + return new FactoryUtil.DefaultDynamicTableContext( + ObjectIdentifier.of("default", "default", "t1"), + new ResolvedCatalogTable( + CatalogTable.of( + Schema.newBuilder().fromResolvedSchema(schema).build(), + "mock context", + Collections.emptyList(), + options), + schema), + Collections.emptyMap(), + new Configuration(), + TestContext.class.getClassLoader(), + false); + } + + public TestContext withOption(String key, String value) { + options.put(key, value); + return this; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkITCase.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkITCase.java new file mode 100644 index 00000000000000..cb87da974df877 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkITCase.java @@ -0,0 +1,181 @@ +/* + * 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.opensearch; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.opensearch.OpensearchUtil; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.opensearch.testutils.SourceSinkDataTestKit; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.testcontainers.opensearch.OpensearchContainer; +import org.apache.flink.util.DockerImageVersions; + +import org.apache.http.HttpHost; +import org.junit.ClassRule; +import org.junit.Test; +import org.opensearch.client.RestHighLevelClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.function.Function; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.fail; + +/** IT cases for the {@link OpensearchSink}. */ +public class OpensearchSinkITCase extends AbstractTestBase { + + private static final Logger LOG = LoggerFactory.getLogger(OpensearchSinkITCase.class); + + @ClassRule + public static final OpensearchContainer OS_CONTAINER = + OpensearchUtil.createOpensearchContainer(DockerImageVersions.OPENSEARCH_1, LOG); + + @Test + public void testOpensearchSink() throws Exception { + runOpensearchSinkTest( + "opensearch-sink-test-json-index", SourceSinkDataTestKit::getJsonSinkFunction); + } + + @Test + public void testOpensearchSinkWithSmile() throws Exception { + runOpensearchSinkTest( + "opensearch-sink-test-smile-index", SourceSinkDataTestKit::getSmileSinkFunction); + } + + @Test + public void testNullAddresses() { + try { + createOpensearchSink( + 1, getClusterName(), null, SourceSinkDataTestKit.getJsonSinkFunction("test")); + } catch (IllegalArgumentException | NullPointerException expectedException) { + // test passes + return; + } + + fail(); + } + + @Test + public void testEmptyAddresses() { + try { + createOpensearchSink( + 1, + getClusterName(), + Collections.emptyList(), + SourceSinkDataTestKit.getJsonSinkFunction("test")); + } catch (IllegalArgumentException expectedException) { + // test passes + return; + } + + fail(); + } + + @Test + public void testInvalidOpensearchCluster() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = + env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + source.addSink( + createOpensearchSinkForNode( + 1, + "invalid-cluster-name", + SourceSinkDataTestKit.getJsonSinkFunction("test"), + "123.123.123.123")); // incorrect ip address + + try { + env.execute("Opensearch Sink Test"); + } catch (JobExecutionException expectedException) { + assertThat(expectedException.getCause(), instanceOf(JobException.class)); + return; + } + + fail(); + } + + private String getClusterName() { + return "docker-cluster"; + } + + private OpensearchSink> createOpensearchSink( + int bulkFlushMaxActions, + String clusterName, + List httpHosts, + OpensearchSinkFunction> opensearchSinkFunction) { + + OpensearchSink.Builder> builder = + new OpensearchSink.Builder<>(httpHosts, opensearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } + + private OpensearchSink> createOpensearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + OpensearchSinkFunction> opensearchSinkFunction, + String hostAddress) { + + ArrayList httpHosts = new ArrayList<>(); + httpHosts.add(HttpHost.create(hostAddress)); + + OpensearchSink.Builder> builder = + new OpensearchSink.Builder<>(httpHosts, opensearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + builder.setRestClientFactory(OpensearchUtil.createClientFactory(OS_CONTAINER)); + + return builder.build(); + } + + private void runOpensearchSinkTest( + String index, + Function>> functionFactory) + throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = + env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + source.addSink( + createOpensearchSinkForNode( + 1, + getClusterName(), + functionFactory.apply(index), + OS_CONTAINER.getHttpHostAddress())); + + env.execute("Opensearch Sink Test"); + + // verify the results + final RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER); + + SourceSinkDataTestKit.verifyProducedSinkData(client, index); + + client.close(); + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/streaming/connectors/opensearch/testutils/SourceSinkDataTestKit.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/streaming/connectors/opensearch/testutils/SourceSinkDataTestKit.java new file mode 100644 index 00000000000000..fe97d5703837e5 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/streaming/connectors/opensearch/testutils/SourceSinkDataTestKit.java @@ -0,0 +1,143 @@ +/* + * 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.opensearch.testutils; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.opensearch.OpensearchSinkFunction; +import org.apache.flink.streaming.connectors.opensearch.RequestIndexer; + +import org.junit.Assert; +import org.opensearch.action.get.GetRequest; +import org.opensearch.action.get.GetResponse; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.RestHighLevelClient; +import org.opensearch.common.xcontent.XContentBuilder; +import org.opensearch.common.xcontent.XContentFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +/** + * This class contains utilities and a pre-defined source function and Opensearch Sink function used + * to simulate and verify data used in tests. + */ +public class SourceSinkDataTestKit { + + private static final int NUM_ELEMENTS = 20; + + private static final String DATA_PREFIX = "message #"; + private static final String DATA_FIELD_NAME = "data"; + + /** + * A {@link SourceFunction} that generates the elements (id, "message #" + id) with id being 0 - + * 20. + */ + public static class TestDataSourceFunction implements SourceFunction> { + private static final long serialVersionUID = 1L; + + private volatile boolean running = true; + + @Override + public void run(SourceFunction.SourceContext> ctx) + throws Exception { + for (int i = 0; i < NUM_ELEMENTS && running; i++) { + ctx.collect(Tuple2.of(i, DATA_PREFIX + i)); + } + } + + @Override + public void cancel() { + running = false; + } + } + + public static OpensearchSinkFunction> getJsonSinkFunction( + String index) { + return new TestOpensearchSinkFunction(index, XContentFactory::jsonBuilder); + } + + public static OpensearchSinkFunction> getSmileSinkFunction( + String index) { + return new TestOpensearchSinkFunction(index, XContentFactory::smileBuilder); + } + + private static class TestOpensearchSinkFunction + implements OpensearchSinkFunction> { + private static final long serialVersionUID = 1L; + + private final String index; + private final XContentBuilderProvider contentBuilderProvider; + + /** + * Create the sink function, specifying a target Opensearch index. + * + * @param index Name of the target Opensearch index. + */ + public TestOpensearchSinkFunction( + String index, XContentBuilderProvider contentBuilderProvider) { + this.index = index; + this.contentBuilderProvider = contentBuilderProvider; + } + + public IndexRequest createIndexRequest(Tuple2 element) { + Map document = new HashMap<>(); + document.put(DATA_FIELD_NAME, element.f1); + + try { + return new IndexRequest(index) + .id(element.f0.toString()) + .source(contentBuilderProvider.getBuilder().map(document)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void process( + Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } + } + + /** + * Verify the results in an Opensearch index. The results must first be produced into the index + * using a {@link TestOpensearchSinkFunction}; + * + * @param client The client to use to connect to Opensearch + * @param index The index to check + * @throws IOException IOException + */ + public static void verifyProducedSinkData(RestHighLevelClient client, String index) + throws IOException { + for (int i = 0; i < NUM_ELEMENTS; i++) { + GetResponse response = + client.get(new GetRequest(index, Integer.toString(i)), RequestOptions.DEFAULT); + Assert.assertEquals(DATA_PREFIX + i, response.getSource().get(DATA_FIELD_NAME)); + } + } + + @FunctionalInterface + private interface XContentBuilderProvider extends Serializable { + XContentBuilder getBuilder() throws IOException; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/testcontainers/opensearch/OpensearchContainer.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/testcontainers/opensearch/OpensearchContainer.java new file mode 100644 index 00000000000000..953f5611158fe4 --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/testcontainers/opensearch/OpensearchContainer.java @@ -0,0 +1,112 @@ +/* + * 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.testcontainers.opensearch; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; +import org.testcontainers.utility.Base58; +import org.testcontainers.utility.DockerImageName; + +import java.net.InetSocketAddress; +import java.time.Duration; + +import static java.net.HttpURLConnection.HTTP_OK; +import static java.net.HttpURLConnection.HTTP_UNAUTHORIZED; + +/** + * The Opensearch Docker container (single node cluster) which exposes by default ports 9200 (http) + * and 9300 (tcp, deprecated). + */ +public class OpensearchContainer extends GenericContainer { + // Default username to connect to Opensearch instance + private static final String DEFAULT_USER = "admin"; + // Default password to connect to Opensearch instance + private static final String DEFAULT_PASSWORD = "admin"; + + // Default HTTP port. + private static final int DEFAULT_HTTP_PORT = 9200; + + // Default TCP port (deprecated and may be removed in future versions). + @Deprecated private static final int DEFAULT_TCP_PORT = 9300; + + // Opensearch Docker base image. + private static final DockerImageName DEFAULT_IMAGE_NAME = + DockerImageName.parse("opensearchproject/opensearch"); + + /** + * Create an Opensearch Container by passing the full docker image name. + * + * @param dockerImageName Full docker image name as a {@link String}, like: + * opensearchproject/opensearch:1.2.4 + */ + public OpensearchContainer(String dockerImageName) { + this(DockerImageName.parse(dockerImageName)); + } + + /** + * Create an Opensearch Container by passing the full docker image name. + * + * @param dockerImageName Full docker image name as a {@link DockerImageName}, like: + * DockerImageName.parse("opensearchproject/opensearch:1.2.4") + */ + public OpensearchContainer(final DockerImageName dockerImageName) { + super(dockerImageName); + dockerImageName.assertCompatibleWith(DEFAULT_IMAGE_NAME); + + withNetworkAliases("opensearch-" + Base58.randomString(6)); + withEnv("discovery.type", "single-node"); + withEnv("DISABLE_SECURITY_PLUGIN", "true"); + addExposedPorts(DEFAULT_HTTP_PORT, DEFAULT_TCP_PORT); + setWaitStrategy( + new HttpWaitStrategy() + .forPort(DEFAULT_HTTP_PORT) + .withBasicCredentials(DEFAULT_USER, DEFAULT_PASSWORD) + .forStatusCodeMatching( + response -> response == HTTP_OK || response == HTTP_UNAUTHORIZED) + .withReadTimeout(Duration.ofSeconds(10)) + .withStartupTimeout(Duration.ofMinutes(5))); + } + + /** + * Return HTTP host and port to connect to Opensearch container. + * + * @return HTTP host and port (in a form of "host:port") + */ + public String getHttpHostAddress() { + return "http://" + getHost() + ":" + getMappedPort(DEFAULT_HTTP_PORT); + } + + /** + * Return socket address to connect to Opensearch over TCP. The TransportClient will is + * deprecated and may be removed in future versions. + * + * @return TCP socket address + */ + @Deprecated + public InetSocketAddress getTcpHost() { + return new InetSocketAddress(getHost(), getMappedPort(DEFAULT_TCP_PORT)); + } + + public String getUsername() { + return DEFAULT_USER; + } + + public String getPassword() { + return DEFAULT_PASSWORD; + } +} diff --git a/flink-connectors/flink-connector-opensearch/src/test/resources/log4j2-test.properties b/flink-connectors/flink-connector-opensearch/src/test/resources/log4j2-test.properties new file mode 100644 index 00000000000000..835c2ec9a3d02c --- /dev/null +++ b/flink-connectors/flink-connector-opensearch/src/test/resources/log4j2-test.properties @@ -0,0 +1,28 @@ +################################################################################ +# 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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-connectors/flink-sql-connector-opensearch/pom.xml b/flink-connectors/flink-sql-connector-opensearch/pom.xml new file mode 100644 index 00000000000000..0308031b724321 --- /dev/null +++ b/flink-connectors/flink-sql-connector-opensearch/pom.xml @@ -0,0 +1,159 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + .. + + + flink-sql-connector-opensearch + Flink : Connectors : SQL : Opensearch + + jar + + + + org.apache.flink + flink-connector-opensearch + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + *:* + + + + com.tdunning:t-digest + joda-time:joda-time + net.sf.jopt-simple:jopt-simple + net.java.dev.jna:jna + org.hdrhistogram:HdrHistogram + org.yaml:snakeyaml + + + + + + org.opensearch:opensearch + + config/** + modules.txt + plugins.txt + org/joda/** + + org/opensearch/bootstrap/** + + + + org.opensearch.client:opensearch-rest-high-level-client + + forbidden/** + + + + org.apache.httpcomponents:httpclient + + mozilla/** + + + + org.apache.lucene:lucene-analyzers-common + + org/tartarus/** + + + + *:* + + + META-INF/versions/** + META-INF/services/com.fasterxml.** + META-INF/services/org.apache.lucene.** + META-INF/services/org.opensearch.** + META-INF/LICENSE.txt + + + + + + + org.apache.commons + org.apache.flink.opensearch.shaded.org.apache.commons + + + org.apache.http + org.apache.flink.opensearch.shaded.org.apache.http + + + org.apache.lucene + org.apache.flink.opensearch.shaded.org.apache.lucene + + + org.opensearch + org.apache.flink.opensearch.shaded.org.opensearch + + + com.fasterxml.jackson + org.apache.flink.opensearch.shaded.com.fasterxml.jackson + + + com.carrotsearch.hppc + org.apache.flink.opensearch.shaded.com.carrotsearch.hppc + + + com.github.mustachejava + org.apache.flink.opensearch.shaded.com.github.mustachejava + + + net.jpountz + org.apache.flink.opensearch.shaded.net.jpountz + + + + + + + + + + diff --git a/flink-connectors/flink-sql-connector-opensearch/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-opensearch/src/main/resources/META-INF/NOTICE new file mode 100644 index 00000000000000..07b09ebcf7a159 --- /dev/null +++ b/flink-connectors/flink-sql-connector-opensearch/src/main/resources/META-INF/NOTICE @@ -0,0 +1,50 @@ +flink-sql-connector-opensearch +Copyright 2014-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.carrotsearch:hppc:0.8.1 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.2 +- com.github.spullara.mustache.java:compiler:0.9.10 +- commons-codec:commons-codec:1.15 +- commons-logging:commons-logging:1.1.3 +- org.apache.commons:commons-compress:1.21 +- org.apache.httpcomponents:httpasyncclient:4.1.4 +- org.apache.httpcomponents:httpclient:4.5.13 +- org.apache.httpcomponents:httpcore:4.4.14 +- org.apache.httpcomponents:httpcore-nio:4.4.12 +- org.apache.lucene:lucene-analyzers-common:8.10.1 +- org.apache.lucene:lucene-backward-codecs:8.10.1 +- org.apache.lucene:lucene-core:8.10.1 +- org.apache.lucene:lucene-grouping:8.10.1 +- org.apache.lucene:lucene-highlighter:8.10.1 +- org.apache.lucene:lucene-join:8.10.1 +- org.apache.lucene:lucene-memory:8.10.1 +- org.apache.lucene:lucene-misc:8.10.1 +- org.apache.lucene:lucene-queries:8.10.1 +- org.apache.lucene:lucene-queryparser:8.10.1 +- org.apache.lucene:lucene-sandbox:8.10.1 +- org.apache.lucene:lucene-spatial3d:8.10.1 +- org.apache.lucene:lucene-spatial-extras:8.10.1 +- org.apache.lucene:lucene-suggest:8.10.1 +- org.opensearch.client:opensearch-rest-client:1.3.0 +- org.opensearch.client:opensearch-rest-high-level-client:1.3.0 +- org.opensearch:opensearch-cli:1.3.0 +- org.opensearch:opensearch-core:1.3.0 +- org.opensearch:opensearch-geo:1.3.0 +- org.opensearch:opensearch:1.3.0 +- org.opensearch:opensearch-secure-sm:1.3.0 +- org.opensearch:opensearch-x-content:1.3.0 +- org.opensearch.plugin:aggs-matrix-stats-client:1.3.0 +- org.opensearch.plugin:lang-mustache-client:1.3.0 +- org.opensearch.plugin:mapper-extras-client:1.3.0 +- org.opensearch.plugin:parent-join-client:1.3.0 +- org.opensearch.plugin:rank-eval-client:1.3.0 +- net.java.dev.jna:jna:5.5.0 diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index 0caf12f5c4fa3b..874d4a4915cefc 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -57,6 +57,7 @@ under the License. flink-file-sink-common flink-connector-files flink-connector-pulsar + flink-connector-opensearch + + + 4.0.0 + + + org.apache.flink + flink-end-to-end-tests + 1.16-SNAPSHOT + .. + + + flink-opensearch-test + Flink : E2E Tests : Opensearch + jar + + + + org.apache.flink + flink-streaming-java + ${project.version} + provided + + + org.apache.flink + flink-connector-opensearch + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + OpensearchSinkExample + package + + shade + + + OpensearchSinkExample + + + com.google.code.findbugs:jsr305 + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + org.apache.flink.streaming.tests.OpensearchSinkExample + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-opensearch/src/main/java/org/apache/flink/streaming/tests/OpensearchSinkExample.java b/flink-end-to-end-tests/flink-end-to-end-tests-opensearch/src/main/java/org/apache/flink/streaming/tests/OpensearchSinkExample.java new file mode 100644 index 00000000000000..c7fbca1b0c4aad --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-opensearch/src/main/java/org/apache/flink/streaming/tests/OpensearchSinkExample.java @@ -0,0 +1,148 @@ +/* + * 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.tests; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.opensearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.opensearch.OpensearchSink; +import org.apache.flink.streaming.connectors.opensearch.RequestIndexer; +import org.apache.flink.util.Collector; + +import org.apache.http.HttpHost; +import org.opensearch.action.ActionRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; +import org.opensearch.client.Requests; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** End to end test for OpensearchSink. */ +public class OpensearchSinkExample { + + public static void main(String[] args) throws Exception { + + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + + if (parameterTool.getNumberOfParameters() < 2) { + System.out.println( + "Missing parameters!\n" + "Usage: --numRecords --index "); + return; + } + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(5000); + + DataStream> source = + env.generateSequence(0, parameterTool.getInt("numRecords") - 1) + .flatMap( + new FlatMapFunction>() { + @Override + public void flatMap( + Long value, Collector> out) { + final String key = String.valueOf(value); + final String message = "message #" + value; + out.collect(Tuple2.of(key, message + "update #1")); + out.collect(Tuple2.of(key, message + "update #2")); + } + }); + + List httpHosts = new ArrayList<>(); + httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); + + OpensearchSink.Builder> osSinkBuilder = + new OpensearchSink.Builder<>( + httpHosts, + (Tuple2 element, + RuntimeContext ctx, + RequestIndexer indexer) -> { + indexer.add(createIndexRequest(element.f1, parameterTool)); + indexer.add(createUpdateRequest(element, parameterTool)); + }); + + osSinkBuilder.setFailureHandler( + new CustomFailureHandler(parameterTool.getRequired("index"))); + + // this instructs the sink to emit after every element, otherwise they would be buffered + osSinkBuilder.setBulkFlushMaxActions(1); + + source.addSink(osSinkBuilder.build()); + + env.execute("Opensearch end to end sink test example"); + } + + private static class CustomFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 942269087742453482L; + + private final String index; + + CustomFailureHandler(String index) { + this.index = index; + } + + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable { + if (action instanceof IndexRequest) { + Map json = new HashMap<>(); + json.put("data", ((IndexRequest) action).source()); + + indexer.add( + Requests.indexRequest() + .index(index) + .id(((IndexRequest) action).id()) + .source(json)); + } else { + throw new IllegalStateException("unexpected"); + } + } + } + + private static IndexRequest createIndexRequest(String element, ParameterTool parameterTool) { + Map json = new HashMap<>(); + json.put("data", element); + + String index; + if (element.startsWith("message #15")) { + index = ":intentional invalid index:"; + } else { + index = parameterTool.getRequired("index"); + } + + return Requests.indexRequest().index(index).id(element).source(json); + } + + private static UpdateRequest createUpdateRequest( + Tuple2 element, ParameterTool parameterTool) { + Map json = new HashMap<>(); + json.put("data", element.f1); + + return new UpdateRequest(parameterTool.getRequired("index"), element.f0) + .doc(json) + .upsert(json); + } +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 5fbcaf181f3699..c1c557eadbef78 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -83,6 +83,7 @@ under the License. flink-end-to-end-tests-elasticsearch7 flink-end-to-end-tests-common-elasticsearch flink-end-to-end-tests-sql + flink-end-to-end-tests-opensearch diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java index e99aff2176518c..2c7588e240f501 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java @@ -27,6 +27,7 @@ * cache_docker_images.sh#DOCKER_IMAGE_CACHE_PATTERN}. */ public class DockerImageVersions { + public static final String OPENSEARCH_1 = "opensearchproject/opensearch:1.2.4"; public static final String ELASTICSEARCH_7 = "docker.elastic.co/elasticsearch/elasticsearch:7.10.2"; diff --git a/tools/azure-pipelines/cache_docker_images.sh b/tools/azure-pipelines/cache_docker_images.sh index 78d31a56901405..f074035a3c8d47 100755 --- a/tools/azure-pipelines/cache_docker_images.sh +++ b/tools/azure-pipelines/cache_docker_images.sh @@ -28,7 +28,7 @@ then fi # This is the pattern that determines which containers we save. -DOCKER_IMAGE_CACHE_PATTERN="testcontainers|kafka|elasticsearch|postgres|mysql|pulsar|cassandra" +DOCKER_IMAGE_CACHE_PATTERN="testcontainers|kafka|elasticsearch|opensearch|postgres|mysql|pulsar|cassandra" # The path to the tar file that will contain the saved docker images. DOCKER_IMAGES_CACHE_PATH="${DOCKER_IMAGES_CACHE_FOLDER}/cache.tar" diff --git a/tools/ci/java-ci-tools/src/main/resources/modules-skipping-deployment.modulelist b/tools/ci/java-ci-tools/src/main/resources/modules-skipping-deployment.modulelist index a1ac43e7cadfcb..878dc23d580e06 100644 --- a/tools/ci/java-ci-tools/src/main/resources/modules-skipping-deployment.modulelist +++ b/tools/ci/java-ci-tools/src/main/resources/modules-skipping-deployment.modulelist @@ -40,3 +40,4 @@ flink-end-to-end-tests-common-kafka flink-end-to-end-tests-pulsar flink-end-to-end-tests-elasticsearch7 flink-end-to-end-tests-elasticsearch6 +flink-opensearch-test diff --git a/tools/ci/stage.sh b/tools/ci/stage.sh index e5c99251531ba7..72e6006a10bb56 100755 --- a/tools/ci/stage.sh +++ b/tools/ci/stage.sh @@ -134,6 +134,8 @@ flink-connectors/flink-connector-elasticsearch7,\ flink-connectors/flink-sql-connector-elasticsearch6,\ flink-connectors/flink-sql-connector-elasticsearch7,\ flink-connectors/flink-connector-elasticsearch-base,\ +flink-connectors/flink-connector-opensearch,\ +flink-connectors/flink-sql-connector-opensearch,\ flink-metrics/flink-metrics-dropwizard,\ flink-metrics/flink-metrics-graphite,\ flink-metrics/flink-metrics-jmx,\