|
|
|
@ -31,6 +31,7 @@ import org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder;
|
|
|
|
|
import co.elastic.clients.elasticsearch.core.bulk.BulkOperationVariant;
|
|
|
|
|
import co.elastic.clients.elasticsearch.core.bulk.DeleteOperation;
|
|
|
|
|
import co.elastic.clients.elasticsearch.core.bulk.IndexOperation;
|
|
|
|
|
import org.apache.http.HttpHost;
|
|
|
|
|
|
|
|
|
|
import java.io.Serializable;
|
|
|
|
|
import java.time.ZoneId;
|
|
|
|
@ -46,7 +47,7 @@ import java.time.ZoneId;
|
|
|
|
|
public class ElasticsearchDataSink<InputT> implements DataSink, Serializable {
|
|
|
|
|
|
|
|
|
|
/** The Elasticsearch sink options. */
|
|
|
|
|
private final ElasticsearchSinkOptions elasticsearchOptions;
|
|
|
|
|
private final ElasticsearchSinkOptions esOptions;
|
|
|
|
|
|
|
|
|
|
/** The time zone ID for handling time-related operations. */
|
|
|
|
|
private final ZoneId zoneId;
|
|
|
|
@ -58,13 +59,13 @@ public class ElasticsearchDataSink<InputT> implements DataSink, Serializable {
|
|
|
|
|
* @param zoneId The time zone ID for handling time-related operations.
|
|
|
|
|
*/
|
|
|
|
|
public ElasticsearchDataSink(ElasticsearchSinkOptions elasticsearchOptions, ZoneId zoneId) {
|
|
|
|
|
this.elasticsearchOptions = elasticsearchOptions;
|
|
|
|
|
this.esOptions = elasticsearchOptions;
|
|
|
|
|
this.zoneId = zoneId;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Override
|
|
|
|
|
public EventSinkProvider getEventSinkProvider() {
|
|
|
|
|
switch (elasticsearchOptions.getVersion()) {
|
|
|
|
|
switch (esOptions.getVersion()) {
|
|
|
|
|
case 6:
|
|
|
|
|
return getElasticsearch6SinkProvider();
|
|
|
|
|
case 7:
|
|
|
|
@ -73,14 +74,14 @@ public class ElasticsearchDataSink<InputT> implements DataSink, Serializable {
|
|
|
|
|
return getElasticsearch8SinkProvider();
|
|
|
|
|
default:
|
|
|
|
|
throw new IllegalArgumentException(
|
|
|
|
|
"Unsupported Elasticsearch version: " + elasticsearchOptions.getVersion());
|
|
|
|
|
"Unsupported Elasticsearch version: " + esOptions.getVersion());
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private EventSinkProvider getElasticsearch6SinkProvider() {
|
|
|
|
|
ElasticsearchEventSerializer serializer = new ElasticsearchEventSerializer(zoneId);
|
|
|
|
|
org.apache.flink.elasticsearch6.shaded.org.apache.http.HttpHost[] hosts =
|
|
|
|
|
elasticsearchOptions.getHosts().stream()
|
|
|
|
|
esOptions.getHosts().stream()
|
|
|
|
|
.map(
|
|
|
|
|
host ->
|
|
|
|
|
new org.apache.flink.elasticsearch6.shaded.org.apache.http
|
|
|
|
@ -91,8 +92,7 @@ public class ElasticsearchDataSink<InputT> implements DataSink, Serializable {
|
|
|
|
|
.toArray(
|
|
|
|
|
org.apache.flink.elasticsearch6.shaded.org.apache.http.HttpHost[]
|
|
|
|
|
::new);
|
|
|
|
|
|
|
|
|
|
return FlinkSinkProvider.of(
|
|
|
|
|
Elasticsearch6SinkBuilder<Event> sinkBuilder =
|
|
|
|
|
new Elasticsearch6SinkBuilder<Event>()
|
|
|
|
|
.setHosts(hosts)
|
|
|
|
|
.setEmitter(
|
|
|
|
@ -109,63 +109,36 @@ public class ElasticsearchDataSink<InputT> implements DataSink, Serializable {
|
|
|
|
|
(DeleteOperation) operation));
|
|
|
|
|
}
|
|
|
|
|
})
|
|
|
|
|
.setBulkFlushMaxActions(elasticsearchOptions.getMaxBatchSize())
|
|
|
|
|
.setBulkFlushInterval(elasticsearchOptions.getMaxTimeInBufferMS())
|
|
|
|
|
.build());
|
|
|
|
|
.setBulkFlushMaxActions(esOptions.getMaxBatchSize())
|
|
|
|
|
.setBulkFlushInterval(esOptions.getMaxTimeInBufferMS());
|
|
|
|
|
if (esOptions.getUsername() != null) {
|
|
|
|
|
sinkBuilder
|
|
|
|
|
.setConnectionUsername(esOptions.getUsername())
|
|
|
|
|
.setConnectionPassword(esOptions.getPassword());
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return FlinkSinkProvider.of(sinkBuilder.build());
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private EventSinkProvider getElasticsearch7SinkProvider() {
|
|
|
|
|
ElasticsearchEventSerializer serializer = new ElasticsearchEventSerializer(zoneId);
|
|
|
|
|
org.apache.flink.elasticsearch6.shaded.org.apache.http.HttpHost[] hosts =
|
|
|
|
|
elasticsearchOptions.getHosts().stream()
|
|
|
|
|
.map(
|
|
|
|
|
host ->
|
|
|
|
|
new org.apache.flink.elasticsearch6.shaded.org.apache.http
|
|
|
|
|
.HttpHost(
|
|
|
|
|
host.getHostName(),
|
|
|
|
|
host.getPort(),
|
|
|
|
|
host.getSchemeName()))
|
|
|
|
|
.toArray(
|
|
|
|
|
org.apache.flink.elasticsearch6.shaded.org.apache.http.HttpHost[]
|
|
|
|
|
::new);
|
|
|
|
|
|
|
|
|
|
return FlinkSinkProvider.of(
|
|
|
|
|
new Elasticsearch6SinkBuilder<Event>()
|
|
|
|
|
.setHosts(hosts)
|
|
|
|
|
.setEmitter(
|
|
|
|
|
(element, context, indexer) -> {
|
|
|
|
|
BulkOperationVariant operation =
|
|
|
|
|
serializer.apply(element, context);
|
|
|
|
|
if (operation instanceof IndexOperation) {
|
|
|
|
|
indexer.add(
|
|
|
|
|
Elasticsearch6RequestCreator.createIndexRequest(
|
|
|
|
|
(IndexOperation<?>) operation));
|
|
|
|
|
} else if (operation instanceof DeleteOperation) {
|
|
|
|
|
indexer.add(
|
|
|
|
|
Elasticsearch6RequestCreator.createDeleteRequest(
|
|
|
|
|
(DeleteOperation) operation));
|
|
|
|
|
}
|
|
|
|
|
})
|
|
|
|
|
.setBulkFlushMaxActions(elasticsearchOptions.getMaxBatchSize())
|
|
|
|
|
.setBulkFlushInterval(elasticsearchOptions.getMaxTimeInBufferMS())
|
|
|
|
|
.build());
|
|
|
|
|
return getElasticsearch6SinkProvider();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private EventSinkProvider getElasticsearch8SinkProvider() {
|
|
|
|
|
return FlinkSinkProvider.of(
|
|
|
|
|
Elasticsearch8AsyncSinkBuilder<Event> sinkBuilder =
|
|
|
|
|
new Elasticsearch8AsyncSinkBuilder<Event>()
|
|
|
|
|
.setHosts(
|
|
|
|
|
elasticsearchOptions
|
|
|
|
|
.getHosts()
|
|
|
|
|
.toArray(new org.apache.http.HttpHost[0]))
|
|
|
|
|
.setHosts(esOptions.getHosts().toArray(new HttpHost[0]))
|
|
|
|
|
.setElementConverter(new ElasticsearchEventSerializer(zoneId))
|
|
|
|
|
.setMaxBatchSize(elasticsearchOptions.getMaxBatchSize())
|
|
|
|
|
.setMaxInFlightRequests(elasticsearchOptions.getMaxInFlightRequests())
|
|
|
|
|
.setMaxBufferedRequests(elasticsearchOptions.getMaxBufferedRequests())
|
|
|
|
|
.setMaxBatchSizeInBytes(elasticsearchOptions.getMaxBatchSizeInBytes())
|
|
|
|
|
.setMaxTimeInBufferMS(elasticsearchOptions.getMaxTimeInBufferMS())
|
|
|
|
|
.setMaxRecordSizeInBytes(elasticsearchOptions.getMaxRecordSizeInBytes())
|
|
|
|
|
.build());
|
|
|
|
|
.setMaxBatchSize(esOptions.getMaxBatchSize())
|
|
|
|
|
.setMaxInFlightRequests(esOptions.getMaxInFlightRequests())
|
|
|
|
|
.setMaxBufferedRequests(esOptions.getMaxBufferedRequests())
|
|
|
|
|
.setMaxBatchSizeInBytes(esOptions.getMaxBatchSizeInBytes())
|
|
|
|
|
.setMaxTimeInBufferMS(esOptions.getMaxTimeInBufferMS())
|
|
|
|
|
.setMaxRecordSizeInBytes(esOptions.getMaxRecordSizeInBytes());
|
|
|
|
|
if (esOptions.getUsername() != null) {
|
|
|
|
|
sinkBuilder.setUsername(esOptions.getUsername()).setPassword(esOptions.getPassword());
|
|
|
|
|
}
|
|
|
|
|
return FlinkSinkProvider.of(sinkBuilder.build());
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Override
|
|
|
|
|