From bf1633ce26f05f9c372ddecb927a747c25b6fba7 Mon Sep 17 00:00:00 2001 From: chengxy Date: Tue, 9 May 2023 17:45:12 +0800 Subject: [PATCH 1/7] add es7 input format --- flink-connector-elasticsearch-base/pom.xml | 23 ++ .../ElasticSearchInputFormatBase.java | 230 ++++++++++++++++++ .../ElasticsearchApiCallBridge.java | 16 +- .../ElasticsearchInputSplit.java | 36 +++ .../ElasticsearchSinkBaseTest.java | 31 +++ .../ElasticsearchSinkTestBase.java | 90 ++++++- .../Elasticsearch6ApiCallBridge.java | 32 +++ .../ElasticSearch7InputFormat.java | 143 +++++++++++ .../ElasticsearchSinkITCase.java | 47 +++- pom.xml | 1 + 10 files changed, 642 insertions(+), 7 deletions(-) create mode 100644 flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticSearchInputFormatBase.java create mode 100644 flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchInputSplit.java create mode 100644 flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticSearch7InputFormat.java diff --git a/flink-connector-elasticsearch-base/pom.xml b/flink-connector-elasticsearch-base/pom.xml index 9c65924d..0f8e8590 100644 --- a/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connector-elasticsearch-base/pom.xml @@ -164,6 +164,29 @@ under the License. test + + org.apache.flink + flink-table-runtime + ${flink.version} + provided + true + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + test + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + test-jar + test + + diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticSearchInputFormatBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticSearchInputFormatBase.java new file mode 100644 index 00000000..e8b57a61 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticSearchInputFormatBase.java @@ -0,0 +1,230 @@ +package org.apache.flink.streaming.connectors.elasticsearch; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.io.LocatableInputSplitAssigner; +import org.apache.flink.api.common.io.RichInputFormat; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputSplitAssigner; + +import org.elasticsearch.action.search.SearchScrollRequest; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.search.Scroll; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.shaded.curator5.com.google.common.base.Preconditions.checkNotNull; + +@Internal +public class ElasticSearchInputFormatBase extends RichInputFormat + implements ResultTypeQueryable { + + + private static final Logger LOG = LoggerFactory.getLogger(ElasticSearchInputFormatBase.class); + private static final long serialVersionUID = 1L; + private final DeserializationSchema deserializationSchema; + private final String index; + private final String type; + + private final String[] fieldNames; + private final QueryBuilder predicate; + private final long limit; + + private final long scrollTimeout; + private final int scrollSize; + + private Scroll scroll; + private String currentScrollWindowId; + private String[] currentScrollWindowHits; + + private int nextRecordIndex = 0; + private long currentReadCount = 0L; + + /** + * Call bridge for different version-specific. + */ + private final ElasticsearchApiCallBridge callBridge; + + private final Map userConfig; + /** + * Elasticsearch client created using the call bridge. + */ + private transient C client; + + public ElasticSearchInputFormatBase( + ElasticsearchApiCallBridge callBridge, + Map userConfig, + DeserializationSchema deserializationSchema, + String[] fieldNames, + String index, + String type, + long scrollTimeout, + int scrollSize, + QueryBuilder predicate, + long limit) { + + this.callBridge = checkNotNull(callBridge); + checkNotNull(userConfig); + // copy config so we can remove entries without side-effects + this.userConfig = new HashMap<>(userConfig); + + this.deserializationSchema = checkNotNull(deserializationSchema); + this.index = index; + this.type = type; + + this.fieldNames = fieldNames; + this.predicate = predicate; + this.limit = limit; + + this.scrollTimeout = scrollTimeout; + this.scrollSize = scrollSize; + + } + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } + + @Override + public void configure(Configuration configuration) { + + try { + client = callBridge.createClient(); + callBridge.verifyClientConnection(client); + } catch (IOException ex) { + LOG.error("Exception while creating connection to Elasticsearch.", ex); + throw new RuntimeException("Cannot create connection to Elasticsearcg.", ex); + } + } + + @Override + public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException { + return null; + } + + @Override + public ElasticsearchInputSplit[] createInputSplits(int minNumSplits) throws IOException { + return callBridge.createInputSplitsInternal(client, index, type, minNumSplits); + } + + @Override + public InputSplitAssigner getInputSplitAssigner(ElasticsearchInputSplit[] inputSplits) { + return new LocatableInputSplitAssigner(inputSplits); + } + + @Override + public void open(ElasticsearchInputSplit split) throws IOException { + SearchRequest searchRequest = new SearchRequest(index); + if (type == null) { + searchRequest.types(Strings.EMPTY_ARRAY); + } else { + searchRequest.types(type); + } + this.scroll = new Scroll(TimeValue.timeValueMinutes(scrollTimeout)); + searchRequest.scroll(scroll); + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + int size; + if (limit > 0) { + size = (int) Math.min(limit, scrollSize); + } else { + size = scrollSize; + } + //elasticsearch default value is 10 here. + searchSourceBuilder.size(size); + searchSourceBuilder.fetchSource(fieldNames, null); + if (predicate != null) { + searchSourceBuilder.query(predicate); + } else { + searchSourceBuilder.query(QueryBuilders.matchAllQuery()); + } + searchRequest.source(searchSourceBuilder); + searchRequest.preference("_shards:" + split.getShard()); + Tuple2 searchResponse = null; + try { + + searchResponse = callBridge.search(client, searchRequest); + } catch (IOException e) { + LOG.error("Search has error: ", e.getMessage()); + } + if (searchResponse != null) { + currentScrollWindowId = searchResponse.f0; + currentScrollWindowHits = searchResponse.f1; + nextRecordIndex = 0; + } + } + + @Override + public boolean reachedEnd() throws IOException { + if (limit > 0 && currentReadCount >= limit) { + return true; + } + + // SearchResponse can be InternalSearchHits.empty(), and the InternalSearchHit[] EMPTY = new InternalSearchHit[0] + if (currentScrollWindowHits.length != 0 && nextRecordIndex > currentScrollWindowHits.length - 1) { + fetchNextScrollWindow(); + } + + return currentScrollWindowHits.length == 0; + } + + @Override + public T nextRecord(T t) throws IOException { + if (reachedEnd()) { + LOG.warn("Already reached the end of the split."); + } + + String hit = currentScrollWindowHits[nextRecordIndex]; + nextRecordIndex++; + currentReadCount++; + LOG.debug("Yielding new record for hit: " + hit); + + return parseSearchHit(hit); + } + + @Override + public void close() throws IOException { + callBridge.close(client); + } + + private void fetchNextScrollWindow() { + Tuple2 searchResponse = null; + SearchScrollRequest scrollRequest = new SearchScrollRequest(currentScrollWindowId); + scrollRequest.scroll(scroll); + + try { + searchResponse = callBridge.scroll(client, scrollRequest); + } catch (IOException e) { + LOG.error("Scroll failed: " + e.getMessage()); + } + + if (searchResponse != null) { + currentScrollWindowId = searchResponse.f0; + currentScrollWindowHits = searchResponse.f1; + nextRecordIndex = 0; + } + } + + private T parseSearchHit(String hit) { + T row = null; + try { + row = deserializationSchema.deserialize(hit.getBytes()); + } catch (IOException e) { + LOG.error("Deserialize search hit failed: " + e.getMessage()); + } + + return row; + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 8061d04a..a49f24f8 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -20,13 +20,18 @@ import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple2; + import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchScrollRequest; import javax.annotation.Nullable; import java.io.IOException; import java.io.Serializable; +import java.util.Map; import java.util.concurrent.atomic.AtomicLong; /** @@ -61,12 +66,19 @@ public interface ElasticsearchApiCallBridge extends Ser */ BulkProcessor.Builder createBulkProcessorBuilder(C client, BulkProcessor.Listener listener); + ElasticsearchInputSplit[] createInputSplitsInternal(C client, String index, String type, int minNumSplits); + + Tuple2 search(C client, SearchRequest searchRequest) throws IOException; + + Tuple2 scroll(C client, SearchScrollRequest searchScrollRequest) throws IOException; + + void close(C client) throws IOException; + /** * 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). + * @return the extracted {@link Throwable} from the response ({@code null} is the response is successful). */ @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchInputSplit.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchInputSplit.java new file mode 100644 index 00000000..07d439d9 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchInputSplit.java @@ -0,0 +1,36 @@ +package org.apache.flink.streaming.connectors.elasticsearch; + +import org.apache.flink.core.io.LocatableInputSplit; + +public class ElasticsearchInputSplit extends LocatableInputSplit { + + private static final long seriaVersionUID = 1L; + + /** The name of the index to retrieve data from. */ + private final String index; + + /** It is null in flink elasticsearch connector 7+. */ + private final String type; + + /** Index will split diffirent shards when index created. */ + private final int shard; + + public ElasticsearchInputSplit(int splitNumber, String[] hostnames, String index, String type, int shard) { + super(splitNumber, hostnames); + this.index = index; + this.type = type; + this.shard = shard; + } + + public String getIndex() { + return index; + } + + public String getType() { + return type; + } + + public int getShard() { + return shard; + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 25cb18b1..93ad6da9 100644 --- a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.elasticsearch; import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.CheckedThread; import org.apache.flink.core.testutils.MultiShotLatch; @@ -35,6 +36,8 @@ import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.client.Client; import org.elasticsearch.client.Requests; import org.junit.Test; @@ -43,6 +46,7 @@ import javax.annotation.Nullable; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -603,6 +607,33 @@ public BulkProcessor.Builder createBulkProcessorBuilder( return null; } + @Override + public ElasticsearchInputSplit[] createInputSplitsInternal( + Client client, + String index, + String type, + int minNumSplits) { + return new ElasticsearchInputSplit[0]; + } + + @Override + public Tuple2 search(Client client, SearchRequest searchRequest) + throws IOException { + return null; + } + + @Override + public Tuple2 scroll( + Client client, + SearchScrollRequest searchScrollRequest) throws IOException { + return null; + } + + @Override + public void close(Client client) throws IOException { + + } + @Nullable @Override public Throwable extractFailureCauseFromBulkItemResponse( diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 71a1e40b..c769b282 100644 --- a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -17,20 +17,38 @@ package org.apache.flink.streaming.connectors.elasticsearch; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.planner.runtime.utils.TestingAppendRowDataSink; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; import org.apache.flink.test.util.AbstractTestBase; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.index.query.QueryBuilder; +import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.function.Function; +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.STRING; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.Assert.assertEquals; /** * Environment preparation and suite of tests for version-specific {@link ElasticsearchSinkBase} @@ -39,9 +57,11 @@ * @param Elasticsearch client type * @param The address type to use */ -public abstract class ElasticsearchSinkTestBase +public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { + protected static final String CLUSTER_NAME = "test-cluster"; + protected abstract RestHighLevelClient getClient(); /** Tests that the Elasticsearch sink works properly with json. */ @@ -77,6 +97,74 @@ private void runElasticSearchSinkTest( client.close(); } + protected void runElasticSearchInputFormatTest() throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + Map userConfig = new HashMap<>(); + userConfig.put("cluster.name", CLUSTER_NAME); + DataType dataType = ROW(FIELD("data", STRING())); + RowType schema = (RowType) dataType.getLogicalType(); + + // pass on missing field + DeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( + schema, InternalTypeInfo.of(schema), false, false, TimestampFormat.ISO_8601); + + ElasticSearchInputFormatBase inputFormat = createElasticsearchInputFormat( + userConfig, + (DeserializationSchema) deserializationSchema, + null, + "elasticsearch-sink-test-json-index", + "flink-es-test-type", + 1000, + 10, + null, + 0 + ); + + DataStream dataStream = env.createInput(inputFormat); + TestingAppendRowDataSink sink = new TestingAppendRowDataSink(InternalTypeInfo.of(schema)); + dataStream.addSink(sink); + env.execute("Elasticsearch Source Test"); + List expected = Arrays.asList( + "+I(message #0)", + "+I(message #1)", + "+I(message #10)", + "+I(message #11)", + "+I(message #12)", + "+I(message #13)", + "+I(message #14)", + "+I(message #15)", + "+I(message #16)", + "+I(message #17)", + "+I(message #18)", + "+I(message #19)", + "+I(message #2)", + "+I(message #3)", + "+I(message #4)", + "+I(message #5)", + "+I(message #6)", + "+I(message #7)", + "+I(message #8)", + "+I(message #9)" + ); + List results = sink.getJavaAppendResults(); + results.sort(String::compareTo); + assertEquals(expected, results); + } + + protected abstract ElasticSearchInputFormatBase createElasticsearchInputFormat( + Map userConfig, + DeserializationSchema deserializationSchema, + String[] fieldNames, + String index, + String type, + long scrollTimeout, + int scrollMaxSize, + QueryBuilder predicate, + int limit + ) throws Exception; + /** * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is {@code * null}. diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index c923db36..88846b7d 100644 --- a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -18,7 +18,9 @@ package org.apache.flink.streaming.connectors.elasticsearch6; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchInputSplit; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.apache.flink.util.Preconditions; @@ -27,6 +29,8 @@ import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; @@ -38,6 +42,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicLong; /** Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. */ @@ -78,6 +83,33 @@ public BulkProcessor.Builder createBulkProcessorBuilder( return BulkProcessor.builder(client::bulkAsync, listener); } + @Override + public ElasticsearchInputSplit[] createInputSplitsInternal( + RestHighLevelClient client, + String index, + String type, + int minNumSplits) { + return new ElasticsearchInputSplit[0]; + } + + @Override + public Tuple2 search(RestHighLevelClient client, SearchRequest searchRequest) + throws IOException { + return null; + } + + @Override + public Tuple2 scroll( + RestHighLevelClient client, + SearchScrollRequest searchScrollRequest) throws IOException { + return null; + } + + @Override + public void close(RestHighLevelClient client) throws IOException { + + } + @Override public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { if (!bulkItemResponse.isFailed()) { diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticSearch7InputFormat.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticSearch7InputFormat.java new file mode 100644 index 00000000..2f34005e --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticSearch7InputFormat.java @@ -0,0 +1,143 @@ +package org.apache.flink.streaming.connectors.elasticsearch7; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticSearchInputFormatBase; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.index.query.QueryBuilder; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +@PublicEvolving +public class ElasticSearch7InputFormat extends + ElasticSearchInputFormatBase { + + private static final long serialVersionUID = 1L; + + public ElasticSearch7InputFormat( + Map userConfig, + List httpHosts, + RestClientFactory restClientFactory, + DeserializationSchema serializationSchema, + String[] fieldNames, + String index, + long scrollTimeout, + int scrollSize, + QueryBuilder predicate, + int limit) { + + super(new Elasticsearch7ApiCallBridge(httpHosts, restClientFactory), userConfig, serializationSchema, fieldNames, index, null, scrollTimeout, scrollSize, predicate, limit); + } + + /** + * A builder for creating an {@link ElasticSearch7InputFormat}. + * + * @param Type of the elements. + */ + @PublicEvolving + public static class Builder { + private Map userConfig = new HashMap<>(); + private List httpHosts; + private RestClientFactory restClientFactory = restClientBuilder -> { + }; + private DeserializationSchema deserializationSchema; + private String index; + + private long scrollTimeout; + private int scrollMaxSize; + + private String[] fieldNames; + private QueryBuilder predicate; + private int limit; + + public Builder() { + } + + /** + * Sets HttpHost which the RestHighLevelClient connects to. + * + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. + */ + public Builder setHttpHosts(List httpHosts) { + this.httpHosts = httpHosts; + return this; + } + + /** + * Sets a REST client factory for custom client configuration. + * + * @param restClientFactory the factory that configures the rest client. + */ + public Builder setRestClientFactory(RestClientFactory restClientFactory) { + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + return this; + } + + public Builder setDeserializationSchema(DeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + return this; + } + + public Builder setIndex(String index) { + this.index = index; + return this; + } + + /** + * Sets the maxinum number of each Elasticsearch scroll request. + * + * @param scrollMaxSize the maxinum number of each Elasticsearch scroll request. + */ + public Builder setScrollMaxSize(int scrollMaxSize) { + Preconditions.checkArgument( + scrollMaxSize > 0, + "Maximum number each Elasticsearch scroll request must be larger than 0."); + + this.scrollMaxSize = scrollMaxSize; + return this; + } + + /** + * Sets the search context alive for scroll requests, in milliseconds. + * + * @param scrollTimeout the search context alive for scroll requests, in milliseconds. + */ + public Builder setScrollTimeout(long scrollTimeout) { + Preconditions.checkArgument( + scrollTimeout >= 0, + "Yhe search context alive for scroll requests must be larger than or equal to 0."); + + this.scrollTimeout = scrollTimeout; + return this; + } + + public Builder setFieldNames(String[] fieldNames) { + this.fieldNames = fieldNames; + return this; + } + + public Builder setPredicate(QueryBuilder predicate) { + this.predicate = predicate; + return this; + } + + public Builder setLimit(int limit) { + this.limit = limit; + return this; + } + + /** + * Creates the ElasticSearch7RowDataInputFormat. + * + * @return the created ElasticSearch7RowDataInputFormat. + */ + public ElasticSearch7InputFormat build() { + return new ElasticSearch7InputFormat(userConfig, httpHosts, restClientFactory, deserializationSchema, fieldNames, index, scrollTimeout, scrollMaxSize, predicate, limit); + } + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index 55ff8ed5..2baa6eed 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -18,9 +18,11 @@ package org.apache.flink.streaming.connectors.elasticsearch7; +import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.connector.elasticsearch.test.DockerImageVersions; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticSearchInputFormatBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; @@ -28,17 +30,21 @@ import org.apache.http.HttpHost; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.index.query.QueryBuilder; import org.junit.ClassRule; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.elasticsearch.ElasticsearchContainer; +import java.net.InetAddress; +import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; +import java.util.Map; /** IT cases for the {@link ElasticsearchSink}. */ -public class ElasticsearchSinkITCase +public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkITCase.class); @@ -64,6 +70,12 @@ public void testElasticsearchSinkWithSmile() throws Exception { runElasticsearchSinkSmileTest(); } + @Test + public void testElasticsearchInputFormat() throws Exception { + runElasticsearchSinkTest(); + runElasticSearchInputFormatTest(); + } + @Test public void testNullAddresses() { runNullAddressesTest(); @@ -82,9 +94,9 @@ public void testInvalidElasticsearchCluster() throws Exception { @Override protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( - int bulkFlushMaxActions, - List httpHosts, - ElasticsearchSinkFunction> elasticsearchSinkFunction) { + int bulkFlushMaxActions, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); @@ -93,6 +105,33 @@ public void testInvalidElasticsearchCluster() throws Exception { return builder.build(); } + protected ElasticSearchInputFormatBase createElasticsearchInputFormat( + Map userConfig, + DeserializationSchema deserializationSchema, + String[] fieldNames, + String index, + String type, + long scrollTimeout, + int scrollMaxSize, + QueryBuilder predicate, + int limit) throws Exception { + List transports = new ArrayList<>(); + transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300)); + + ElasticSearch7InputFormat builder = new ElasticSearch7InputFormat.Builder() + .setDeserializationSchema(deserializationSchema) + .setFieldNames(fieldNames) + .setIndex(index) + .setScrollTimeout(scrollTimeout) + .setScrollMaxSize(scrollMaxSize) + .setPredicate(predicate) + .setLimit(limit) + .build(); + return builder; + } + + + @Override protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( diff --git a/pom.xml b/pom.xml index 529c5a2d..235dce68 100644 --- a/pom.xml +++ b/pom.xml @@ -52,6 +52,7 @@ under the License. 1.17.0 16.1 + 2.12 2.13.4.20221013 4.13.2 From 88f51f8974fca9e09fc470e93812bd5e4679b897 Mon Sep 17 00:00:00 2001 From: chengxy Date: Wed, 10 May 2023 14:27:51 +0800 Subject: [PATCH 2/7] add es7 source --- .../table/ElasticsearchConfiguration.java | 22 ++ .../table/ElasticsearchConnectorOptions.java | 31 +++ .../table/ElasticsearchLookupOptions.java | 84 ++++++++ .../ElasticsearchRowDataLookupFunction.java | 202 ++++++++++++++++++ .../table/Elasticsearch7DynamicSource.java | 99 +++++++++ ...=> Elasticsearch7DynamicTableFactory.java} | 77 ++++++- .../Elasticsearch7ApiCallBridge.java | 38 +++- .../org.apache.flink.table.factories.Factory | 2 +- ...lasticsearch7DynamicTableFactoryTest.java} | 2 +- .../Elasticsearch7DynamicSinkITCase.java | 14 +- .../Elasticsearch7DynamicSourceITCase.java | 82 +++++++ ...lasticsearch7DynamicTableFactoryTest.java} | 22 +- .../Elasticsearch7DynamicTableTestBase.java | 168 +++++++++++++++ .../ElasticsearchSinkITCase.java | 7 +- 14 files changed, 822 insertions(+), 28 deletions(-) create mode 100644 flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchLookupOptions.java create mode 100644 flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java create mode 100644 flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java rename flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/{Elasticsearch7DynamicSinkFactory.java => Elasticsearch7DynamicTableFactory.java} (69%) rename flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/{Elasticsearch7DynamicSinkFactoryTest.java => Elasticsearch7DynamicTableFactoryTest.java} (95%) create mode 100644 flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSourceITCase.java rename flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/{Elasticsearch7DynamicSinkFactoryTest.java => Elasticsearch7DynamicTableFactoryTest.java} (91%) create mode 100644 flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableTestBase.java diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java index 04c76333..00244b6e 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java @@ -38,6 +38,8 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.SCROLL_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.SCROLL_TIMEOUT_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; /** Accessor methods to elasticsearch options. */ @@ -150,6 +152,26 @@ public Optional getPathPrefix() { return config.getOptional(ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX); } + public Optional getScrollMaxSize() { + return config.getOptional(SCROLL_MAX_SIZE_OPTION); + } + + public Optional getScrollTimeout() { + return config.getOptional(SCROLL_TIMEOUT_OPTION).map(Duration::toMillis); + } + + public long getCacheMaxSize() { + return config.get(ElasticsearchConnectorOptions.LOOKUP_CACHE_MAX_ROWS); + } + + public Duration getCacheExpiredMs() { + return config.get(ElasticsearchConnectorOptions.LOOKUP_CACHE_TTL); + } + + public int getMaxRetryTimes() { + return config.get(ElasticsearchConnectorOptions.LOOKUP_MAX_RETRIES); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java index 4838b035..6b53184d 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -77,6 +77,19 @@ public class ElasticsearchConnectorOptions { .withDescription( "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); + // elasticsearch source config options + public static final ConfigOption SCROLL_MAX_SIZE_OPTION = + ConfigOptions.key("scan.scroll.max-size") + .intType() + .noDefaultValue() + .withDescription("Maximum number of hits to be returned with each Elasticsearch scroll request"); + + public static final ConfigOption SCROLL_TIMEOUT_OPTION = + ConfigOptions.key("scan.scroll.timeout") + .durationType() + .noDefaultValue() + .withDescription("Amount of time Elasticsearch will keep the search context alive for scroll requests"); + public static final ConfigOption FAILURE_HANDLER_OPTION = ConfigOptions.key("failure-handler") .stringType() @@ -152,6 +165,24 @@ public class ElasticsearchConnectorOptions { "The format must produce a valid JSON document. " + "Please refer to the documentation on formats for more details."); + // look up config options + public static final ConfigOption LOOKUP_CACHE_MAX_ROWS = ConfigOptions + .key("lookup.cache.max-rows") + .longType() + .defaultValue(-1L) + .withDescription("the max number of rows of lookup cache, over this value, the oldest rows will " + + "be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " + + "specified. Cache is not enabled as default."); + public static final ConfigOption LOOKUP_CACHE_TTL = ConfigOptions + .key("lookup.cache.ttl") + .durationType() + .defaultValue(Duration.ofSeconds(10)) + .withDescription("the cache time to live."); + public static final ConfigOption LOOKUP_MAX_RETRIES = ConfigOptions + .key("lookup.max-retries") + .intType() + .defaultValue(3) + .withDescription("the max retry times if lookup database failed."); // -------------------------------------------------------------------------------------------- // Enums // -------------------------------------------------------------------------------------------- diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchLookupOptions.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchLookupOptions.java new file mode 100644 index 00000000..f2e5b470 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchLookupOptions.java @@ -0,0 +1,84 @@ +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import java.io.Serializable; +import java.util.Objects; + +public class ElasticsearchLookupOptions implements Serializable { + + private final long cacheMaxSize; + private final long cacheExpireMs; + private final int maxRetryTimes; + + public ElasticsearchLookupOptions(long cacheMaxSize, long cacheExpireMs, int maxRetryTimes) { + this.cacheMaxSize = cacheMaxSize; + this.cacheExpireMs = cacheExpireMs; + this.maxRetryTimes = maxRetryTimes; + } + + public long getCacheMaxSize() { + return cacheMaxSize; + } + + public long getCacheExpireMs() { + return cacheExpireMs; + } + + public int getMaxRetryTimes() { + return maxRetryTimes; + } + + public static Builder builder() { + return new Builder(); + } + + @Override + public boolean equals(Object o) { + if (o instanceof ElasticsearchLookupOptions) { + ElasticsearchLookupOptions options = (ElasticsearchLookupOptions) o; + return Objects.equals(cacheMaxSize, options.cacheMaxSize) && + Objects.equals(cacheExpireMs, options.cacheExpireMs) && + Objects.equals(maxRetryTimes, options.maxRetryTimes); + } else { + return false; + } + } + + /** + * Builder of {@link ElasticsearchLookupOptions}. + */ + public static class Builder { + private long cacheMaxSize = -1L; + private long cacheExpireMs = -1L; + private int maxRetryTimes = 3; + + /** + * optional, lookup cache max size, over this value, the old data will be eliminated. + */ + public Builder setCacheMaxSize(long cacheMaxSize) { + this.cacheMaxSize = cacheMaxSize; + return this; + } + + /** + * optional, lookup cache expire mills, over this time, the old data will expire. + */ + public Builder setCacheExpireMs(long cacheExpireMs) { + this.cacheExpireMs = cacheExpireMs; + return this; + } + + /** + * optional, max retry times for jdbc connector. + */ + public Builder setMaxRetryTimes(int maxRetryTimes) { + this.maxRetryTimes = maxRetryTimes; + return this; + } + + public ElasticsearchLookupOptions build() { + return new ElasticsearchLookupOptions(cacheMaxSize, cacheExpireMs, maxRetryTimes); + } + } + + +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java new file mode 100644 index 00000000..5f307695 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java @@ -0,0 +1,202 @@ +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.data.util.DataFormatConverters.DataFormatConverter; +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.functions.TableFunction; + +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.Row; + +import org.apache.flink.util.Preconditions; + +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.cache.Cache; +import org.elasticsearch.common.cache.CacheBuilder; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.elasticsearch.common.cache.CacheBuilder.builder; + +public class ElasticsearchRowDataLookupFunction extends TableFunction { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchRowDataLookupFunction.class); + + private final DeserializationSchema deserializationSchema; + + private final String index; + + private final String type; + + private final String[] producedNames; + + private final String[] lookupKeys; + + // converters to convert data from internal to external in order to generate keys for the cache. + private final DataFormatConverter[] converters; + private SearchRequest searchRequest; + private SearchSourceBuilder searchSourceBuilder; + private final long cacheMaxSize; + private final long cacheExpireMs; + private final long maxRetryTimes; + private final ElasticsearchApiCallBridge callBridge; + + private transient C client; + private transient Cache> cache; + + public ElasticsearchRowDataLookupFunction( + DeserializationSchema deserializationSchema, + ElasticsearchLookupOptions lookupOptions, + String index, + String type, + String[] producedNames, + DataType[] producedTypes, + String[] lookupKeys, + ElasticsearchApiCallBridge callBridge) { + + checkNotNull(deserializationSchema, "No DeserializationSchema supplied."); + checkNotNull(lookupOptions, "No ElasticsearchLookupOptions supplied."); + checkNotNull(producedNames, "No fieldNames supplied."); + checkNotNull(producedTypes, "No fieldTypes supplied."); + checkNotNull(lookupKeys, "No keyNames supplied."); + checkNotNull(callBridge, "No ElasticsearchApiCallBridge supplied."); + + this.deserializationSchema = deserializationSchema; + this.cacheExpireMs = lookupOptions.getCacheExpireMs(); + this.cacheMaxSize = lookupOptions.getCacheMaxSize(); + this.maxRetryTimes = lookupOptions.getMaxRetryTimes(); + + this.index = index; + this.type = type; + this.producedNames = producedNames; + this.lookupKeys = lookupKeys; + this.converters = new DataFormatConverter[lookupKeys.length]; + Map nameToIndex = IntStream.range(0, producedNames.length).boxed().collect( + Collectors.toMap(i -> producedNames[i], i -> i)); + for (int i = 0; i < lookupKeys.length; i++) { + Integer position = nameToIndex.get(lookupKeys[i]); + Preconditions.checkArgument(position != null, "Lookup keys %s not selected", Arrays.toString(lookupKeys)); + converters[i] = DataFormatConverters.getConverterForDataType(producedTypes[position]); + } + this.callBridge = callBridge; + } + + @Override + public void open(FunctionContext context) throws Exception { + this.cache = cacheMaxSize == -1 || cacheExpireMs == -1 ? null : CacheBuilder.>builder().setExpireAfterWrite(TimeValue.timeValueMillis(cacheExpireMs)) + .setMaximumWeight(cacheMaxSize).build(); + this.client = callBridge.createClient(); + //Set searchRequest in open method in case of amount of calling in eval method when every record comes. + this.searchRequest = new SearchRequest(index); + if (type == null) { + searchRequest.types(Strings.EMPTY_ARRAY); + } else { + searchRequest.types(type); + } + searchSourceBuilder = new SearchSourceBuilder(); + searchSourceBuilder.fetchSource(producedNames, null); + } + + + /** + * This is a lookup method which is called by Flink framework in runtime. + * + * @param keys lookup keys + */ + public void eval(Object... keys) { + RowData keyRow = GenericRowData.of(keys); + if (cache != null) { + List cachedRows = cache.get(keyRow); + if (cachedRows != null) { + for (RowData cachedRow : cachedRows) { + collect(cachedRow); + } + return; + } + } + + BoolQueryBuilder lookupCondition = new BoolQueryBuilder(); + for (int i = 0; i < lookupKeys.length; i++) { + lookupCondition.must(new TermQueryBuilder(lookupKeys[i], converters[i].toExternal(keys[i]))); + } + searchSourceBuilder.query(lookupCondition); + searchRequest.source(searchSourceBuilder); + + Tuple2 searchResponse = null; + + for (int retry = 1; retry <= maxRetryTimes; retry++) { + try { + searchResponse = callBridge.search(client, searchRequest); + if (searchResponse.f1.length > 0) { + String[] result = searchResponse.f1; + // if cache disabled + if (cache == null) { + for (int i = 0; i < result.length; i++) { + RowData row = parseSearchHit(result[i]); + collect(row); + } + } else { // if cache enabled + ArrayList rows = new ArrayList<>(); + for (int i = 0; i < result.length; i++) { + RowData row = parseSearchHit(result[i]); + collect(row); + rows.add(row); + } + cache.put(keyRow, rows); + } + } + break; + } catch (IOException e) { + LOG.error(String.format("Elasticsearch search error, retry times = %d", retry), e); + if (retry >= maxRetryTimes) { + throw new RuntimeException("Execution of Elasticsearch search failed.", e); + } + try { + Thread.sleep(1000 * retry); + } catch (InterruptedException e1) { + LOG.warn("Interrupted while waiting to retry failed elasticsearch search, aborting"); + throw new RuntimeException(e1); + } + } + } + } + + private RowData parseSearchHit(String hit) { + RowData row = null; + try { + row = deserializationSchema.deserialize(hit.getBytes()); + } catch (IOException e) { + LOG.error("Deserialize search hit failed: " + e.getMessage()); + } + + return row; + } + + + + + + + + +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java new file mode 100644 index 00000000..7580483e --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java @@ -0,0 +1,99 @@ +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch7.Elasticsearch7ApiCallBridge; +import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.LookupTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.TableFunctionProvider; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +@Internal +public class Elasticsearch7DynamicSource implements ScanTableSource, LookupTableSource, + SupportsProjectionPushDown { + + private final DecodingFormat> format; + private final Elasticsearch7Configuration config; + private final ElasticsearchLookupOptions lookupOptions; + private TableSchema physicalSchema; + + public Elasticsearch7DynamicSource( + DecodingFormat> format, + Elasticsearch7Configuration config, + TableSchema physicalSchema, + ElasticsearchLookupOptions lookupOptions) { + this.format = format; + this.config = config; + this.physicalSchema = physicalSchema; + this.lookupOptions = lookupOptions; + } + + @Override + public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext lookupContext) { + RestClientFactory restClientFactory = null; + if (config.getPathPrefix().isPresent()) { + restClientFactory = new Elasticsearch7DynamicSink.DefaultRestClientFactory(config.getPathPrefix().get()); + } else { + restClientFactory = restClientBuilder -> {}; + } + + Elasticsearch7ApiCallBridge elasticsearch7ApiCallBridge = new Elasticsearch7ApiCallBridge( + config.getHosts(), restClientFactory); + + // Elasticsearch only support non-nested look up keys + String[] lookupKeys = new String[lookupContext.getKeys().length]; + String [] columnNames = physicalSchema.getFieldNames(); + for (int i = 0; i < lookupKeys.length; i++) { + int[] innerKeyArr = lookupContext.getKeys()[i]; + Preconditions.checkArgument(innerKeyArr.length == 1, "Elasticsearch only support non-nested look up keys"); + lookupKeys[i] = columnNames[innerKeyArr[0]]; + } + DataType[] columnDataTypes = physicalSchema.getFieldDataTypes(); + + return TableFunctionProvider.of(new ElasticsearchRowDataLookupFunction( + this.format.createRuntimeDecoder(lookupContext, physicalSchema.toRowDataType()), + lookupOptions, + config.getIndex(), + config.getDocumentType(), + columnNames, + columnDataTypes, + lookupKeys, + elasticsearch7ApiCallBridge + )); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { + return null; + } + + @Override + public DynamicTableSource copy() { + return null; + } + + @Override + public String asSummaryString() { + return null; + } + + @Override + public boolean supportsNestedProjection() { + return false; + } +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java similarity index 69% rename from flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java rename to flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java index 70b0134d..2a9dc4c2 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; @@ -26,10 +27,15 @@ import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.SerializationFormatFactory; import org.apache.flink.table.utils.TableSchemaUtils; @@ -54,16 +60,23 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.LOOKUP_CACHE_MAX_ROWS; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.LOOKUP_CACHE_TTL; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.LOOKUP_MAX_RETRIES; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.SCROLL_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.SCROLL_TIMEOUT_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; /** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. */ @Internal -public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory { +public class Elasticsearch7DynamicTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory { private static final Set> requiredOptions = Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); private static final Set> optionalOptions = Stream.of( + SCROLL_MAX_SIZE_OPTION, + SCROLL_TIMEOUT_OPTION, KEY_DELIMITER_OPTION, FAILURE_HANDLER_OPTION, FLUSH_ON_CHECKPOINT_OPTION, @@ -75,10 +88,29 @@ public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory BULK_FLUSH_BACKOFF_DELAY_OPTION, CONNECTION_PATH_PREFIX, FORMAT_OPTION, + LOOKUP_CACHE_MAX_ROWS, + LOOKUP_CACHE_TTL, + LOOKUP_MAX_RETRIES, PASSWORD_OPTION, USERNAME_OPTION) .collect(Collectors.toSet()); + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + + TableSchema schema = context.getCatalogTable().getSchema(); + final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + final DecodingFormat> format = helper.discoverDecodingFormat( + DeserializationFormatFactory.class, FORMAT_OPTION); + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable().getOptions().forEach(configuration::setString); + Elasticsearch7Configuration config = new Elasticsearch7Configuration(configuration, context.getClassLoader()); + + return new Elasticsearch7DynamicSource(format, config, TableSchemaUtils.getPhysicalSchema(schema), + new ElasticsearchLookupOptions.Builder().build()); + } + @Override public DynamicTableSink createDynamicTableSink(Context context) { TableSchema tableSchema = context.getCatalogTable().getSchema(); @@ -161,6 +193,48 @@ private void validate(Elasticsearch7Configuration config, Configuration original } } + private void validateSource(Elasticsearch7Configuration config, Configuration originalConfiguration) { + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + validate( + config.getScrollMaxSize().map(scrollMaxSize -> scrollMaxSize >= 1).orElse(true), + () -> String.format( + "'%s' must be at least 1. Got: %s", + SCROLL_MAX_SIZE_OPTION.key(), + config.getScrollMaxSize().get()) + ); + validate(config.getScrollTimeout().map(scrollTimeout -> scrollTimeout >= 1).orElse(true), + () -> String.format( + "'%s' must be at least 1. Got: %s", + SCROLL_TIMEOUT_OPTION.key(), + config.getScrollTimeout().get()) + ); + long cacheMaxSize = config.getCacheMaxSize(); + validate( + cacheMaxSize == -1 || cacheMaxSize >= 1, + () -> String.format( + "'%s' must be at least 1. Got: %s", + LOOKUP_CACHE_MAX_ROWS.key(), + cacheMaxSize) + ); + validate( + config.getCacheExpiredMs().getSeconds() >= 1, + () -> String.format( + "'%s' must be at least 1. Got: %s", + LOOKUP_CACHE_TTL.key(), + config.getCacheExpiredMs().getSeconds()) + ); + validate( + config.getMaxRetryTimes() >= 1, + () -> String.format( + "'%s' must be at least 1. Got: %s", + LOOKUP_MAX_RETRIES.key(), + config.getMaxRetryTimes()) + ); + } + private static void validate(boolean condition, Supplier message) { if (!condition) { throw new ValidationException(message.get()); @@ -181,4 +255,5 @@ public Set> requiredOptions() { public Set> optionalOptions() { return optionalOptions; } + } diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java index 37a2d598..329f866e 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java @@ -18,7 +18,12 @@ package org.apache.flink.streaming.connectors.elasticsearch7; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple2; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchInputSplit; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.apache.flink.util.Preconditions; @@ -27,6 +32,8 @@ import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; @@ -56,7 +63,9 @@ public class Elasticsearch7ApiCallBridge /** The factory to configure the rest client. */ private final RestClientFactory restClientFactory; - Elasticsearch7ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { + private final ObjectMapper jsonParser = new ObjectMapper(); + + public Elasticsearch7ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); this.httpHosts = httpHosts; this.restClientFactory = Preconditions.checkNotNull(restClientFactory); @@ -82,6 +91,33 @@ public BulkProcessor.Builder createBulkProcessorBuilder( listener); } + @Override + public ElasticsearchInputSplit[] createInputSplitsInternal( + RestHighLevelClient client, + String index, + String type, + int minNumSplits) { + return new ElasticsearchInputSplit[0]; + } + + @Override + public Tuple2 search(RestHighLevelClient client, SearchRequest searchRequest) + throws IOException { + return null; + } + + @Override + public Tuple2 scroll( + RestHighLevelClient client, + SearchScrollRequest searchScrollRequest) throws IOException { + return null; + } + + @Override + public void close(RestHighLevelClient client) throws IOException { + + } + @Override public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { if (!bulkItemResponse.isFailed()) { diff --git a/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 10e4846e..eb17142b 100644 --- a/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7DynamicSinkFactory +org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7DynamicTableFactory diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java similarity index 95% rename from flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java index 26c0df17..f2b8ef96 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java @@ -26,7 +26,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ -public class Elasticsearch7DynamicSinkFactoryTest extends ElasticsearchDynamicSinkFactoryBaseTest { +public class Elasticsearch7DynamicTableFactoryTest extends ElasticsearchDynamicSinkFactoryBaseTest { @Override ElasticsearchDynamicSinkFactoryBase createSinkFactory() { return new Elasticsearch7DynamicSinkFactory(); diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 7489b98f..ae7e70e3 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -37,6 +37,7 @@ 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.test.util.AbstractTestBase; import org.apache.flink.types.RowKind; import org.apache.flink.util.TestLogger; @@ -63,22 +64,15 @@ import java.util.Map; import java.util.Optional; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7DynamicTableTestBase.elasticsearchContainer; import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; import static org.apache.flink.table.api.Expressions.row; import static org.assertj.core.api.Assertions.assertThat; /** IT tests for {@link Elasticsearch7DynamicSink}. */ -public class Elasticsearch7DynamicSinkITCase extends TestLogger { +public class Elasticsearch7DynamicSinkITCase extends Elasticsearch7DynamicTableTestBase { - @ClassRule - public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)); - @SuppressWarnings("deprecation") - protected final RestHighLevelClient getClient() { - return new RestHighLevelClient( - RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); - } @Test public void testWritingDocuments() throws Exception { @@ -107,7 +101,7 @@ public void testWritingDocuments() throws Exception { LocalDateTime.parse("2012-12-12T12:12:12"))); String index = "writing-documents"; - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); SinkFunctionProvider sinkRuntimeProvider = (SinkFunctionProvider) diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSourceITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSourceITCase.java new file mode 100644 index 00000000..f0676972 --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSourceITCase.java @@ -0,0 +1,82 @@ +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import static org.junit.Assert.assertTrue; + +public class Elasticsearch7DynamicSourceITCase extends Elasticsearch7DynamicTableTestBase { + + private final String scanKeywordIndex = "scan-keyword-index"; + private final String scanKeywordType = "scan-keyword-type"; + + @Before + public void before() throws IOException, ExecutionException, InterruptedException { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + assertTrue(createIndex(getClient(), scanKeywordIndex, scanKeywordType, "keyword")); + insertData(tEnv, scanKeywordIndex, scanKeywordType); + } + + @Test + public void testElasticsearchSource() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + tEnv.executeSql("CREATE TABLE esTableSource (" + + "a BIGINT NOT NULL,\n" + + "b STRING NOT NULL,\n" + + "c FLOAT,\n" + + "d TINYINT NOT NULL,\n" + + "e TIME,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (c, d) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format("'%s'='%s',\n", ElasticsearchConnectorOptions.INDEX_OPTION.key(), scanKeywordIndex) + + String.format("'%s'='%s',\n", ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s',\n", ElasticsearchConnectorOptions.SCROLL_MAX_SIZE_OPTION.key(), 10) + + String.format("'%s'='%s'\n", ElasticsearchConnectorOptions.SCROLL_TIMEOUT_OPTION.key(), 1000) + + ")"); + + Iterator collected = tEnv.executeSql("SELECT a, b, c, d, e, f, g, h FROM esTableSource").collect(); + List result = Lists.newArrayList(collected).stream() + .map(Row::toString) + .sorted() + .collect(Collectors.toList()); + + List expected = + Stream.of( + "1,A B,12.1,2,00:00:12,2003-10-20,2012-12-12T12:12:12,3", + "1,A,12.11,2,00:00:12,2003-10-20,2012-12-12T12:12:12,3", + "1,A,12.12,2,00:00:12,2003-10-20,2012-12-12T12:12:12,3", + "2,B,12.13,3,00:00:12,2003-10-21,2012-12-12T12:12:13,4", + "3,C,12.14,4,00:00:12,2003-10-22,2012-12-12T12:12:14,5", + "4,D,12.15,5,00:00:12,2003-10-23,2012-12-12T12:12:15,6", + "5,E,12.16,6,00:00:12,2003-10-24,2012-12-12T12:12:16,7") + .sorted() + .collect( + Collectors.toList() + ); + assertEquals(expected, result); + } + +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java similarity index 91% rename from flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java index 3ff21ed1..d0d68477 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java @@ -35,13 +35,13 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; -/** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ -public class Elasticsearch7DynamicSinkFactoryTest extends TestLogger { +/** Tests for validation in {@link Elasticsearch7DynamicTableFactory}. */ +public class Elasticsearch7DynamicTableFactoryTest extends TestLogger { @Rule public ExpectedException thrown = ExpectedException.none(); @Test public void validateEmptyConfiguration() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); thrown.expectMessage( @@ -56,7 +56,7 @@ public void validateEmptyConfiguration() { @Test public void validateWrongIndex() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); thrown.expectMessage("'index' must not be empty"); @@ -69,7 +69,7 @@ public void validateWrongIndex() { @Test public void validateWrongHosts() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); thrown.expectMessage( @@ -80,7 +80,7 @@ public void validateWrongHosts() { @Test public void validateWrongFlushSize() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); thrown.expectMessage( @@ -99,7 +99,7 @@ public void validateWrongFlushSize() { @Test public void validateWrongRetries() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); @@ -118,7 +118,7 @@ public void validateWrongRetries() { @Test public void validateWrongMaxActions() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); @@ -136,7 +136,7 @@ public void validateWrongMaxActions() { @Test public void validateWrongBackoffDelay() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); @@ -154,7 +154,7 @@ public void validateWrongBackoffDelay() { @Test public void validatePrimaryKeyOnIllegalColumn() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); thrown.expectMessage( @@ -216,7 +216,7 @@ public void validatePrimaryKeyOnIllegalColumn() { @Test public void validateWrongCredential() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); thrown.expectMessage( diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableTestBase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableTestBase.java new file mode 100644 index 00000000..741f271b --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableTestBase.java @@ -0,0 +1,168 @@ +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.connector.elasticsearch.test.DockerImageVersions; +import org.apache.flink.test.util.AbstractTestBase; + +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.client.indices.CreateIndexRequest; +import org.elasticsearch.client.indices.CreateIndexResponse; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.junit.ClassRule; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Random; +import java.util.concurrent.ExecutionException; + +import static org.apache.flink.table.api.Expressions.row; + +public class Elasticsearch7DynamicTableTestBase extends AbstractTestBase { + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)); + + public final RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); + } + + public boolean createIndex(RestHighLevelClient client, String index, String type, String stringType) throws + IOException { + // create index + CreateIndexRequest request = new CreateIndexRequest(index); + request.settings( + Settings.builder() + .put("index.number_of_shards", 3) + .put("index.number_of_replicas", 0) + ); + + //set the string field if 'b' to keyword or text + /** + * request.mapping( + * type, + * "{\n" + + * " \"properties\": {\n" + + * " \"b\": {\n" + + * " \"type\": \"text\"\n" + + * " }\n" + + * " }\n" + + * "}", + * XContentType.JSON + * ); + */ + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.startObject(); + { + builder.startObject("properties"); + { + builder.startObject("b"); + { + builder.field("type", stringType); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + request.mapping(builder); + CreateIndexResponse createIndexResponse = client.indices().create(request, RequestOptions.DEFAULT); + return createIndexResponse.isAcknowledged(); + } + + public void insertData(StreamTableEnvironment tEnv, String index, String type) throws ExecutionException, InterruptedException { + String sinkTable = "esTable" + Math.abs(new Random().nextInt()); + + tEnv.executeSql("CREATE TABLE " + sinkTable + "(" + + "a BIGINT NOT NULL,\n" + + "b STRING NOT NULL,\n" + + "c FLOAT,\n" + + "d TINYINT NOT NULL,\n" + + "e TIME,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (c, d) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format("'%s'='%s',\n", ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format("'%s'='%s',\n", ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s'\n", ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); + + tEnv.fromValues( + row( + 1L, + "A B", + 12.10f, + (byte) 2, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 1L, + "A", + 12.11f, + (byte) 2, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 1L, + "A", + 12.12f, + (byte) 2, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + "B", + 12.13f, + (byte) 3, + LocalTime.ofNanoOfDay(12346L * 1_000_000L), + LocalDate.ofEpochDay(12346), + LocalDateTime.parse("2012-12-12T12:12:13")), + row( + 3L, + "C", + 12.14f, + (byte) 4, + LocalTime.ofNanoOfDay(12347L * 1_000_000L), + LocalDate.ofEpochDay(12347), + LocalDateTime.parse("2012-12-12T12:12:14")), + row( + 4L, + "D", + 12.15f, + (byte) 5, + LocalTime.ofNanoOfDay(12348L * 1_000_000L), + LocalDate.ofEpochDay(12348), + LocalDateTime.parse("2012-12-12T12:12:15")), + row( + 5L, + "E", + 12.16f, + (byte) 6, + LocalTime.ofNanoOfDay(12349L * 1_000_000L), + LocalDate.ofEpochDay(12349), + LocalDateTime.parse("2012-12-12T12:12:16")) + ).executeInsert(sinkTable) + .getJobClient() + .get() + .getJobExecutionResult() + .get(); + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index 2baa6eed..3bc306b2 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -45,7 +45,7 @@ /** IT cases for the {@link ElasticsearchSink}. */ public class ElasticsearchSinkITCase - extends ElasticsearchSinkTestBase { + extends ElasticsearchSinkTestBase { private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkITCase.class); @@ -60,6 +60,7 @@ protected final RestHighLevelClient getClient() { RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); } + @Test public void testElasticsearchSink() throws Exception { runElasticsearchSinkTest(); @@ -95,7 +96,7 @@ public void testInvalidElasticsearchCluster() throws Exception { protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( int bulkFlushMaxActions, - List httpHosts, + List httpHosts, ElasticsearchSinkFunction> elasticsearchSinkFunction) { ElasticsearchSink.Builder> builder = @@ -107,7 +108,7 @@ public void testInvalidElasticsearchCluster() throws Exception { protected ElasticSearchInputFormatBase createElasticsearchInputFormat( Map userConfig, - DeserializationSchema deserializationSchema, + DeserializationSchema deserializationSchema, String[] fieldNames, String index, String type, From 255ca0ca611b6f5c52dc7b5b6296cfbecb20e2de Mon Sep 17 00:00:00 2001 From: chengxy Date: Wed, 10 May 2023 14:35:54 +0800 Subject: [PATCH 3/7] add es7 tableFactoryTest --- ...Elasticsearch7DynamicTableFactoryTest.java | 86 +++++++++++++++++++ 1 file changed, 86 insertions(+) diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java index d0d68477..e4d09cc7 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; @@ -78,6 +79,91 @@ public void validateWrongHosts() { context().withOption("index", "MyIndex").withOption("hosts", "wrong-host").build()); } + @Test + public void validateWrongScrollMaxSize() { + Elasticsearch7DynamicTableFactory tableFactory = new Elasticsearch7DynamicTableFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'scan.scroll.max-size' must be at least 1. Got: 0"); + tableFactory.createDynamicTableSource( + context() + .withSchema(ResolvedSchema.of(Column.physical("a", DataTypes.TIME()))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchConnectorOptions.SCROLL_MAX_SIZE_OPTION.key(), "0") + .build() + ); + } + + @Test + public void validateWrongScrollTimeout() { + Elasticsearch7DynamicTableFactory tableFactory = new Elasticsearch7DynamicTableFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'scan.scroll.timeout' must be at least 1. Got: 0"); + tableFactory.createDynamicTableSource( + context() + .withSchema(ResolvedSchema.of(Column.physical("a", DataTypes.TIME()))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchConnectorOptions.SCROLL_TIMEOUT_OPTION.key(), "0") + .build() + ); + } + + @Test + public void validateWrongCacheMaxSize() { + Elasticsearch7DynamicTableFactory tableFactory = new Elasticsearch7DynamicTableFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'lookup.cache.max-rows' must be at least 1. Got: 0"); + tableFactory.createDynamicTableSource( + context() + .withSchema(ResolvedSchema.of(Column.physical("a", DataTypes.TIME()))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchConnectorOptions.LOOKUP_CACHE_MAX_ROWS.key(), "0") + .build() + ); + } + + @Test + public void validateWrongCacheTTL() { + Elasticsearch7DynamicTableFactory tableFactory = new Elasticsearch7DynamicTableFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'lookup.cache.ttl' must be at least 1. Got: 0"); + tableFactory.createDynamicTableSource( + context() + .withSchema(ResolvedSchema.of(Column.physical("a", DataTypes.TIME()))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchConnectorOptions.LOOKUP_CACHE_TTL.key(), "0") + .build() + ); + } + + @Test + public void validateWrongMaxRetries() { + Elasticsearch7DynamicTableFactory tableFactory = new Elasticsearch7DynamicTableFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'lookup.max-retries' must be at least 1. Got: 0"); + tableFactory.createDynamicTableSource( + context() + .withSchema(ResolvedSchema.of(Column.physical("a", DataTypes.TIME()))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchConnectorOptions.LOOKUP_MAX_RETRIES.key(), "0") + .build() + ); + } + @Test public void validateWrongFlushSize() { Elasticsearch7DynamicTableFactory sinkFactory = new Elasticsearch7DynamicTableFactory(); From df31e9a6da3727abcd1700f0c1f640b4581a1370 Mon Sep 17 00:00:00 2001 From: chengxy Date: Fri, 12 May 2023 17:13:12 +0800 Subject: [PATCH 4/7] support projectpushdown --- .../table/Elasticsearch7DynamicSource.java | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java index 7580483e..f9aab9fd 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java @@ -16,8 +16,15 @@ import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.FieldsDataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.table.utils.TableSchemaUtils; import org.apache.flink.util.Preconditions; +import static org.apache.flink.table.utils.TableSchemaUtils.containsPhysicalColumnsOnly; +import static org.apache.flink.util.Preconditions.checkArgument; + @Internal public class Elasticsearch7DynamicSource implements ScanTableSource, LookupTableSource, SupportsProjectionPushDown { @@ -96,4 +103,25 @@ public String asSummaryString() { public boolean supportsNestedProjection() { return false; } + + @Override + public void applyProjection(int[][] projectedFields) { + this.physicalSchema = projectSchema(physicalSchema, projectedFields); + } + + public static TableSchema projectSchema(TableSchema tableSchema, int[][] projectedFields) { + checkArgument( + containsPhysicalColumnsOnly(tableSchema), + "Projection is only supported for physical columns."); + TableSchema.Builder builder = TableSchema.builder(); + + FieldsDataType fields = + (FieldsDataType) + DataTypeUtils.projectRow(tableSchema.toRowDataType(), projectedFields); + RowType topFields = (RowType) fields.getLogicalType(); + for (int i = 0; i < topFields.getFieldCount(); i++) { + builder.field(topFields.getFieldNames().get(i), fields.getChildren().get(i)); + } + return builder.build(); + } } From 82d4e9bbf246ba17b2b40abeedc1d0f61902d0af Mon Sep 17 00:00:00 2001 From: complone Date: Sat, 13 May 2023 20:06:20 +0800 Subject: [PATCH 5/7] support filterpushdown and limitpushdown --- flink-connector-elasticsearch-base/pom.xml | 9 + .../ElasticSearchInputFormatBase.java | 27 ++- .../ElasticsearchApiCallBridge.java | 11 +- .../ElasticsearchInputSplit.java | 3 +- .../table/ElasticsearchConnectorOptions.java | 41 ++-- .../table/ElasticsearchLookupOptions.java | 24 +-- .../ElasticsearchRowDataLookupFunction.java | 47 +++-- .../ElasticsearchSinkBaseTest.java | 12 +- .../ElasticsearchSinkTestBase.java | 86 ++++---- .../Elasticsearch6ApiCallBridge.java | 14 +- .../table/Elasticsearch7DynamicSource.java | 166 +++++++++++++-- .../Elasticsearch7DynamicTableFactory.java | 71 +++---- .../elasticsearch/table/FilterUtils.java | 82 ++++++++ .../ElasticSearch7InputFormat.java | 37 +++- .../Elasticsearch7ApiCallBridge.java | 21 +- .../Elasticsearch7DynamicSinkITCase.java | 10 - .../Elasticsearch7DynamicSourceITCase.java | 83 ++++---- ...Elasticsearch7DynamicTableFactoryTest.java | 51 +++-- .../Elasticsearch7DynamicTableTestBase.java | 191 +++++++++--------- .../ElasticsearchSinkITCase.java | 31 ++- pom.xml | 3 + 21 files changed, 624 insertions(+), 396 deletions(-) create mode 100644 flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/FilterUtils.java diff --git a/flink-connector-elasticsearch-base/pom.xml b/flink-connector-elasticsearch-base/pom.xml index 0f8e8590..3337bfb6 100644 --- a/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connector-elasticsearch-base/pom.xml @@ -66,6 +66,15 @@ under the License. true + + + org.apache.flink + flink-table-common + ${flink.version} + provided + true + + diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticSearchInputFormatBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticSearchInputFormatBase.java index e8b57a61..7b93cd50 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticSearchInputFormatBase.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticSearchInputFormatBase.java @@ -11,11 +11,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.InputSplitAssigner; -import org.elasticsearch.action.search.SearchScrollRequest; -import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.common.Strings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.Scroll; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -29,9 +29,8 @@ import static org.apache.flink.shaded.curator5.com.google.common.base.Preconditions.checkNotNull; @Internal -public class ElasticSearchInputFormatBase extends RichInputFormat - implements ResultTypeQueryable { - +public class ElasticSearchInputFormatBase + extends RichInputFormat implements ResultTypeQueryable { private static final Logger LOG = LoggerFactory.getLogger(ElasticSearchInputFormatBase.class); private static final long serialVersionUID = 1L; @@ -53,15 +52,11 @@ public class ElasticSearchInputFormatBase extends R private int nextRecordIndex = 0; private long currentReadCount = 0L; - /** - * Call bridge for different version-specific. - */ + /** Call bridge for different version-specific. */ private final ElasticsearchApiCallBridge callBridge; private final Map userConfig; - /** - * Elasticsearch client created using the call bridge. - */ + /** Elasticsearch client created using the call bridge. */ private transient C client; public ElasticSearchInputFormatBase( @@ -91,8 +86,8 @@ public ElasticSearchInputFormatBase( this.scrollTimeout = scrollTimeout; this.scrollSize = scrollSize; - } + @Override public TypeInformation getProducedType() { return deserializationSchema.getProducedType(); @@ -142,7 +137,7 @@ public void open(ElasticsearchInputSplit split) throws IOException { } else { size = scrollSize; } - //elasticsearch default value is 10 here. + // elasticsearch default value is 10 here. searchSourceBuilder.size(size); searchSourceBuilder.fetchSource(fieldNames, null); if (predicate != null) { @@ -172,8 +167,10 @@ public boolean reachedEnd() throws IOException { return true; } - // SearchResponse can be InternalSearchHits.empty(), and the InternalSearchHit[] EMPTY = new InternalSearchHit[0] - if (currentScrollWindowHits.length != 0 && nextRecordIndex > currentScrollWindowHits.length - 1) { + // SearchResponse can be InternalSearchHits.empty(), and the InternalSearchHit[] EMPTY = new + // InternalSearchHit[0] + if (currentScrollWindowHits.length != 0 + && nextRecordIndex > currentScrollWindowHits.length - 1) { fetchNextScrollWindow(); } diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index a49f24f8..29784c1e 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -19,7 +19,6 @@ package org.apache.flink.streaming.connectors.elasticsearch; import org.apache.flink.annotation.Internal; - import org.apache.flink.api.java.tuple.Tuple2; import org.elasticsearch.action.bulk.BulkItemResponse; @@ -31,7 +30,6 @@ import java.io.IOException; import java.io.Serializable; -import java.util.Map; import java.util.concurrent.atomic.AtomicLong; /** @@ -66,11 +64,13 @@ public interface ElasticsearchApiCallBridge extends Ser */ BulkProcessor.Builder createBulkProcessorBuilder(C client, BulkProcessor.Listener listener); - ElasticsearchInputSplit[] createInputSplitsInternal(C client, String index, String type, int minNumSplits); + ElasticsearchInputSplit[] createInputSplitsInternal( + C client, String index, String type, int minNumSplits); Tuple2 search(C client, SearchRequest searchRequest) throws IOException; - Tuple2 scroll(C client, SearchScrollRequest searchScrollRequest) throws IOException; + Tuple2 scroll(C client, SearchScrollRequest searchScrollRequest) + throws IOException; void close(C client) throws IOException; @@ -78,7 +78,8 @@ public interface ElasticsearchApiCallBridge extends Ser * 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). + * @return the extracted {@link Throwable} from the response ({@code null} is the response is + * successful). */ @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchInputSplit.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchInputSplit.java index 07d439d9..86e787e5 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchInputSplit.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchInputSplit.java @@ -15,7 +15,8 @@ public class ElasticsearchInputSplit extends LocatableInputSplit { /** Index will split diffirent shards when index created. */ private final int shard; - public ElasticsearchInputSplit(int splitNumber, String[] hostnames, String index, String type, int shard) { + public ElasticsearchInputSplit( + int splitNumber, String[] hostnames, String index, String type, int shard) { super(splitNumber, hostnames); this.index = index; this.type = type; diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java index 6b53184d..2e0d30db 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -82,13 +82,15 @@ public class ElasticsearchConnectorOptions { ConfigOptions.key("scan.scroll.max-size") .intType() .noDefaultValue() - .withDescription("Maximum number of hits to be returned with each Elasticsearch scroll request"); + .withDescription( + "Maximum number of hits to be returned with each Elasticsearch scroll request"); public static final ConfigOption SCROLL_TIMEOUT_OPTION = ConfigOptions.key("scan.scroll.timeout") .durationType() .noDefaultValue() - .withDescription("Amount of time Elasticsearch will keep the search context alive for scroll requests"); + .withDescription( + "Amount of time Elasticsearch will keep the search context alive for scroll requests"); public static final ConfigOption FAILURE_HANDLER_OPTION = ConfigOptions.key("failure-handler") @@ -166,23 +168,24 @@ public class ElasticsearchConnectorOptions { + "Please refer to the documentation on formats for more details."); // look up config options - public static final ConfigOption LOOKUP_CACHE_MAX_ROWS = ConfigOptions - .key("lookup.cache.max-rows") - .longType() - .defaultValue(-1L) - .withDescription("the max number of rows of lookup cache, over this value, the oldest rows will " + - "be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " + - "specified. Cache is not enabled as default."); - public static final ConfigOption LOOKUP_CACHE_TTL = ConfigOptions - .key("lookup.cache.ttl") - .durationType() - .defaultValue(Duration.ofSeconds(10)) - .withDescription("the cache time to live."); - public static final ConfigOption LOOKUP_MAX_RETRIES = ConfigOptions - .key("lookup.max-retries") - .intType() - .defaultValue(3) - .withDescription("the max retry times if lookup database failed."); + public static final ConfigOption LOOKUP_CACHE_MAX_ROWS = + ConfigOptions.key("lookup.cache.max-rows") + .longType() + .defaultValue(-1L) + .withDescription( + "the max number of rows of lookup cache, over this value, the oldest rows will " + + "be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " + + "specified. Cache is not enabled as default."); + public static final ConfigOption LOOKUP_CACHE_TTL = + ConfigOptions.key("lookup.cache.ttl") + .durationType() + .defaultValue(Duration.ofSeconds(10)) + .withDescription("the cache time to live."); + public static final ConfigOption LOOKUP_MAX_RETRIES = + ConfigOptions.key("lookup.max-retries") + .intType() + .defaultValue(3) + .withDescription("the max retry times if lookup database failed."); // -------------------------------------------------------------------------------------------- // Enums // -------------------------------------------------------------------------------------------- diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchLookupOptions.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchLookupOptions.java index f2e5b470..ee0704bb 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchLookupOptions.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchLookupOptions.java @@ -35,41 +35,33 @@ public static Builder builder() { public boolean equals(Object o) { if (o instanceof ElasticsearchLookupOptions) { ElasticsearchLookupOptions options = (ElasticsearchLookupOptions) o; - return Objects.equals(cacheMaxSize, options.cacheMaxSize) && - Objects.equals(cacheExpireMs, options.cacheExpireMs) && - Objects.equals(maxRetryTimes, options.maxRetryTimes); + return Objects.equals(cacheMaxSize, options.cacheMaxSize) + && Objects.equals(cacheExpireMs, options.cacheExpireMs) + && Objects.equals(maxRetryTimes, options.maxRetryTimes); } else { return false; } } - /** - * Builder of {@link ElasticsearchLookupOptions}. - */ + /** Builder of {@link ElasticsearchLookupOptions}. */ public static class Builder { private long cacheMaxSize = -1L; private long cacheExpireMs = -1L; private int maxRetryTimes = 3; - /** - * optional, lookup cache max size, over this value, the old data will be eliminated. - */ + /** optional, lookup cache max size, over this value, the old data will be eliminated. */ public Builder setCacheMaxSize(long cacheMaxSize) { this.cacheMaxSize = cacheMaxSize; return this; } - /** - * optional, lookup cache expire mills, over this time, the old data will expire. - */ + /** optional, lookup cache expire mills, over this time, the old data will expire. */ public Builder setCacheExpireMs(long cacheExpireMs) { this.cacheExpireMs = cacheExpireMs; return this; } - /** - * optional, max retry times for jdbc connector. - */ + /** optional, max retry times for jdbc connector. */ public Builder setMaxRetryTimes(int maxRetryTimes) { this.maxRetryTimes = maxRetryTimes; return this; @@ -79,6 +71,4 @@ public ElasticsearchLookupOptions build() { return new ElasticsearchLookupOptions(cacheMaxSize, cacheExpireMs, maxRetryTimes); } } - - } diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java index 5f307695..67a1f93d 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java @@ -9,10 +9,7 @@ import org.apache.flink.table.data.util.DataFormatConverters.DataFormatConverter; import org.apache.flink.table.functions.FunctionContext; import org.apache.flink.table.functions.TableFunction; - import org.apache.flink.table.types.DataType; -import org.apache.flink.types.Row; - import org.apache.flink.util.Preconditions; import org.elasticsearch.action.search.SearchRequest; @@ -31,16 +28,16 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.elasticsearch.common.cache.CacheBuilder.builder; -public class ElasticsearchRowDataLookupFunction extends TableFunction { +public class ElasticsearchRowDataLookupFunction + extends TableFunction { - private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchRowDataLookupFunction.class); + private static final Logger LOG = + LoggerFactory.getLogger(ElasticsearchRowDataLookupFunction.class); private final DeserializationSchema deserializationSchema; @@ -91,11 +88,14 @@ public ElasticsearchRowDataLookupFunction( this.producedNames = producedNames; this.lookupKeys = lookupKeys; this.converters = new DataFormatConverter[lookupKeys.length]; - Map nameToIndex = IntStream.range(0, producedNames.length).boxed().collect( - Collectors.toMap(i -> producedNames[i], i -> i)); + Map nameToIndex = + IntStream.range(0, producedNames.length) + .boxed() + .collect(Collectors.toMap(i -> producedNames[i], i -> i)); for (int i = 0; i < lookupKeys.length; i++) { Integer position = nameToIndex.get(lookupKeys[i]); - Preconditions.checkArgument(position != null, "Lookup keys %s not selected", Arrays.toString(lookupKeys)); + Preconditions.checkArgument( + position != null, "Lookup keys %s not selected", Arrays.toString(lookupKeys)); converters[i] = DataFormatConverters.getConverterForDataType(producedTypes[position]); } this.callBridge = callBridge; @@ -103,10 +103,16 @@ public ElasticsearchRowDataLookupFunction( @Override public void open(FunctionContext context) throws Exception { - this.cache = cacheMaxSize == -1 || cacheExpireMs == -1 ? null : CacheBuilder.>builder().setExpireAfterWrite(TimeValue.timeValueMillis(cacheExpireMs)) - .setMaximumWeight(cacheMaxSize).build(); + this.cache = + cacheMaxSize == -1 || cacheExpireMs == -1 + ? null + : CacheBuilder.>builder() + .setExpireAfterWrite(TimeValue.timeValueMillis(cacheExpireMs)) + .setMaximumWeight(cacheMaxSize) + .build(); this.client = callBridge.createClient(); - //Set searchRequest in open method in case of amount of calling in eval method when every record comes. + // Set searchRequest in open method in case of amount of calling in eval method when every + // record comes. this.searchRequest = new SearchRequest(index); if (type == null) { searchRequest.types(Strings.EMPTY_ARRAY); @@ -117,7 +123,6 @@ public void open(FunctionContext context) throws Exception { searchSourceBuilder.fetchSource(producedNames, null); } - /** * This is a lookup method which is called by Flink framework in runtime. * @@ -137,7 +142,8 @@ public void eval(Object... keys) { BoolQueryBuilder lookupCondition = new BoolQueryBuilder(); for (int i = 0; i < lookupKeys.length; i++) { - lookupCondition.must(new TermQueryBuilder(lookupKeys[i], converters[i].toExternal(keys[i]))); + lookupCondition.must( + new TermQueryBuilder(lookupKeys[i], converters[i].toExternal(keys[i]))); } searchSourceBuilder.query(lookupCondition); searchRequest.source(searchSourceBuilder); @@ -174,7 +180,8 @@ public void eval(Object... keys) { try { Thread.sleep(1000 * retry); } catch (InterruptedException e1) { - LOG.warn("Interrupted while waiting to retry failed elasticsearch search, aborting"); + LOG.warn( + "Interrupted while waiting to retry failed elasticsearch search, aborting"); throw new RuntimeException(e1); } } @@ -191,12 +198,4 @@ private RowData parseSearchHit(String hit) { return row; } - - - - - - - - } diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 93ad6da9..04fc2442 100644 --- a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -609,10 +609,7 @@ public BulkProcessor.Builder createBulkProcessorBuilder( @Override public ElasticsearchInputSplit[] createInputSplitsInternal( - Client client, - String index, - String type, - int minNumSplits) { + Client client, String index, String type, int minNumSplits) { return new ElasticsearchInputSplit[0]; } @@ -624,15 +621,12 @@ public Tuple2 search(Client client, SearchRequest searchReques @Override public Tuple2 scroll( - Client client, - SearchScrollRequest searchScrollRequest) throws IOException { + Client client, SearchScrollRequest searchScrollRequest) throws IOException { return null; } @Override - public void close(Client client) throws IOException { - - } + public void close(Client client) throws IOException {} @Nullable @Override diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index c769b282..318757a2 100644 --- a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -17,9 +17,6 @@ package org.apache.flink.streaming.connectors.elasticsearch; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.planner.runtime.utils.TestingAppendRowDataSink; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.formats.common.TimestampFormat; @@ -30,6 +27,8 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.planner.runtime.utils.TestingAppendRowDataSink; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.test.util.AbstractTestBase; @@ -107,47 +106,52 @@ protected void runElasticSearchInputFormatTest() throws Exception { RowType schema = (RowType) dataType.getLogicalType(); // pass on missing field - DeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - schema, InternalTypeInfo.of(schema), false, false, TimestampFormat.ISO_8601); - - ElasticSearchInputFormatBase inputFormat = createElasticsearchInputFormat( - userConfig, - (DeserializationSchema) deserializationSchema, - null, - "elasticsearch-sink-test-json-index", - "flink-es-test-type", - 1000, - 10, - null, - 0 - ); + DeserializationSchema deserializationSchema = + new JsonRowDataDeserializationSchema( + schema, + InternalTypeInfo.of(schema), + false, + false, + TimestampFormat.ISO_8601); + + ElasticSearchInputFormatBase inputFormat = + createElasticsearchInputFormat( + userConfig, + (DeserializationSchema) deserializationSchema, + null, + "elasticsearch-sink-test-json-index", + "flink-es-test-type", + 1000, + 10, + null, + 0); DataStream dataStream = env.createInput(inputFormat); TestingAppendRowDataSink sink = new TestingAppendRowDataSink(InternalTypeInfo.of(schema)); dataStream.addSink(sink); env.execute("Elasticsearch Source Test"); - List expected = Arrays.asList( - "+I(message #0)", - "+I(message #1)", - "+I(message #10)", - "+I(message #11)", - "+I(message #12)", - "+I(message #13)", - "+I(message #14)", - "+I(message #15)", - "+I(message #16)", - "+I(message #17)", - "+I(message #18)", - "+I(message #19)", - "+I(message #2)", - "+I(message #3)", - "+I(message #4)", - "+I(message #5)", - "+I(message #6)", - "+I(message #7)", - "+I(message #8)", - "+I(message #9)" - ); + List expected = + Arrays.asList( + "+I(message #0)", + "+I(message #1)", + "+I(message #10)", + "+I(message #11)", + "+I(message #12)", + "+I(message #13)", + "+I(message #14)", + "+I(message #15)", + "+I(message #16)", + "+I(message #17)", + "+I(message #18)", + "+I(message #19)", + "+I(message #2)", + "+I(message #3)", + "+I(message #4)", + "+I(message #5)", + "+I(message #6)", + "+I(message #7)", + "+I(message #8)", + "+I(message #9)"); List results = sink.getJavaAppendResults(); results.sort(String::compareTo); assertEquals(expected, results); @@ -162,8 +166,8 @@ protected abstract ElasticSearchInputFormatBase createElasticsearchInputFormat( long scrollTimeout, int scrollMaxSize, QueryBuilder predicate, - int limit - ) throws Exception; + int limit) + throws Exception; /** * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is {@code diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index 88846b7d..93f3eceb 100644 --- a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -42,7 +42,6 @@ import java.io.IOException; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicLong; /** Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. */ @@ -85,10 +84,7 @@ public BulkProcessor.Builder createBulkProcessorBuilder( @Override public ElasticsearchInputSplit[] createInputSplitsInternal( - RestHighLevelClient client, - String index, - String type, - int minNumSplits) { + RestHighLevelClient client, String index, String type, int minNumSplits) { return new ElasticsearchInputSplit[0]; } @@ -100,15 +96,13 @@ public Tuple2 search(RestHighLevelClient client, SearchRequest @Override public Tuple2 scroll( - RestHighLevelClient client, - SearchScrollRequest searchScrollRequest) throws IOException { + RestHighLevelClient client, SearchScrollRequest searchScrollRequest) + throws IOException { return null; } @Override - public void close(RestHighLevelClient client) throws IOException { - - } + public void close(RestHighLevelClient client) throws IOException {} @Override public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java index f9aab9fd..d1e5e441 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java @@ -2,37 +2,70 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticSearch7InputFormat; import org.apache.flink.streaming.connectors.elasticsearch7.Elasticsearch7ApiCallBridge; import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; +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.catalog.WatermarkSpec; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.InputFormatProvider; import org.apache.flink.table.connector.source.LookupTableSource; import org.apache.flink.table.connector.source.ScanTableSource; import org.apache.flink.table.connector.source.TableFunctionProvider; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.FieldsDataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.table.utils.TableSchemaUtils; import org.apache.flink.util.Preconditions; +import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.ExistsQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.query.QueryStringQueryBuilder; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.index.query.TermsQueryBuilder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; + import static org.apache.flink.table.utils.TableSchemaUtils.containsPhysicalColumnsOnly; import static org.apache.flink.util.Preconditions.checkArgument; @Internal -public class Elasticsearch7DynamicSource implements ScanTableSource, LookupTableSource, - SupportsProjectionPushDown { +public class Elasticsearch7DynamicSource + implements ScanTableSource, LookupTableSource, SupportsProjectionPushDown, SupportsFilterPushDown, SupportsLimitPushDown { private final DecodingFormat> format; private final Elasticsearch7Configuration config; private final ElasticsearchLookupOptions lookupOptions; private TableSchema physicalSchema; + private Set filterableFields; + private Long limit; + private List filterPredicates; public Elasticsearch7DynamicSource( DecodingFormat> format, @@ -43,40 +76,46 @@ public Elasticsearch7DynamicSource( this.config = config; this.physicalSchema = physicalSchema; this.lookupOptions = lookupOptions; + List fieldNameList = Arrays.asList(physicalSchema.getFieldNames()); + this.filterableFields = new HashSet(fieldNameList); } @Override public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext lookupContext) { RestClientFactory restClientFactory = null; if (config.getPathPrefix().isPresent()) { - restClientFactory = new Elasticsearch7DynamicSink.DefaultRestClientFactory(config.getPathPrefix().get()); + restClientFactory = + new Elasticsearch7DynamicSink.DefaultRestClientFactory( + config.getPathPrefix().get()); } else { restClientFactory = restClientBuilder -> {}; } - Elasticsearch7ApiCallBridge elasticsearch7ApiCallBridge = new Elasticsearch7ApiCallBridge( - config.getHosts(), restClientFactory); + Elasticsearch7ApiCallBridge elasticsearch7ApiCallBridge = + new Elasticsearch7ApiCallBridge(config.getHosts(), restClientFactory); // Elasticsearch only support non-nested look up keys String[] lookupKeys = new String[lookupContext.getKeys().length]; - String [] columnNames = physicalSchema.getFieldNames(); + String[] columnNames = physicalSchema.getFieldNames(); for (int i = 0; i < lookupKeys.length; i++) { int[] innerKeyArr = lookupContext.getKeys()[i]; - Preconditions.checkArgument(innerKeyArr.length == 1, "Elasticsearch only support non-nested look up keys"); + Preconditions.checkArgument( + innerKeyArr.length == 1, "Elasticsearch only support non-nested look up keys"); lookupKeys[i] = columnNames[innerKeyArr[0]]; } DataType[] columnDataTypes = physicalSchema.getFieldDataTypes(); - return TableFunctionProvider.of(new ElasticsearchRowDataLookupFunction( - this.format.createRuntimeDecoder(lookupContext, physicalSchema.toRowDataType()), + return TableFunctionProvider.of( + new ElasticsearchRowDataLookupFunction( + this.format.createRuntimeDecoder( + lookupContext, physicalSchema.toRowDataType()), lookupOptions, config.getIndex(), config.getDocumentType(), columnNames, columnDataTypes, lookupKeys, - elasticsearch7ApiCallBridge - )); + elasticsearch7ApiCallBridge)); } @Override @@ -85,8 +124,30 @@ public ChangelogMode getChangelogMode() { } @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { - return null; + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + ElasticSearch7InputFormat.Builder elasticsearchInputformatBuilder = new ElasticSearch7InputFormat.Builder(); + elasticsearchInputformatBuilder.setHttpHosts(config.getHosts()); + + RestClientFactory restClientFactory = null; + if (config.getPathPrefix().isPresent()) { + restClientFactory = new Elasticsearch7DynamicSink.DefaultRestClientFactory(config.getPathPrefix().get()); + } else { + restClientFactory = restClientBuilder -> { }; + } + + elasticsearchInputformatBuilder.setRestClientFactory(restClientFactory); + elasticsearchInputformatBuilder.setDeserializationSchema(this.format.createRuntimeDecoder(runtimeProviderContext, physicalSchema.toRowDataType())); + elasticsearchInputformatBuilder.setFieldNames(physicalSchema.getFieldNames()); + elasticsearchInputformatBuilder.setIndex(config.getIndex()); + elasticsearchInputformatBuilder.setPredicate(assembleQuery(filterPredicates)); + elasticsearchInputformatBuilder.setLimit(limit.intValue()); + config.getScrollMaxSize().ifPresent(elasticsearchInputformatBuilder::setScrollMaxSize); + config.getScrollTimeout().ifPresent(elasticsearchInputformatBuilder::setScrollTimeout); + + + return InputFormatProvider.of( + elasticsearchInputformatBuilder.build() + ); } @Override @@ -109,6 +170,27 @@ public void applyProjection(int[][] projectedFields) { this.physicalSchema = projectSchema(physicalSchema, projectedFields); } + @Override + public Result applyFilters(List filters) { + List acceptedFilters = new ArrayList<>(); + List remainingFilters = new ArrayList<>(); + for (ResolvedExpression expr : filters) { + if (FilterUtils.shouldPushDown(expr, filterableFields)) { + acceptedFilters.add(expr); + } else { + remainingFilters.add(expr); + } + } + this.filterPredicates = acceptedFilters; + return Result.of(acceptedFilters, remainingFilters); + } + + + @Override + public void applyLimit(long limit) { + this.limit = limit; + } + public static TableSchema projectSchema(TableSchema tableSchema, int[][] projectedFields) { checkArgument( containsPhysicalColumnsOnly(tableSchema), @@ -124,4 +206,60 @@ public static TableSchema projectSchema(TableSchema tableSchema, int[][] project } return builder.build(); } + + public static QueryBuilder assembleQuery(List filterPredicates) { + + BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery(); + for (ResolvedExpression resolvedExpression : filterPredicates) { + + if (!(resolvedExpression instanceof CallExpression)) { + continue; + } + CallExpression callExpression = (CallExpression) resolvedExpression; + FunctionDefinition functionDefinition = callExpression.getFunctionDefinition(); + ResolvedExpression valueLiteralExpression = Optional.of(callExpression.getResolvedChildren()).get().get(0); + ResolvedExpression fieldReferenceExpression = Optional.of(callExpression.getResolvedChildren()).get().get(1); + ValueLiteralExpression value = (ValueLiteralExpression) valueLiteralExpression; + FieldReferenceExpression field = (FieldReferenceExpression) fieldReferenceExpression; + if (functionDefinition.equals(BuiltInFunctionDefinitions.AND)) { + boolQueryBuilder.must(QueryBuilders.wildcardQuery(field.getName(), value.asSummaryString())); + } + + if (functionDefinition.equals(BuiltInFunctionDefinitions.OR)) { + boolQueryBuilder.should(QueryBuilders.wildcardQuery(field.getName(), value.asSummaryString())); + } + + if (functionDefinition.equals(BuiltInFunctionDefinitions.NOT)) { + boolQueryBuilder.mustNot(QueryBuilders.wildcardQuery(field.getName(), value.asSummaryString())); + } + + if (functionDefinition.equals(BuiltInFunctionDefinitions.LESS_THAN)) { + RangeQueryBuilder rangeQueryBuilder = QueryBuilders.rangeQuery(field.getName()) + .gte(value.asSummaryString()); + boolQueryBuilder.must(rangeQueryBuilder); + } + + if (functionDefinition.equals(functionDefinition.equals(BuiltInFunctionDefinitions.GREATER_THAN))) { + RangeQueryBuilder rangeQueryBuilder = QueryBuilders.rangeQuery(field.getName()) + .gte( value.asSummaryString()); + boolQueryBuilder.must(rangeQueryBuilder); + } + + if (functionDefinition.equals(BuiltInFunctionDefinitions.EQUALS)) { + TermQueryBuilder termQueryBuilder = QueryBuilders.termQuery(field.getName(), value.asSummaryString()); + boolQueryBuilder.must(termQueryBuilder); + } + + if (functionDefinition.equals(BuiltInFunctionDefinitions.IF_NULL)) { + ExistsQueryBuilder existsQueryBuilder = QueryBuilders.existsQuery(field.getName()); + boolQueryBuilder.must(existsQueryBuilder); + } + + if (functionDefinition.equals(BuiltInFunctionDefinitions.IS_NOT_NULL)) { + ExistsQueryBuilder existsQueryBuilder = QueryBuilders.existsQuery(field.getName()); + boolQueryBuilder.must(existsQueryBuilder); + } + } + return boolQueryBuilder; + } } diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java index 2a9dc4c2..468c10c6 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java @@ -27,7 +27,6 @@ import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -70,7 +69,8 @@ /** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. */ @Internal -public class Elasticsearch7DynamicTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory { +public class Elasticsearch7DynamicTableFactory + implements DynamicTableSourceFactory, DynamicTableSinkFactory { private static final Set> requiredOptions = Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); private static final Set> optionalOptions = @@ -99,15 +99,20 @@ public class Elasticsearch7DynamicTableFactory implements DynamicTableSourceFact public DynamicTableSource createDynamicTableSource(Context context) { TableSchema schema = context.getCatalogTable().getSchema(); - final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); - final DecodingFormat> format = helper.discoverDecodingFormat( - DeserializationFormatFactory.class, FORMAT_OPTION); + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + final DecodingFormat> format = + helper.discoverDecodingFormat(DeserializationFormatFactory.class, FORMAT_OPTION); helper.validate(); Configuration configuration = new Configuration(); context.getCatalogTable().getOptions().forEach(configuration::setString); - Elasticsearch7Configuration config = new Elasticsearch7Configuration(configuration, context.getClassLoader()); + Elasticsearch7Configuration config = + new Elasticsearch7Configuration(configuration, context.getClassLoader()); - return new Elasticsearch7DynamicSource(format, config, TableSchemaUtils.getPhysicalSchema(schema), + return new Elasticsearch7DynamicSource( + format, + config, + TableSchemaUtils.getPhysicalSchema(schema), new ElasticsearchLookupOptions.Builder().build()); } @@ -193,46 +198,43 @@ private void validate(Elasticsearch7Configuration config, Configuration original } } - private void validateSource(Elasticsearch7Configuration config, Configuration originalConfiguration) { + private void validateSource( + Elasticsearch7Configuration config, Configuration originalConfiguration) { config.getHosts(); // validate hosts validate( config.getIndex().length() >= 1, () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); validate( config.getScrollMaxSize().map(scrollMaxSize -> scrollMaxSize >= 1).orElse(true), - () -> String.format( - "'%s' must be at least 1. Got: %s", - SCROLL_MAX_SIZE_OPTION.key(), - config.getScrollMaxSize().get()) - ); - validate(config.getScrollTimeout().map(scrollTimeout -> scrollTimeout >= 1).orElse(true), - () -> String.format( - "'%s' must be at least 1. Got: %s", - SCROLL_TIMEOUT_OPTION.key(), - config.getScrollTimeout().get()) - ); + () -> + String.format( + "'%s' must be at least 1. Got: %s", + SCROLL_MAX_SIZE_OPTION.key(), config.getScrollMaxSize().get())); + validate( + config.getScrollTimeout().map(scrollTimeout -> scrollTimeout >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + SCROLL_TIMEOUT_OPTION.key(), config.getScrollTimeout().get())); long cacheMaxSize = config.getCacheMaxSize(); validate( cacheMaxSize == -1 || cacheMaxSize >= 1, - () -> String.format( - "'%s' must be at least 1. Got: %s", - LOOKUP_CACHE_MAX_ROWS.key(), - cacheMaxSize) - ); + () -> + String.format( + "'%s' must be at least 1. Got: %s", + LOOKUP_CACHE_MAX_ROWS.key(), cacheMaxSize)); validate( config.getCacheExpiredMs().getSeconds() >= 1, - () -> String.format( - "'%s' must be at least 1. Got: %s", - LOOKUP_CACHE_TTL.key(), - config.getCacheExpiredMs().getSeconds()) - ); + () -> + String.format( + "'%s' must be at least 1. Got: %s", + LOOKUP_CACHE_TTL.key(), config.getCacheExpiredMs().getSeconds())); validate( config.getMaxRetryTimes() >= 1, - () -> String.format( - "'%s' must be at least 1. Got: %s", - LOOKUP_MAX_RETRIES.key(), - config.getMaxRetryTimes()) - ); + () -> + String.format( + "'%s' must be at least 1. Got: %s", + LOOKUP_MAX_RETRIES.key(), config.getMaxRetryTimes())); } private static void validate(boolean condition, Supplier message) { @@ -255,5 +257,4 @@ public Set> requiredOptions() { public Set> optionalOptions() { return optionalOptions; } - } diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/FilterUtils.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/FilterUtils.java new file mode 100644 index 00000000..c3221b9b --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/FilterUtils.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.elasticsearch.table; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.util.Preconditions; + +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; + +import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.LOWER; +import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.UPPER; + +/** Utils for source to filter partition or row. */ +public class FilterUtils { + + public static boolean shouldPushDown(ResolvedExpression expr, Set filterableFields) { + if (expr instanceof CallExpression && expr.getChildren().size() == 2) { + return shouldPushDownUnaryExpression( + expr.getResolvedChildren().get(0), filterableFields) + && shouldPushDownUnaryExpression( + expr.getResolvedChildren().get(1), filterableFields); + } + return false; + } + + private static boolean shouldPushDownUnaryExpression( + ResolvedExpression expr, Set filterableFields) { + // validate that type is comparable + if (!isComparable(expr.getOutputDataType().getConversionClass())) { + return false; + } + if (expr instanceof FieldReferenceExpression) { + if (filterableFields.contains(((FieldReferenceExpression) expr).getName())) { + return true; + } + } + + if (expr instanceof ValueLiteralExpression) { + return true; + } + + if (expr instanceof CallExpression && expr.getChildren().size() == 1) { + if (((CallExpression) expr).getFunctionDefinition().equals(UPPER) + || ((CallExpression) expr).getFunctionDefinition().equals(LOWER)) { + return shouldPushDownUnaryExpression( + expr.getResolvedChildren().get(0), filterableFields); + } + } + // other resolved expressions return false + return false; + } + + private static boolean isComparable(Class clazz) { + return Comparable.class.isAssignableFrom(clazz); + } +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticSearch7InputFormat.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticSearch7InputFormat.java index 2f34005e..56ee7d25 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticSearch7InputFormat.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticSearch7InputFormat.java @@ -14,8 +14,8 @@ import java.util.Map; @PublicEvolving -public class ElasticSearch7InputFormat extends - ElasticSearchInputFormatBase { +public class ElasticSearch7InputFormat + extends ElasticSearchInputFormatBase { private static final long serialVersionUID = 1L; @@ -31,7 +31,17 @@ public ElasticSearch7InputFormat( QueryBuilder predicate, int limit) { - super(new Elasticsearch7ApiCallBridge(httpHosts, restClientFactory), userConfig, serializationSchema, fieldNames, index, null, scrollTimeout, scrollSize, predicate, limit); + super( + new Elasticsearch7ApiCallBridge(httpHosts, restClientFactory), + userConfig, + serializationSchema, + fieldNames, + index, + null, + scrollTimeout, + scrollSize, + predicate, + limit); } /** @@ -43,8 +53,7 @@ public ElasticSearch7InputFormat( public static class Builder { private Map userConfig = new HashMap<>(); private List httpHosts; - private RestClientFactory restClientFactory = restClientBuilder -> { - }; + private RestClientFactory restClientFactory = restClientBuilder -> {}; private DeserializationSchema deserializationSchema; private String index; @@ -55,13 +64,13 @@ public static class Builder { private QueryBuilder predicate; private int limit; - public Builder() { - } + public Builder() {} /** * Sets HttpHost which the RestHighLevelClient connects to. * - * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} + * connects to. */ public Builder setHttpHosts(List httpHosts) { this.httpHosts = httpHosts; @@ -137,7 +146,17 @@ public Builder setLimit(int limit) { * @return the created ElasticSearch7RowDataInputFormat. */ public ElasticSearch7InputFormat build() { - return new ElasticSearch7InputFormat(userConfig, httpHosts, restClientFactory, deserializationSchema, fieldNames, index, scrollTimeout, scrollMaxSize, predicate, limit); + return new ElasticSearch7InputFormat( + userConfig, + httpHosts, + restClientFactory, + deserializationSchema, + fieldNames, + index, + scrollTimeout, + scrollMaxSize, + predicate, + limit); } } } diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java index 329f866e..d01e6ad7 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java @@ -19,15 +19,14 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.tuple.Tuple2; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; - import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchInputSplit; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.apache.flink.util.Preconditions; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + import org.apache.http.HttpHost; import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.bulk.BulkItemResponse; @@ -65,7 +64,8 @@ public class Elasticsearch7ApiCallBridge private final ObjectMapper jsonParser = new ObjectMapper(); - public Elasticsearch7ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { + public Elasticsearch7ApiCallBridge( + List httpHosts, RestClientFactory restClientFactory) { Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); this.httpHosts = httpHosts; this.restClientFactory = Preconditions.checkNotNull(restClientFactory); @@ -93,10 +93,7 @@ public BulkProcessor.Builder createBulkProcessorBuilder( @Override public ElasticsearchInputSplit[] createInputSplitsInternal( - RestHighLevelClient client, - String index, - String type, - int minNumSplits) { + RestHighLevelClient client, String index, String type, int minNumSplits) { return new ElasticsearchInputSplit[0]; } @@ -108,15 +105,13 @@ public Tuple2 search(RestHighLevelClient client, SearchRequest @Override public Tuple2 scroll( - RestHighLevelClient client, - SearchScrollRequest searchScrollRequest) throws IOException { + RestHighLevelClient client, SearchScrollRequest searchScrollRequest) + throws IOException { return null; } @Override - public void close(RestHighLevelClient client) throws IOException { - - } + public void close(RestHighLevelClient client) throws IOException {} @Override public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index ae7e70e3..3b5ffebb 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.connector.elasticsearch.test.DockerImageVersions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.table.api.DataTypes; @@ -37,21 +36,14 @@ 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.test.util.AbstractTestBase; import org.apache.flink.types.RowKind; -import org.apache.flink.util.TestLogger; -import org.apache.http.HttpHost; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.client.RequestOptions; -import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.search.SearchHits; -import org.junit.ClassRule; import org.junit.Test; -import org.testcontainers.elasticsearch.ElasticsearchContainer; -import org.testcontainers.utility.DockerImageName; import java.time.Duration; import java.time.LocalDate; @@ -72,8 +64,6 @@ /** IT tests for {@link Elasticsearch7DynamicSink}. */ public class Elasticsearch7DynamicSinkITCase extends Elasticsearch7DynamicTableTestBase { - - @Test public void testWritingDocuments() throws Exception { ResolvedSchema schema = diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSourceITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSourceITCase.java index f0676972..e6d915d2 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSourceITCase.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSourceITCase.java @@ -1,11 +1,11 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + import org.junit.Before; import org.junit.Test; @@ -19,8 +19,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertTrue; - public class Elasticsearch7DynamicSourceITCase extends Elasticsearch7DynamicTableTestBase { private final String scanKeywordIndex = "scan-keyword-index"; @@ -38,45 +36,54 @@ public void before() throws IOException, ExecutionException, InterruptedExceptio public void testElasticsearchSource() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); - tEnv.executeSql("CREATE TABLE esTableSource (" + - "a BIGINT NOT NULL,\n" + - "b STRING NOT NULL,\n" + - "c FLOAT,\n" + - "d TINYINT NOT NULL,\n" + - "e TIME,\n" + - "f DATE,\n" + - "g TIMESTAMP NOT NULL,\n" + - "h as a + 2,\n" + - "PRIMARY KEY (c, d) NOT ENFORCED\n" + - ")\n" + - "WITH (\n" + - String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + - String.format("'%s'='%s',\n", ElasticsearchConnectorOptions.INDEX_OPTION.key(), scanKeywordIndex) + - String.format("'%s'='%s',\n", ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + - String.format("'%s'='%s',\n", ElasticsearchConnectorOptions.SCROLL_MAX_SIZE_OPTION.key(), 10) + - String.format("'%s'='%s'\n", ElasticsearchConnectorOptions.SCROLL_TIMEOUT_OPTION.key(), 1000) + - ")"); + tEnv.executeSql( + "CREATE TABLE esTableSource (" + + "a BIGINT NOT NULL,\n" + + "b STRING NOT NULL,\n" + + "c FLOAT,\n" + + "d TINYINT NOT NULL,\n" + + "e TIME,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (c, d) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), scanKeywordIndex) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://127.0.0.1:9200") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.SCROLL_MAX_SIZE_OPTION.key(), 10) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.SCROLL_TIMEOUT_OPTION.key(), 1000) + + ")"); - Iterator collected = tEnv.executeSql("SELECT a, b, c, d, e, f, g, h FROM esTableSource").collect(); - List result = Lists.newArrayList(collected).stream() - .map(Row::toString) - .sorted() - .collect(Collectors.toList()); + Iterator collected = + tEnv.executeSql("SELECT a, b, c, d, e, f, g, h FROM esTableSource").collect(); + List result = + Lists.newArrayList(collected).stream() + .map(Row::toString) + .sorted() + .collect(Collectors.toList()); List expected = Stream.of( - "1,A B,12.1,2,00:00:12,2003-10-20,2012-12-12T12:12:12,3", - "1,A,12.11,2,00:00:12,2003-10-20,2012-12-12T12:12:12,3", - "1,A,12.12,2,00:00:12,2003-10-20,2012-12-12T12:12:12,3", - "2,B,12.13,3,00:00:12,2003-10-21,2012-12-12T12:12:13,4", - "3,C,12.14,4,00:00:12,2003-10-22,2012-12-12T12:12:14,5", - "4,D,12.15,5,00:00:12,2003-10-23,2012-12-12T12:12:15,6", - "5,E,12.16,6,00:00:12,2003-10-24,2012-12-12T12:12:16,7") + "1,A B,12.1,2,00:00:12,2003-10-20,2012-12-12T12:12:12,3", + "1,A,12.11,2,00:00:12,2003-10-20,2012-12-12T12:12:12,3", + "1,A,12.12,2,00:00:12,2003-10-20,2012-12-12T12:12:12,3", + "2,B,12.13,3,00:00:12,2003-10-21,2012-12-12T12:12:13,4", + "3,C,12.14,4,00:00:12,2003-10-22,2012-12-12T12:12:14,5", + "4,D,12.15,5,00:00:12,2003-10-23,2012-12-12T12:12:15,6", + "5,E,12.16,6,00:00:12,2003-10-24,2012-12-12T12:12:16,7") .sorted() - .collect( - Collectors.toList() - ); + .collect(Collectors.toList()); assertEquals(expected, result); } - } diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java index e4d09cc7..47fbc42f 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactoryTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; @@ -84,16 +83,16 @@ public void validateWrongScrollMaxSize() { Elasticsearch7DynamicTableFactory tableFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); - thrown.expectMessage( - "'scan.scroll.max-size' must be at least 1. Got: 0"); + thrown.expectMessage("'scan.scroll.max-size' must be at least 1. Got: 0"); tableFactory.createDynamicTableSource( context() .withSchema(ResolvedSchema.of(Column.physical("a", DataTypes.TIME()))) .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption(ElasticsearchConnectorOptions.SCROLL_MAX_SIZE_OPTION.key(), "0") - .build() - ); + .build()); } @Test @@ -101,16 +100,16 @@ public void validateWrongScrollTimeout() { Elasticsearch7DynamicTableFactory tableFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); - thrown.expectMessage( - "'scan.scroll.timeout' must be at least 1. Got: 0"); + thrown.expectMessage("'scan.scroll.timeout' must be at least 1. Got: 0"); tableFactory.createDynamicTableSource( context() .withSchema(ResolvedSchema.of(Column.physical("a", DataTypes.TIME()))) .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption(ElasticsearchConnectorOptions.SCROLL_TIMEOUT_OPTION.key(), "0") - .build() - ); + .build()); } @Test @@ -118,16 +117,16 @@ public void validateWrongCacheMaxSize() { Elasticsearch7DynamicTableFactory tableFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); - thrown.expectMessage( - "'lookup.cache.max-rows' must be at least 1. Got: 0"); + thrown.expectMessage("'lookup.cache.max-rows' must be at least 1. Got: 0"); tableFactory.createDynamicTableSource( context() .withSchema(ResolvedSchema.of(Column.physical("a", DataTypes.TIME()))) .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption(ElasticsearchConnectorOptions.LOOKUP_CACHE_MAX_ROWS.key(), "0") - .build() - ); + .build()); } @Test @@ -135,16 +134,16 @@ public void validateWrongCacheTTL() { Elasticsearch7DynamicTableFactory tableFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); - thrown.expectMessage( - "'lookup.cache.ttl' must be at least 1. Got: 0"); + thrown.expectMessage("'lookup.cache.ttl' must be at least 1. Got: 0"); tableFactory.createDynamicTableSource( context() .withSchema(ResolvedSchema.of(Column.physical("a", DataTypes.TIME()))) .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption(ElasticsearchConnectorOptions.LOOKUP_CACHE_TTL.key(), "0") - .build() - ); + .build()); } @Test @@ -152,16 +151,16 @@ public void validateWrongMaxRetries() { Elasticsearch7DynamicTableFactory tableFactory = new Elasticsearch7DynamicTableFactory(); thrown.expect(ValidationException.class); - thrown.expectMessage( - "'lookup.max-retries' must be at least 1. Got: 0"); + thrown.expectMessage("'lookup.max-retries' must be at least 1. Got: 0"); tableFactory.createDynamicTableSource( context() .withSchema(ResolvedSchema.of(Column.physical("a", DataTypes.TIME()))) .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption(ElasticsearchConnectorOptions.LOOKUP_MAX_RETRIES.key(), "0") - .build() - ); + .build()); } @Test diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableTestBase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableTestBase.java index 741f271b..e6f7948c 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableTestBase.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableTestBase.java @@ -1,9 +1,8 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; import org.apache.flink.connector.elasticsearch.test.DockerImageVersions; -import org.apache.flink.test.util.AbstractTestBase; - import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.AbstractTestBase; import org.apache.http.HttpHost; import org.elasticsearch.client.RequestOptions; @@ -38,29 +37,20 @@ public final RestHighLevelClient getClient() { RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); } - public boolean createIndex(RestHighLevelClient client, String index, String type, String stringType) throws - IOException { + public boolean createIndex( + RestHighLevelClient client, String index, String type, String stringType) + throws IOException { // create index CreateIndexRequest request = new CreateIndexRequest(index); request.settings( Settings.builder() .put("index.number_of_shards", 3) - .put("index.number_of_replicas", 0) - ); + .put("index.number_of_replicas", 0)); - //set the string field if 'b' to keyword or text + // set the string field if 'b' to keyword or text /** - * request.mapping( - * type, - * "{\n" + - * " \"properties\": {\n" + - * " \"b\": {\n" + - * " \"type\": \"text\"\n" + - * " }\n" + - * " }\n" + - * "}", - * XContentType.JSON - * ); + * request.mapping( type, "{\n" + " \"properties\": {\n" + " \"b\": {\n" + " \"type\": + * \"text\"\n" + " }\n" + " }\n" + "}", XContentType.JSON ); */ XContentBuilder builder = XContentFactory.jsonBuilder(); builder.startObject(); @@ -77,89 +67,102 @@ public boolean createIndex(RestHighLevelClient client, String index, String type } builder.endObject(); request.mapping(builder); - CreateIndexResponse createIndexResponse = client.indices().create(request, RequestOptions.DEFAULT); + CreateIndexResponse createIndexResponse = + client.indices().create(request, RequestOptions.DEFAULT); return createIndexResponse.isAcknowledged(); } - public void insertData(StreamTableEnvironment tEnv, String index, String type) throws ExecutionException, InterruptedException { + public void insertData(StreamTableEnvironment tEnv, String index, String type) + throws ExecutionException, InterruptedException { String sinkTable = "esTable" + Math.abs(new Random().nextInt()); - tEnv.executeSql("CREATE TABLE " + sinkTable + "(" + - "a BIGINT NOT NULL,\n" + - "b STRING NOT NULL,\n" + - "c FLOAT,\n" + - "d TINYINT NOT NULL,\n" + - "e TIME,\n" + - "f DATE,\n" + - "g TIMESTAMP NOT NULL,\n" + - "h as a + 2,\n" + - "PRIMARY KEY (c, d) NOT ENFORCED\n" + - ")\n" + - "WITH (\n" + - String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + - String.format("'%s'='%s',\n", ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + - String.format("'%s'='%s',\n", ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + - String.format("'%s'='%s'\n", ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + - ")"); + tEnv.executeSql( + "CREATE TABLE " + + sinkTable + + "(" + + "a BIGINT NOT NULL,\n" + + "b STRING NOT NULL,\n" + + "c FLOAT,\n" + + "d TINYINT NOT NULL,\n" + + "e TIME,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (c, d) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://127.0.0.1:9200") + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); tEnv.fromValues( - row( - 1L, - "A B", - 12.10f, - (byte) 2, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")), - row( - 1L, - "A", - 12.11f, - (byte) 2, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")), - row( - 1L, - "A", - 12.12f, - (byte) 2, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")), - row( - 2L, - "B", - 12.13f, - (byte) 3, - LocalTime.ofNanoOfDay(12346L * 1_000_000L), - LocalDate.ofEpochDay(12346), - LocalDateTime.parse("2012-12-12T12:12:13")), - row( - 3L, - "C", - 12.14f, - (byte) 4, - LocalTime.ofNanoOfDay(12347L * 1_000_000L), - LocalDate.ofEpochDay(12347), - LocalDateTime.parse("2012-12-12T12:12:14")), - row( - 4L, - "D", - 12.15f, - (byte) 5, - LocalTime.ofNanoOfDay(12348L * 1_000_000L), - LocalDate.ofEpochDay(12348), - LocalDateTime.parse("2012-12-12T12:12:15")), - row( - 5L, - "E", - 12.16f, - (byte) 6, - LocalTime.ofNanoOfDay(12349L * 1_000_000L), - LocalDate.ofEpochDay(12349), - LocalDateTime.parse("2012-12-12T12:12:16")) - ).executeInsert(sinkTable) + row( + 1L, + "A B", + 12.10f, + (byte) 2, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 1L, + "A", + 12.11f, + (byte) 2, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 1L, + "A", + 12.12f, + (byte) 2, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + "B", + 12.13f, + (byte) 3, + LocalTime.ofNanoOfDay(12346L * 1_000_000L), + LocalDate.ofEpochDay(12346), + LocalDateTime.parse("2012-12-12T12:12:13")), + row( + 3L, + "C", + 12.14f, + (byte) 4, + LocalTime.ofNanoOfDay(12347L * 1_000_000L), + LocalDate.ofEpochDay(12347), + LocalDateTime.parse("2012-12-12T12:12:14")), + row( + 4L, + "D", + 12.15f, + (byte) 5, + LocalTime.ofNanoOfDay(12348L * 1_000_000L), + LocalDate.ofEpochDay(12348), + LocalDateTime.parse("2012-12-12T12:12:15")), + row( + 5L, + "E", + 12.16f, + (byte) 6, + LocalTime.ofNanoOfDay(12349L * 1_000_000L), + LocalDate.ofEpochDay(12349), + LocalDateTime.parse("2012-12-12T12:12:16"))) + .executeInsert(sinkTable) .getJobClient() .get() .getJobExecutionResult() diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index 3bc306b2..9d159b5f 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -60,7 +60,6 @@ protected final RestHighLevelClient getClient() { RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); } - @Test public void testElasticsearchSink() throws Exception { runElasticsearchSinkTest(); @@ -95,9 +94,9 @@ public void testInvalidElasticsearchCluster() throws Exception { @Override protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( - int bulkFlushMaxActions, - List httpHosts, - ElasticsearchSinkFunction> elasticsearchSinkFunction) { + int bulkFlushMaxActions, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); @@ -115,24 +114,24 @@ protected ElasticSearchInputFormatBase createElasticsearchInputFormat( long scrollTimeout, int scrollMaxSize, QueryBuilder predicate, - int limit) throws Exception { + int limit) + throws Exception { List transports = new ArrayList<>(); transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300)); - ElasticSearch7InputFormat builder = new ElasticSearch7InputFormat.Builder() - .setDeserializationSchema(deserializationSchema) - .setFieldNames(fieldNames) - .setIndex(index) - .setScrollTimeout(scrollTimeout) - .setScrollMaxSize(scrollMaxSize) - .setPredicate(predicate) - .setLimit(limit) - .build(); + ElasticSearch7InputFormat builder = + new ElasticSearch7InputFormat.Builder() + .setDeserializationSchema(deserializationSchema) + .setFieldNames(fieldNames) + .setIndex(index) + .setScrollTimeout(scrollTimeout) + .setScrollMaxSize(scrollMaxSize) + .setPredicate(predicate) + .setLimit(limit) + .build(); return builder; } - - @Override protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( diff --git a/pom.xml b/pom.xml index 235dce68..0554ce59 100644 --- a/pom.xml +++ b/pom.xml @@ -408,6 +408,9 @@ under the License. org.apache.maven.plugins maven-checkstyle-plugin + + true + com.diffplug.spotless From 1335cce55dc04ce55826af35a26d13d63f1335ee Mon Sep 17 00:00:00 2001 From: complone Date: Sat, 13 May 2023 20:19:24 +0800 Subject: [PATCH 6/7] fix bug boolQueryBuilder --- .../table/Elasticsearch7DynamicSource.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java index d1e5e441..b91857c1 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java @@ -222,42 +222,42 @@ public static QueryBuilder assembleQuery(List filterPredicat ValueLiteralExpression value = (ValueLiteralExpression) valueLiteralExpression; FieldReferenceExpression field = (FieldReferenceExpression) fieldReferenceExpression; if (functionDefinition.equals(BuiltInFunctionDefinitions.AND)) { - boolQueryBuilder.must(QueryBuilders.wildcardQuery(field.getName(), value.asSummaryString())); + boolQueryBuilder = boolQueryBuilder.must(QueryBuilders.wildcardQuery(field.getName(), value.asSummaryString())); } if (functionDefinition.equals(BuiltInFunctionDefinitions.OR)) { - boolQueryBuilder.should(QueryBuilders.wildcardQuery(field.getName(), value.asSummaryString())); + boolQueryBuilder = boolQueryBuilder.should(QueryBuilders.wildcardQuery(field.getName(), value.asSummaryString())); } if (functionDefinition.equals(BuiltInFunctionDefinitions.NOT)) { - boolQueryBuilder.mustNot(QueryBuilders.wildcardQuery(field.getName(), value.asSummaryString())); + boolQueryBuilder = boolQueryBuilder.mustNot(QueryBuilders.wildcardQuery(field.getName(), value.asSummaryString())); } if (functionDefinition.equals(BuiltInFunctionDefinitions.LESS_THAN)) { RangeQueryBuilder rangeQueryBuilder = QueryBuilders.rangeQuery(field.getName()) .gte(value.asSummaryString()); - boolQueryBuilder.must(rangeQueryBuilder); + boolQueryBuilder = boolQueryBuilder.must(rangeQueryBuilder); } if (functionDefinition.equals(functionDefinition.equals(BuiltInFunctionDefinitions.GREATER_THAN))) { RangeQueryBuilder rangeQueryBuilder = QueryBuilders.rangeQuery(field.getName()) .gte( value.asSummaryString()); - boolQueryBuilder.must(rangeQueryBuilder); + boolQueryBuilder = boolQueryBuilder.must(rangeQueryBuilder); } if (functionDefinition.equals(BuiltInFunctionDefinitions.EQUALS)) { TermQueryBuilder termQueryBuilder = QueryBuilders.termQuery(field.getName(), value.asSummaryString()); - boolQueryBuilder.must(termQueryBuilder); + boolQueryBuilder = boolQueryBuilder.must(termQueryBuilder); } if (functionDefinition.equals(BuiltInFunctionDefinitions.IF_NULL)) { ExistsQueryBuilder existsQueryBuilder = QueryBuilders.existsQuery(field.getName()); - boolQueryBuilder.must(existsQueryBuilder); + boolQueryBuilder = boolQueryBuilder.must(existsQueryBuilder); } if (functionDefinition.equals(BuiltInFunctionDefinitions.IS_NOT_NULL)) { ExistsQueryBuilder existsQueryBuilder = QueryBuilders.existsQuery(field.getName()); - boolQueryBuilder.must(existsQueryBuilder); + boolQueryBuilder = boolQueryBuilder.must(existsQueryBuilder); } } return boolQueryBuilder; From b7a7cee95146e2c605b43cac51b1c90cdd490d23 Mon Sep 17 00:00:00 2001 From: windwheel Date: Sat, 6 Jan 2024 10:06:46 +0800 Subject: [PATCH 7/7] fix es7 conflict --- .../ElasticsearchRowDataLookupFunction.java | 75 +++---------------- .../ElasticsearchSinkTestBase.java | 36 ++++----- .../ElasticsearchSinkITCase.java | 6 +- .../table/Elasticsearch7DynamicSource.java | 1 + .../Elasticsearch7DynamicTableFactory.java | 18 +---- .../Elasticsearch7ApiCallBridge.java | 18 +++++ .../ElasticsearchSinkITCase.java | 4 +- 7 files changed, 57 insertions(+), 101 deletions(-) diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java index f711cf6c..e4a77836 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchRowDataLookupFunction.java @@ -157,6 +157,17 @@ public Collection lookup(RowData keyRow) { lookupCondition.must( new TermQueryBuilder(lookupKeys[i], converters[i].toExternal(keyRow, i))); } + + if (cache != null) { + List cachedRows = cache.get(keyRow); + if (cachedRows != null) { + for (RowData cachedRow : cachedRows) { + collect(cachedRow); + } + return new ArrayList<>(); + } + } + searchSourceBuilder.query(lookupCondition); searchRequest.source(searchSourceBuilder); @@ -189,70 +200,6 @@ public Collection lookup(RowData keyRow) { } return Collections.emptyList(); } - /** - * This is a lookup method which is called by Flink framework in runtime. - * - * @param keys lookup keys - */ - public void eval(Object... keys) { - RowData keyRow = GenericRowData.of(keys); - if (cache != null) { - List cachedRows = cache.get(keyRow); - if (cachedRows != null) { - for (RowData cachedRow : cachedRows) { - collect(cachedRow); - } - return; - } - } - - BoolQueryBuilder lookupCondition = new BoolQueryBuilder(); - for (int i = 0; i < lookupKeys.length; i++) { - lookupCondition.must( - new TermQueryBuilder(lookupKeys[i], converters[i].toExternal(keys[i]))); - } - searchSourceBuilder.query(lookupCondition); - searchRequest.source(searchSourceBuilder); - - Tuple2 searchResponse = null; - - for (int retry = 1; retry <= maxRetryTimes; retry++) { - try { - searchResponse = callBridge.search(client, searchRequest); - if (searchResponse.f1.length > 0) { - String[] result = searchResponse.f1; - // if cache disabled - if (cache == null) { - for (int i = 0; i < result.length; i++) { - RowData row = parseSearchHit(result[i]); - collect(row); - } - } else { // if cache enabled - ArrayList rows = new ArrayList<>(); - for (int i = 0; i < result.length; i++) { - RowData row = parseSearchHit(result[i]); - collect(row); - rows.add(row); - } - cache.put(keyRow, rows); - } - } - break; - } catch (IOException e) { - LOG.error(String.format("Elasticsearch search error, retry times = %d", retry), e); - if (retry >= maxRetryTimes) { - throw new RuntimeException("Execution of Elasticsearch search failed.", e); - } - try { - Thread.sleep(1000 * retry); - } catch (InterruptedException e1) { - LOG.warn( - "Interrupted while waiting to retry failed elasticsearch search, aborting"); - throw new RuntimeException(e1); - } - } - } - } private RowData parseSearchHit(String hit) { RowData row = null; diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 318757a2..99a1e961 100644 --- a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -56,7 +56,7 @@ * @param Elasticsearch client type * @param The address type to use */ -public abstract class ElasticsearchSinkTestBase +public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { protected static final String CLUSTER_NAME = "test-cluster"; @@ -175,9 +175,9 @@ protected abstract ElasticSearchInputFormatBase createElasticsearchInputFormat( */ public void runNullAddressesTest() { assertThatThrownBy( - () -> - createElasticsearchSink( - 1, null, SourceSinkDataTestKit.getJsonSinkFunction("test"))) + () -> + createElasticsearchSink( + 1, null, SourceSinkDataTestKit.getJsonSinkFunction("test"))) .isInstanceOfAny(IllegalArgumentException.class, NullPointerException.class); } @@ -186,11 +186,11 @@ public void runNullAddressesTest() { */ public void runEmptyAddressesTest() { assertThatThrownBy( - () -> - createElasticsearchSink( - 1, - Collections.emptyList(), - SourceSinkDataTestKit.getJsonSinkFunction("test"))) + () -> + createElasticsearchSink( + 1, + Collections.emptyList(), + SourceSinkDataTestKit.getJsonSinkFunction("test"))) .isInstanceOf(IllegalArgumentException.class); } @@ -226,18 +226,18 @@ protected abstract ElasticsearchSinkBase, C> createElast * because the Elasticsearch Java API to do so is incompatible across different versions. */ protected abstract ElasticsearchSinkBase, C> - createElasticsearchSinkForEmbeddedNode( - int bulkFlushMaxActions, - ElasticsearchSinkFunction> elasticsearchSinkFunction) - throws Exception; + createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + ElasticsearchSinkFunction> elasticsearchSinkFunction) + throws Exception; /** * Creates a version-specific Elasticsearch sink to connect to a specific Elasticsearch node. */ protected abstract ElasticsearchSinkBase, C> - createElasticsearchSinkForNode( - int bulkFlushMaxActions, - ElasticsearchSinkFunction> elasticsearchSinkFunction, - String ipAddress) - throws Exception; + createElasticsearchSinkForNode( + int bulkFlushMaxActions, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) + throws Exception; } diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 68bb99e2..ac86ff26 100644 --- a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -82,9 +82,9 @@ public void testInvalidElasticsearchCluster() throws Exception { @Override protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( - int bulkFlushMaxActions, - List httpHosts, - ElasticsearchSinkFunction> elasticsearchSinkFunction) { + int bulkFlushMaxActions, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java index 56226eab..9e5b999e 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSource.java @@ -135,6 +135,7 @@ public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext lookupContex this.format.createRuntimeDecoder( lookupContext, physicalSchema.toRowDataType()), lookupOptions, + config.getMaxRetryTimes(), config.getIndex(), config.getDocumentType(), columnNames, diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java index 04026a2b..9c977615 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicTableFactory.java @@ -121,7 +121,7 @@ public class Elasticsearch7DynamicTableFactory @Override public DynamicTableSource createDynamicTableSource(Context context) { - DataType physicalRowDataType = context.getPhysicalRowDataType(); + TableSchema schema = context.getCatalogTable().getSchema(); final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); final ReadableConfig options = helper.getOptions(); @@ -141,9 +141,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { return new Elasticsearch7DynamicSource( format, config, - physicalRowDataType, - options.get(MAX_RETRIES), - getLookupCache(options)); + TableSchemaUtils.getPhysicalSchema(schema), + new ElasticsearchLookupOptions.Builder().setMaxRetryTimes(options.get(MAX_RETRIES)).build()); } @Override @@ -172,17 +171,6 @@ public DynamicTableSink createDynamicTableSink(Context context) { getLocalTimeZoneId(context.getConfiguration())); } - @Nullable - private LookupCache getLookupCache(ReadableConfig tableOptions) { - LookupCache cache = null; - if (tableOptions - .get(LookupOptions.CACHE_TYPE) - .equals(LookupOptions.LookupCacheType.PARTIAL)) { - cache = DefaultLookupCache.fromConfig(tableOptions); - } - return cache; - } - ZoneId getLocalTimeZoneId(ReadableConfig readableConfig) { final String zone = readableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE); final ZoneId zoneId = diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java index e0b173c2..f9b31131 100644 --- a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchInputSplit; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.apache.flink.util.Preconditions; @@ -30,6 +31,7 @@ import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; @@ -88,6 +90,22 @@ public BulkProcessor.Builder createBulkProcessorBuilder( listener); } + @Override + public ElasticsearchInputSplit[] createInputSplitsInternal( + RestHighLevelClient client, + String index, + String type, + int minNumSplits) { + return new ElasticsearchInputSplit[0]; + } + + @Override + public Tuple2 scroll( + RestHighLevelClient client, + SearchScrollRequest searchScrollRequest) throws IOException { + return null; + } + @Override public Tuple2 search(RestHighLevelClient client, SearchRequest searchRequest) throws IOException { diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index 9d159b5f..b486f03e 100644 --- a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -45,7 +45,7 @@ /** IT cases for the {@link ElasticsearchSink}. */ public class ElasticsearchSinkITCase - extends ElasticsearchSinkTestBase { + extends ElasticsearchSinkTestBase { private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkITCase.class); @@ -105,6 +105,8 @@ public void testInvalidElasticsearchCluster() throws Exception { return builder.build(); } + + @Override protected ElasticSearchInputFormatBase createElasticsearchInputFormat( Map userConfig, DeserializationSchema deserializationSchema,