AHeise commented on a change in pull request #17363: URL: https://github.com/apache/flink/pull/17363#discussion_r718220854
########## File path: flink-connectors/flink-connector-elasticsearch7/pom.xml ########## @@ -92,6 +92,12 @@ under the License. <scope>test</scope> </dependency> + <dependency> Review comment: The bom just aligns versions. I'd just make it available to all modules similar to junit. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java ########## @@ -0,0 +1,78 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +class BulkProcessorConfig implements Serializable { Review comment: Good question. Probably not. So ignore. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java ########## @@ -0,0 +1,78 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +class BulkProcessorConfig implements Serializable { + + private final int bulkFlushMaxActions; + private final int bulkFlushMaxMb; + private final long bulkFlushInterval; + @Nullable private final ElasticsearchSinkBase.FlushBackoffType flushBackoffType; Review comment: ```suggestion private final ElasticsearchSinkBase.FlushBackoffType flushBackoffType = NONE; ... enum FlushBackoffType { NONE, ...; } ``` ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchProcessor.java ########## @@ -0,0 +1,79 @@ +/* + * 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.elasticsearch.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.elasticsearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * Creates multiple {@link ActionRequest ActionRequests} from the incoming elements. + * + * <p>This is used by sinks to prepare elements for sending them to Elasticsearch. + * + * <p>Example: + * + * <pre>{@code + * private static class TestElasticsearchProcessor implements ElasticsearchProcessor<Tuple2<Integer, String>> { + * + * public IndexRequest createIndexRequest(Tuple2<Integer, String> element) { + * Map<String, Object> json = new HashMap<>(); + * json.put("data", element.f1); + * + * return Requests.indexRequest() + * .index("my-index") + * .type("my-type") + * .id(element.f0.toString()) + * .source(json); + * } + * + * public void process(Tuple2<Integer, String> element, RequestIndexer indexer) { + * indexer.add(createIndexRequest(element)); + * } + * } + * + * }</pre> + * + * @param <T> The type of the element handled by this {@link ElasticsearchProcessor} + */ +@PublicEvolving +public interface ElasticsearchProcessor<T> extends Function, Serializable { Review comment: Agreed that this is already better, we can still think of better names. I was even thinking of ESSerializationSchema. ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchProcessor.java ########## @@ -0,0 +1,79 @@ +/* + * 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.elasticsearch.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.elasticsearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * Creates multiple {@link ActionRequest ActionRequests} from the incoming elements. + * + * <p>This is used by sinks to prepare elements for sending them to Elasticsearch. + * + * <p>Example: + * + * <pre>{@code + * private static class TestElasticsearchProcessor implements ElasticsearchProcessor<Tuple2<Integer, String>> { + * + * public IndexRequest createIndexRequest(Tuple2<Integer, String> element) { + * Map<String, Object> json = new HashMap<>(); + * json.put("data", element.f1); + * + * return Requests.indexRequest() + * .index("my-index") + * .type("my-type") + * .id(element.f0.toString()) + * .source(json); + * } + * + * public void process(Tuple2<Integer, String> element, RequestIndexer indexer) { + * indexer.add(createIndexRequest(element)); + * } + * } + * + * }</pre> + * + * @param <T> The type of the element handled by this {@link ElasticsearchProcessor} + */ +@PublicEvolving +public interface ElasticsearchProcessor<T> extends Function, Serializable { + + /** + * 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 process(T element, SinkWriter.Context context, RequestIndexer indexer); Review comment: I'm wondering if we can't simply return a list of requests? ########## File path: flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java ########## @@ -0,0 +1,276 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.http.HttpHost; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Builder to construct a {@link ElasticsearchSink}. + * + * <p>The following example shows the minimum setup to create a ElasticsearchSink that submits + * actions on checkpoint. Review comment: Is this good? O_O I'd go with a default bulk processor that emits every 1k requests or so. I see plenty of new users running in OOM otherwise. How does that even work without checkpointing? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org