|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.flink.connector.opensearch.sink; |
| 20 | + |
| 21 | +import org.apache.flink.annotation.Internal; |
| 22 | +import org.apache.flink.annotation.PublicEvolving; |
| 23 | +import org.apache.flink.connector.base.sink.AsyncSinkBase; |
| 24 | +import org.apache.flink.connector.base.sink.writer.BufferedRequestState; |
| 25 | +import org.apache.flink.connector.base.sink.writer.ElementConverter; |
| 26 | +import org.apache.flink.core.io.SimpleVersionedSerializer; |
| 27 | + |
| 28 | +import org.apache.http.HttpHost; |
| 29 | + |
| 30 | +import java.io.IOException; |
| 31 | +import java.util.Collection; |
| 32 | +import java.util.Collections; |
| 33 | +import java.util.List; |
| 34 | + |
| 35 | +import static org.apache.flink.util.Preconditions.checkArgument; |
| 36 | +import static org.apache.flink.util.Preconditions.checkNotNull; |
| 37 | + |
| 38 | +/** |
| 39 | + * Apache Flink's Async Sink to insert or update data in an Opensearch index (see please {@link |
| 40 | + * OpensearchAsyncWriter}). |
| 41 | + * |
| 42 | + * @param <InputT> type of the records converted to Opensearch actions (instances of {@link |
| 43 | + * DocSerdeRequest}) |
| 44 | + * @see OpensearchAsyncSinkBuilder on how to construct a OpensearchAsyncSink |
| 45 | + */ |
| 46 | +@PublicEvolving |
| 47 | +public class OpensearchAsyncSink<InputT> extends AsyncSinkBase<InputT, DocSerdeRequest<?>> { |
| 48 | + private static final long serialVersionUID = 1L; |
| 49 | + |
| 50 | + private final List<HttpHost> hosts; |
| 51 | + private final NetworkClientConfig networkClientConfig; |
| 52 | + |
| 53 | + /** |
| 54 | + * Constructor creating an Opensearch async sink. |
| 55 | + * |
| 56 | + * @param maxBatchSize the maximum size of a batch of entries that may be sent |
| 57 | + * @param maxInFlightRequests he maximum number of in flight requests that may exist, if any |
| 58 | + * more in flight requests need to be initiated once the maximum has been reached, then it |
| 59 | + * will be blocked until some have completed |
| 60 | + * @param maxBufferedRequests the maximum number of elements held in the buffer, requests to add |
| 61 | + * elements will be blocked while the number of elements in the buffer is at the maximum |
| 62 | + * @param maxBatchSizeInBytes the maximum size of a batch of entries that may be sent to KDS |
| 63 | + * measured in bytes |
| 64 | + * @param maxTimeInBufferMS the maximum amount of time an entry is allowed to live in the |
| 65 | + * buffer, if any element reaches this age, the entire buffer will be flushed immediately |
| 66 | + * @param maxRecordSizeInBytes the maximum size of a record the sink will accept into the |
| 67 | + * buffer, a record of size larger than this will be rejected when passed to the sink |
| 68 | + * @param elementConverter converting incoming records to Opensearch write document requests |
| 69 | + * @param hosts the reachable Opensearch cluster nodes |
| 70 | + * @param networkClientConfig describing properties of the network connection used to connect to |
| 71 | + * the Opensearch cluster |
| 72 | + */ |
| 73 | + OpensearchAsyncSink( |
| 74 | + int maxBatchSize, |
| 75 | + int maxInFlightRequests, |
| 76 | + int maxBufferedRequests, |
| 77 | + long maxBatchSizeInBytes, |
| 78 | + long maxTimeInBufferMS, |
| 79 | + long maxRecordSizeInBytes, |
| 80 | + ElementConverter<InputT, DocSerdeRequest<?>> elementConverter, |
| 81 | + List<HttpHost> hosts, |
| 82 | + NetworkClientConfig networkClientConfig) { |
| 83 | + super( |
| 84 | + elementConverter, |
| 85 | + maxBatchSize, |
| 86 | + maxInFlightRequests, |
| 87 | + maxBufferedRequests, |
| 88 | + maxBatchSizeInBytes, |
| 89 | + maxTimeInBufferMS, |
| 90 | + maxRecordSizeInBytes); |
| 91 | + this.hosts = checkNotNull(hosts); |
| 92 | + checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); |
| 93 | + this.networkClientConfig = checkNotNull(networkClientConfig); |
| 94 | + } |
| 95 | + |
| 96 | + /** |
| 97 | + * Create a {@link OpensearchAsyncSinkBuilder} to construct a new {@link OpensearchAsyncSink}. |
| 98 | + * |
| 99 | + * @param <InputT> type of incoming records |
| 100 | + * @return {@link OpensearchAsyncSinkBuilder} |
| 101 | + */ |
| 102 | + public static <InputT> OpensearchAsyncSinkBuilder<InputT> builder() { |
| 103 | + return new OpensearchAsyncSinkBuilder<>(); |
| 104 | + } |
| 105 | + |
| 106 | + @Internal |
| 107 | + @Override |
| 108 | + public StatefulSinkWriter<InputT, BufferedRequestState<DocSerdeRequest<?>>> createWriter( |
| 109 | + InitContext context) throws IOException { |
| 110 | + return new OpensearchAsyncWriter<>( |
| 111 | + context, |
| 112 | + getElementConverter(), |
| 113 | + getMaxBatchSize(), |
| 114 | + getMaxInFlightRequests(), |
| 115 | + getMaxBufferedRequests(), |
| 116 | + getMaxBatchSizeInBytes(), |
| 117 | + getMaxTimeInBufferMS(), |
| 118 | + getMaxRecordSizeInBytes(), |
| 119 | + hosts, |
| 120 | + networkClientConfig, |
| 121 | + Collections.emptyList()); |
| 122 | + } |
| 123 | + |
| 124 | + @Internal |
| 125 | + @Override |
| 126 | + public StatefulSinkWriter<InputT, BufferedRequestState<DocSerdeRequest<?>>> restoreWriter( |
| 127 | + InitContext context, |
| 128 | + Collection<BufferedRequestState<DocSerdeRequest<?>>> recoveredState) |
| 129 | + throws IOException { |
| 130 | + return new OpensearchAsyncWriter<>( |
| 131 | + context, |
| 132 | + getElementConverter(), |
| 133 | + getMaxBatchSize(), |
| 134 | + getMaxInFlightRequests(), |
| 135 | + getMaxBufferedRequests(), |
| 136 | + getMaxBatchSizeInBytes(), |
| 137 | + getMaxTimeInBufferMS(), |
| 138 | + getMaxRecordSizeInBytes(), |
| 139 | + hosts, |
| 140 | + networkClientConfig, |
| 141 | + recoveredState); |
| 142 | + } |
| 143 | + |
| 144 | + @Internal |
| 145 | + @Override |
| 146 | + public SimpleVersionedSerializer<BufferedRequestState<DocSerdeRequest<?>>> |
| 147 | + getWriterStateSerializer() { |
| 148 | + return new OpensearchWriterStateSerializer(); |
| 149 | + } |
| 150 | +} |
0 commit comments