AHeise commented on a change in pull request #17363:
URL: https://github.com/apache/flink/pull/17363#discussion_r721188286



##########
File path: 
flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.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.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;
+
+/**
+ * Creates none or 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 TestElasticsearchEmitter implements 
ElasticsearchEmitter<Tuple2<Integer, String>> {
+ *
+ *     public IndexRequest createIndexRequest(Tuple2<Integer, String> element) 
{
+ *         Map<String, Object> document = new HashMap<>();
+ *                document.put("data", element.f1);
+ *
+ *            return Requests.indexRequest()
+ *                    .index("my-index")
+ *                        .type("my-type")
+ *                        .id(element.f0.toString())
+ *                        .source(document);
+ *     }
+ *
+ *        public void emit(Tuple2<Integer, String> element, RequestIndexer 
indexer) {
+ *            indexer.add(createIndexRequest(element));
+ *     }
+ * }
+ *
+ * }</pre>
+ *
+ * @param <T> The type of the element handled by this {@link 
ElasticsearchEmitter}
+ */
+@PublicEvolving
+public interface ElasticsearchEmitter<T> 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);

Review comment:
       The context is pretty much only for watermark, right?

##########
File path: 
flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestContext.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.elasticsearch.ElasticsearchStatusException;
+import org.elasticsearch.action.get.GetRequest;
+import org.elasticsearch.action.get.GetResponse;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestHighLevelClient;
+
+import java.io.IOException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+class TestContext {

Review comment:
       How about `TestClient` then? ;)

##########
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:
       I feel like this is not completely addressed. It seems like you set 
actions to 1k by default, but then this comment should also reflect that.




-- 
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


Reply via email to