Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-20768][Connectors/ElasticSearch] Support routing field for Elasticsearch connector #14493

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 7 additions & 0 deletions docs/dev/table/connectors/elasticsearch.md
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,13 @@ Connector Options
<td>String</td>
<td>Delimiter for composite keys ("_" by default), e.g., "$" would result in IDs "KEY1$KEY2$KEY3"."</td>
</tr>
<tr>
<td><h5>routing.filed-name</h5></td>
<td>optional</td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Using field value in the record to dynamically generate routing filed.</td>
</tr>
<tr>
<td><h5>username</h5></td>
<td>optional</td>
Expand Down
7 changes: 7 additions & 0 deletions docs/dev/table/connectors/elasticsearch.zh.md
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,13 @@ Connector Options
<td>String</td>
<td>Delimiter for composite keys ("_" by default), e.g., "$" would result in IDs "KEY1$KEY2$KEY3"."</td>
</tr>
<tr>
<td><h5>routing.filed-name</h5></td>
<td>optional</td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Using field value in the record to dynamically generate routing filed.</td>
</tr>
<tr>
<td><h5>username</h5></td>
<td>optional</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,10 @@ public String getKeyDelimiter() {
return config.get(ElasticsearchOptions.KEY_DELIMITER_OPTION);
}

public Optional<String> getRoutingField() {
return config.getOptional(ElasticsearchOptions.ROUTING_FILED_NAME);
}

public Optional<String> getPathPrefix() {
return config.getOptional(ElasticsearchOptions.CONNECTION_PATH_PREFIX);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,11 @@ public enum BackOffType {
.defaultValue("_")
.withDescription(
"Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\".");
public static final ConfigOption<String> ROUTING_FILED_NAME =
ConfigOptions.key("routing.filed-name")
.stringType()
.noDefaultValue()
.withDescription("Elasticsearch routing filed.");
public static final ConfigOption<String> FAILURE_HANDLER_OPTION =
ConfigOptions.key("failure-handler")
.stringType()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/*
* 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.elasticsearch.table;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.TableColumn;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.data.RowData;

import javax.annotation.Nullable;

import java.io.Serializable;
import java.util.List;
import java.util.function.Function;

/**
* An extractor for a Elasticsearch routing from a {@link RowData}.
*/
@Internal
public class RoutingExtractor {
private RoutingExtractor() {
}

public static Function<RowData, String> createRoutingExtractor(
TableSchema schema,
@Nullable String filedName) {
if (filedName == null) {
return null;
}
List<TableColumn> tableColumns = schema.getTableColumns();
for (int i = 0; i < schema.getFieldCount(); i++) {
TableColumn column = tableColumns.get(i);
if (column.getName().equals(filedName)) {
RowData.FieldGetter fieldGetter = RowData.createFieldGetter(
column.getType().getLogicalType(),
i);
return (Function<RowData, String> & Serializable) (row) -> {
Object fieldOrNull = fieldGetter.getFieldOrNull(row);
if (fieldOrNull != null) {
return fieldOrNull.toString();
} else {
return null;
}
};
}
}
throw new IllegalArgumentException("Filed " + filedName + " not exist in table schema.");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.flink.util.Preconditions;

import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.update.UpdateRequest;
Expand All @@ -50,20 +51,23 @@ class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction<RowData>
private final XContentType contentType;
private final RequestFactory requestFactory;
private final Function<RowData, String> createKey;
private final Function<RowData, String> createRouting;

public RowElasticsearchSinkFunction(
IndexGenerator indexGenerator,
@Nullable String docType, // this is deprecated in es 7+
SerializationSchema<RowData> serializationSchema,
XContentType contentType,
RequestFactory requestFactory,
Function<RowData, String> createKey) {
Function<RowData, String> createKey,
@Nullable Function<RowData, String> createRouting) {
this.indexGenerator = Preconditions.checkNotNull(indexGenerator);
this.docType = docType;
this.serializationSchema = Preconditions.checkNotNull(serializationSchema);
this.contentType = Preconditions.checkNotNull(contentType);
this.requestFactory = Preconditions.checkNotNull(requestFactory);
this.createKey = Preconditions.checkNotNull(createKey);
this.createRouting = createRouting;
}

@Override
Expand Down Expand Up @@ -94,11 +98,13 @@ private void processUpsert(RowData row, RequestIndexer indexer) {
final UpdateRequest updateRequest =
requestFactory.createUpdateRequest(
indexGenerator.generate(row), docType, key, contentType, document);
addRouting(updateRequest, row);
indexer.add(updateRequest);
} else {
final IndexRequest indexRequest =
requestFactory.createIndexRequest(
indexGenerator.generate(row), docType, key, contentType, document);
addRouting(indexRequest, row);
indexer.add(indexRequest);
}
}
Expand All @@ -107,9 +113,17 @@ private void processDelete(RowData row, RequestIndexer indexer) {
final String key = createKey.apply(row);
final DeleteRequest deleteRequest =
requestFactory.createDeleteRequest(indexGenerator.generate(row), docType, key);
addRouting(deleteRequest, row);
indexer.add(deleteRequest);
}

private void addRouting(DocWriteRequest<?> request, RowData row) {
if (null != createRouting) {
String routing = createRouting.apply(row);
request.routing(routing);
}
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand Down
2 changes: 1 addition & 1 deletion flink-connectors/flink-connector-elasticsearch6/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ under the License.
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>elasticsearch</artifactId>
<version>1.15.0</version>
<version>1.15.1</version>
<scope>test</scope>
</dependency>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,9 @@ public SinkFunctionProvider getSinkRuntimeProvider(Context context) {
format,
XContentType.JSON,
REQUEST_FACTORY,
KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter()));
KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter()),
RoutingExtractor.createRoutingExtractor(
schema, config.getRoutingField().orElse(null)));

final ElasticsearchSink.Builder<RowData> builder =
builderProvider.createBuilder(config.getHosts(), upsertFunction);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@
import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.INDEX_OPTION;
import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.KEY_DELIMITER_OPTION;
import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION;
import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.ROUTING_FILED_NAME;
import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION;

/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}. */
Expand All @@ -64,6 +65,7 @@ public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory
private static final Set<ConfigOption<?>> optionalOptions =
Stream.of(
KEY_DELIMITER_OPTION,
ROUTING_FILED_NAME,
FAILURE_HANDLER_OPTION,
FLUSH_ON_CHECKPOINT_OPTION,
BULK_FLASH_MAX_SIZE_OPTION,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.flink.types.RowKind;

import org.elasticsearch.action.get.GetRequest;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
Expand Down Expand Up @@ -369,6 +370,84 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception {
assertThat(response, equalTo(expectedMap));
}

@Test
public void testWritingDocumentsWithRouting() throws Exception {
TableSchema schema =
TableSchema.builder()
.field("a", DataTypes.BIGINT().notNull())
.field("b", DataTypes.TIME())
.field("c", DataTypes.STRING().notNull())
.field("d", DataTypes.FLOAT())
.field("e", DataTypes.TINYINT().notNull())
.field("f", DataTypes.DATE())
.field("g", DataTypes.TIMESTAMP().notNull())
.primaryKey("a", "g")
.build();
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";
String myType = "MyType";
Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory();

SinkFunctionProvider sinkRuntimeProvider =
(SinkFunctionProvider)
sinkFactory
.createDynamicTableSink(
context()
.withSchema(schema)
.withOption(
ElasticsearchOptions.INDEX_OPTION.key(),
index)
.withOption(
ElasticsearchOptions.DOCUMENT_TYPE_OPTION
.key(),
myType)
.withOption(
ElasticsearchOptions.ROUTING_FILED_NAME
.key(),
"c")
.withOption(
ElasticsearchOptions.HOSTS_OPTION.key(),
elasticsearchContainer.getHttpHostAddress())
.withOption(
ElasticsearchOptions
.FLUSH_ON_CHECKPOINT_OPTION
.key(),
"false")
.build())
.getSinkRuntimeProvider(new MockContext());

SinkFunction<RowData> sinkFunction = sinkRuntimeProvider.createSinkFunction();
StreamExecutionEnvironment environment =
StreamExecutionEnvironment.getExecutionEnvironment();
rowData.setRowKind(RowKind.UPDATE_AFTER);
environment.<RowData>fromElements(rowData).addSink(sinkFunction);
environment.execute();

Client client = getClient();
GetResponse response =
client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12").routing("ABCDE"))
.actionGet();
Map<Object, Object> 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");
assertThat(response.getSource(), equalTo(expectedMap));
}

private static class MockContext implements DynamicTableSink.Context {
@Override
public boolean isBounded() {
Expand Down
2 changes: 1 addition & 1 deletion flink-connectors/flink-connector-elasticsearch7/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ under the License.
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>elasticsearch</artifactId>
<version>1.15.0</version>
<version>1.15.1</version>
<scope>test</scope>
</dependency>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,9 @@ public SinkFunctionProvider getSinkRuntimeProvider(Context context) {
format,
XContentType.JSON,
REQUEST_FACTORY,
KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter()));
KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter()),
RoutingExtractor.createRoutingExtractor(
schema, config.getRoutingField().orElse(null)));

final ElasticsearchSink.Builder<RowData> builder =
builderProvider.createBuilder(config.getHosts(), upsertFunction);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@
import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.INDEX_OPTION;
import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.KEY_DELIMITER_OPTION;
import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION;
import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.ROUTING_FILED_NAME;
import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION;

/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. */
Expand All @@ -63,6 +64,7 @@ public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory
private static final Set<ConfigOption<?>> optionalOptions =
Stream.of(
KEY_DELIMITER_OPTION,
ROUTING_FILED_NAME,
FAILURE_HANDLER_OPTION,
FLUSH_ON_CHECKPOINT_OPTION,
BULK_FLASH_MAX_SIZE_OPTION,
Expand Down
Loading