-
Notifications
You must be signed in to change notification settings - Fork 1.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[Feature][Connector-V2][Elasticsearch] Elasticsearch supports spaced write and is relatively real-time data source friendly #4347 #4349
Changes from all commits
0200ea8
624d570
8d3dd6f
aad46d8
d3b73d5
d7461e9
89636df
fad8a31
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -42,6 +42,11 @@ | |
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.Optional; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.ScheduledExecutorService; | ||
import java.util.concurrent.ScheduledFuture; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
/** | ||
* ElasticsearchSinkWriter is a sink writer that will write {@link SeaTunnelRow} to Elasticsearch. | ||
|
@@ -54,20 +59,28 @@ public class ElasticsearchSinkWriter | |
|
||
private final int maxBatchSize; | ||
|
||
private final int batchIntervalMs; | ||
|
||
private final SeaTunnelRowSerializer seaTunnelRowSerializer; | ||
private final List<String> requestEsList; | ||
private EsRestClient esRestClient; | ||
private RetryMaterial retryMaterial; | ||
private static final long DEFAULT_SLEEP_TIME_MS = 200L; | ||
|
||
private transient ScheduledExecutorService scheduler; | ||
private transient ScheduledFuture<?> scheduledFuture; | ||
private transient boolean isClose; | ||
|
||
public ElasticsearchSinkWriter( | ||
SinkWriter.Context context, | ||
SeaTunnelRowType seaTunnelRowType, | ||
Config pluginConfig, | ||
int maxBatchSize, | ||
int maxRetryCount) { | ||
int maxRetryCount, | ||
int batchIntervalMs) { | ||
this.context = context; | ||
this.maxBatchSize = maxBatchSize; | ||
this.batchIntervalMs = batchIntervalMs; | ||
|
||
IndexInfo indexInfo = new IndexInfo(pluginConfig); | ||
esRestClient = EsRestClient.createInstance(pluginConfig); | ||
|
@@ -78,6 +91,11 @@ public ElasticsearchSinkWriter( | |
this.requestEsList = new ArrayList<>(maxBatchSize); | ||
this.retryMaterial = | ||
new RetryMaterial(maxRetryCount, true, exception -> true, DEFAULT_SLEEP_TIME_MS); | ||
// Initialize the interval flush | ||
if (this.batchIntervalMs > 0) { | ||
startScheduler(); | ||
log.info("The initial scheduling is complete batch_interval_ms :" + batchIntervalMs); | ||
} | ||
} | ||
|
||
@Override | ||
|
@@ -93,6 +111,32 @@ public void write(SeaTunnelRow element) { | |
} | ||
} | ||
|
||
public void startScheduler() { | ||
this.scheduler = | ||
Executors.newScheduledThreadPool( | ||
1, | ||
runnable -> { | ||
AtomicInteger cnt = new AtomicInteger(0); | ||
Thread thread = new Thread(runnable); | ||
thread.setDaemon(true); | ||
thread.setName( | ||
"sink-elasticsearch-interval" + "-" + cnt.incrementAndGet()); | ||
return thread; | ||
}); | ||
this.scheduledFuture = | ||
this.scheduler.scheduleWithFixedDelay( | ||
() -> { | ||
synchronized (ElasticsearchSinkWriter.this) { | ||
if (requestEsList.size() > 0 && !isClose) { | ||
bulkEsWithRetry(this.esRestClient, this.requestEsList); | ||
} | ||
} | ||
}, | ||
this.batchIntervalMs, | ||
this.batchIntervalMs, | ||
TimeUnit.MILLISECONDS); | ||
} | ||
|
||
@Override | ||
public Optional<ElasticsearchCommitInfo> prepareCommit() { | ||
bulkEsWithRetry(this.esRestClient, this.requestEsList); | ||
|
@@ -115,6 +159,9 @@ public synchronized void bulkEsWithRetry( | |
ElasticsearchConnectorErrorCode.BULK_RESPONSE_ERROR, | ||
"bulk es error: " + bulkResponse.getResponse()); | ||
} | ||
log.info( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I support not to introduce ScheduledThreadPool,but run Thead.sleep(batchIntervalMs) after bulk request. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it's going to be a little bit more write friendly for real-time programs and even if the consumption data is less than the current batch it's going to flush to es There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seem like don't a better way to flush data without |
||
"bulk es successfully written to the rowNum: " | ||
+ requestEsList.size()); | ||
return bulkResponse; | ||
} | ||
return null; | ||
|
@@ -132,6 +179,12 @@ public synchronized void bulkEsWithRetry( | |
@Override | ||
public void close() throws IOException { | ||
bulkEsWithRetry(this.esRestClient, this.requestEsList); | ||
esRestClient.close(); | ||
this.isClose = true; | ||
if (esRestClient != null) { | ||
esRestClient.close(); | ||
} | ||
if (this.scheduler != null) { | ||
this.scheduler.shutdown(); | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do you need a default
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.