|
@@ -13,9 +13,8 @@ import org.apache.lucene.search.MatchAllDocsQuery;
|
|
|
import org.apache.lucene.search.ScoreMode;
|
|
|
import org.apache.lucene.util.BytesRef;
|
|
|
import org.elasticsearch.ElasticsearchException;
|
|
|
-import org.elasticsearch.action.bulk.BackoffPolicy;
|
|
|
import org.elasticsearch.action.bulk.BulkItemResponse;
|
|
|
-import org.elasticsearch.action.bulk.BulkProcessor;
|
|
|
+import org.elasticsearch.action.bulk.BulkProcessor2;
|
|
|
import org.elasticsearch.action.bulk.BulkRequest;
|
|
|
import org.elasticsearch.action.bulk.BulkResponse;
|
|
|
import org.elasticsearch.action.index.IndexRequestBuilder;
|
|
@@ -75,6 +74,7 @@ class RollupShardIndexer {
|
|
|
private static final Logger logger = LogManager.getLogger(RollupShardIndexer.class);
|
|
|
public static final int ROLLUP_BULK_ACTIONS = 10000;
|
|
|
public static final ByteSizeValue ROLLUP_BULK_SIZE = new ByteSizeValue(1, ByteSizeUnit.MB);
|
|
|
+ public static final ByteSizeValue ROLLUP_MAX_BYTES_IN_FLIGHT = new ByteSizeValue(50, ByteSizeUnit.MB);
|
|
|
|
|
|
private final IndexShard indexShard;
|
|
|
private final Client client;
|
|
@@ -87,6 +87,8 @@ class RollupShardIndexer {
|
|
|
private final List<FieldValueFetcher> fieldValueFetchers;
|
|
|
private final RollupShardTask task;
|
|
|
private volatile boolean abort = false;
|
|
|
+ ByteSizeValue rollupBulkSize = ROLLUP_BULK_SIZE;
|
|
|
+ ByteSizeValue rollupMaxBytesInFlight = ROLLUP_MAX_BYTES_IN_FLIGHT;
|
|
|
|
|
|
RollupShardIndexer(
|
|
|
RollupShardTask task,
|
|
@@ -129,7 +131,7 @@ class RollupShardIndexer {
|
|
|
|
|
|
public DownsampleIndexerAction.ShardDownsampleResponse execute() throws IOException {
|
|
|
long startTime = System.currentTimeMillis();
|
|
|
- BulkProcessor bulkProcessor = createBulkProcessor();
|
|
|
+ BulkProcessor2 bulkProcessor = createBulkProcessor();
|
|
|
try (searcher; bulkProcessor) {
|
|
|
final TimeSeriesIndexSearcher timeSeriesSearcher = new TimeSeriesIndexSearcher(searcher, List.of(this::checkCancelled));
|
|
|
TimeSeriesBucketCollector bucketCollector = new TimeSeriesBucketCollector(bulkProcessor);
|
|
@@ -160,6 +162,18 @@ class RollupShardIndexer {
|
|
|
);
|
|
|
}
|
|
|
|
|
|
+ if (task.getNumFailed() > 0) {
|
|
|
+ throw new ElasticsearchException(
|
|
|
+ "Shard ["
|
|
|
+ + indexShard.shardId()
|
|
|
+ + "] failed to index all rollup documents. Sent ["
|
|
|
+ + task.getNumSent()
|
|
|
+ + "], failed ["
|
|
|
+ + task.getNumFailed()
|
|
|
+ + "]."
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
return new DownsampleIndexerAction.ShardDownsampleResponse(indexShard.shardId(), task.getNumIndexed());
|
|
|
}
|
|
|
|
|
@@ -176,8 +190,8 @@ class RollupShardIndexer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private BulkProcessor createBulkProcessor() {
|
|
|
- final BulkProcessor.Listener listener = new BulkProcessor.Listener() {
|
|
|
+ private BulkProcessor2 createBulkProcessor() {
|
|
|
+ final BulkProcessor2.Listener listener = new BulkProcessor2.Listener() {
|
|
|
@Override
|
|
|
public void beforeBulk(long executionId, BulkRequest request) {
|
|
|
task.addNumSent(request.numberOfActions());
|
|
@@ -206,7 +220,7 @@ class RollupShardIndexer {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void afterBulk(long executionId, BulkRequest request, Throwable failure) {
|
|
|
+ public void afterBulk(long executionId, BulkRequest request, Exception failure) {
|
|
|
if (failure != null) {
|
|
|
long items = request.numberOfActions();
|
|
|
task.addNumFailed(items);
|
|
@@ -218,24 +232,23 @@ class RollupShardIndexer {
|
|
|
}
|
|
|
};
|
|
|
|
|
|
- return BulkProcessor.builder(client::bulk, listener, "rollup-shard-indexer")
|
|
|
+ return BulkProcessor2.builder(client::bulk, listener, client.threadPool())
|
|
|
.setBulkActions(ROLLUP_BULK_ACTIONS)
|
|
|
.setBulkSize(ROLLUP_BULK_SIZE)
|
|
|
- // execute the bulk request on the same thread
|
|
|
- .setConcurrentRequests(0)
|
|
|
- .setBackoffPolicy(BackoffPolicy.exponentialBackoff(TimeValue.timeValueMillis(1000), 3))
|
|
|
+ .setMaxBytesInFlight(rollupMaxBytesInFlight)
|
|
|
+ .setMaxNumberOfRetries(3)
|
|
|
.build();
|
|
|
}
|
|
|
|
|
|
private class TimeSeriesBucketCollector extends BucketCollector {
|
|
|
- private final BulkProcessor bulkProcessor;
|
|
|
+ private final BulkProcessor2 bulkProcessor;
|
|
|
private final RollupBucketBuilder rollupBucketBuilder;
|
|
|
private long docsProcessed;
|
|
|
private long bucketsCreated;
|
|
|
long lastTimestamp = Long.MAX_VALUE;
|
|
|
long lastHistoTimestamp = Long.MAX_VALUE;
|
|
|
|
|
|
- TimeSeriesBucketCollector(BulkProcessor bulkProcessor) {
|
|
|
+ TimeSeriesBucketCollector(BulkProcessor2 bulkProcessor) {
|
|
|
this.bulkProcessor = bulkProcessor;
|
|
|
List<AbstractDownsampleFieldProducer> rollupFieldProducers = fieldValueFetchers.stream()
|
|
|
.map(FieldValueFetcher::rollupFieldProducer)
|
|
@@ -336,7 +349,7 @@ class RollupShardIndexer {
|
|
|
if (logger.isTraceEnabled()) {
|
|
|
logger.trace("Indexing rollup doc: [{}]", Strings.toString(doc));
|
|
|
}
|
|
|
- bulkProcessor.add(request.request());
|
|
|
+ bulkProcessor.addWithBackpressure(request.request(), () -> abort);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -352,7 +365,6 @@ class RollupShardIndexer {
|
|
|
XContentBuilder doc = rollupBucketBuilder.buildRollupDocument();
|
|
|
indexBucket(doc);
|
|
|
}
|
|
|
- bulkProcessor.flush();
|
|
|
|
|
|
// check cancel after the flush all data
|
|
|
checkCancelled();
|