Skip to content

TSDB: Downsampling support cancelled #88496

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

Merged
merged 19 commits into from
Dec 6, 2022

Conversation

weizijun
Copy link
Contributor

@weizijun weizijun commented Jul 13, 2022

support cancel downsampling task:

  • POST _tasks/oTUltX4IQMOUUVeiohTt8A:12345/_cancel
  • POST _tasks/_cancel?actions=*rollup_indexer*
  • POST _tasks/_cancel? parent_task_id=xxx

weizijun added 3 commits July 12, 2022 16:13
* upstream/master: (38 commits)
  Simplify map copying (elastic#88432)
  Make DiffableUtils.diff implementation agnostic (elastic#88403)
  Ingest: Start separating Metadata from IngestSourceAndMetadata (elastic#88401)
  Move runtime fields base scripts out of scripting fields api package. (elastic#88488)
  Enable TRACE Logging for test and increase timeout (elastic#88477)
  Mute ReactiveStorageIT#testScaleDuringSplitOrClone (elastic#88480)
  Track the count of failed invocations since last successful policy snapshot (elastic#88398)
  Avoid noisy exceptions on data nodes when aborting snapshots (elastic#88476)
  Fix ReactiveStorageDeciderServiceTests testNodeSizeForDataBelowLowWatermark (elastic#88452)
  INFO logging of snapshot restore and completion (elastic#88257)
  unmute test (elastic#88454)
  Updatable API keys - noop check (elastic#88346)
  Corrected an incomplete sentence. (elastic#86542)
  Use consistent shard map type in IndexService (elastic#88465)
  Stop registering TestGeoShapeFieldMapperPlugin in ESIntegTestCase (elastic#88460)
  TSDB: RollupShardIndexer logging improvements (elastic#88416)
  Audit API key ID when create or grant API keys (elastic#88456)
  Bound random negative size test in SearchSourceBuilderTests#testNegativeSizeErrors (elastic#88457)
  Updatable API keys - logging audit trail event (elastic#88276)
  Polish reworked LoggedExec task (elastic#88424)
  ...

# Conflicts:
#	x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RollupShardIndexer.java
@elasticsearchmachine elasticsearchmachine added v8.4.0 external-contributor Pull request authored by a developer outside the Elasticsearch team labels Jul 13, 2022
@csoulios csoulios self-assigned this Jul 13, 2022
@csoulios csoulios added >non-issue :StorageEngine/Rollup Turn fine-grained time-based data into coarser-grained data labels Jul 13, 2022
@weizijun weizijun marked this pull request as ready for review July 14, 2022 08:46
@elasticmachine elasticmachine added the Team:Analytics Meta label for analytical engine team (ESQL/Aggs/Geo) label Jul 14, 2022
@elasticmachine
Copy link
Collaborator

Pinging @elastic/es-analytics-geo (Team:Analytics)

@weizijun weizijun changed the title [WIP]TSDB: Downsampling support cancelled TSDB: Downsampling support cancelled Jul 14, 2022
@@ -172,17 +195,23 @@ public void beforeBulk(long executionId, BulkRequest request) {
public void afterBulk(long executionId, BulkRequest request, BulkResponse response) {
numIndexed.addAndGet(request.numberOfActions());
if (response.hasFailures()) {
Map<String, String> failures = Arrays.stream(response.getItems())
List<BulkItemResponse> failedItems = Arrays.stream(response.getItems())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't get why we need this change here. failedItems.size() should be equal to failures.size(). Isn't enough to just call status.setCancelled()?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

when the _id is null, the map key is the same, so the failures.size() is 1, so I use a list to same the failures

@@ -114,7 +120,7 @@ public void setup() {
rollupIndex = "rollup-" + sourceIndex;
startTime = randomLongBetween(946769284000L, 1607470084000L); // random date between 2000-2020
docCount = randomIntBetween(10, 9000);
numOfShards = randomIntBetween(1, 4);
numOfShards = 4;// randomIntBetween(1, 4);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not setting this value randomly?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not setting this value randomly?

sorry, I will revert it

IndicesService indexServices = getInstanceFromNode(IndicesService.class);
Index srcIndex = resolveIndex(sourceIndex);
IndexService indexService = indexServices.indexServiceSafe(srcIndex);
IndexShard shard = indexService.getShard(0);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we select the shard randomly?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we select the shard randomly?

ok

@elasticsearchmachine elasticsearchmachine changed the base branch from master to main July 22, 2022 23:05
@mark-vieira mark-vieira added v8.5.0 and removed v8.4.0 labels Jul 27, 2022
@csoulios csoulios added v8.6.0 and removed v8.5.0 labels Sep 21, 2022
@martijnvg martijnvg self-requested a review November 2, 2022 10:12
@salvatore-campagna
Copy link
Contributor

@weizijun do you mind updating this PR so to resolve conflicts? We are reviewing it and considering merging it. Thanks.

@weizijun
Copy link
Contributor Author

weizijun commented Nov 3, 2022

@weizijun do you mind updating this PR so to resolve conflicts? We are reviewing it and considering merging it. Thanks.

Okay!

* main: (1300 commits)
  update c2id/c2id-server-demo docker image to support ARM (elastic#91144)
  Allow legacy index settings on legacy indices (elastic#90264)
  Skip prevoting if single-node discovery (elastic#91255)
  Chunked encoding for snapshot status API (elastic#90801)
  Allow different decay values depending on the score function (elastic#91195)
  Fix handling indexed envelopes crossing the dateline in mvt API (elastic#91105)
  Ensure cleanups succeed in JoinValidationService (elastic#90601)
  Add overflow behaviour test for RecyclerBytesStreamOutput (elastic#90638)
  More actionable error for ancient indices (elastic#91243)
  Fix APM configuration file delete (elastic#91058)
  Clean up handshake test class (elastic#90966)
  Improve H3#hexRing logic and add H3#areNeighborCells method (elastic#91140)
  Restrict direct use of `ApplicationPrivilege` constructor (elastic#91176)
  [ML] Allow NLP truncate option to be updated when span is set (elastic#91224)
  Support multi-intersection for FieldPermissions (elastic#91169)
  Support intersecting multi-sets of queries with DocumentPermissions (elastic#91151)
  Ensure TermsEnum action works correctly with API keys (elastic#91170)
  Fix NPE in auditing authenticationSuccess for non-existing run-as user (elastic#91171)
  Ensure PKI's delegated_by_realm metadata respect run-as (elastic#91173)
  [ML] Update API documentation for anomaly score explanation (elastic#91177)
  ...

# Conflicts:
#	x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java
#	x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/RollupShardIndexer.java
#	x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/TransportRollupIndexerAction.java
#	x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/RollupActionSingleNodeTests.java
@martijnvg
Copy link
Member

@elasticmachine test this please

@weizijun
Copy link
Contributor Author

weizijun commented Nov 3, 2022

@salvatore-campagna @martijnvg there is a notice: the RollupShardStatus is referenced by RollupShardIndexer. I'm not sure if RollupShardStatus will change the reference.
If the reference changes, the new RollupShardStatus changed to ABORT, but the old RollupShardStatus reference by RollupShardIndexer don't changed, so the task can not be stopped.

Copy link
Member

@martijnvg martijnvg left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @weizijun for working on this. I left comments around the integration with the task framework.


import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg;

public class RollupShardStatus implements Task.Status {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can the status class be immutable? Typically the task class has modifiable state and when we create the status class it reads fields from the task and then this status class is immutable (an immutable view of the state).

The RollupShardTask should then contain the AtomicInteger counters and the this class should then be passed down to RollupShardIndexer.

For an example you can take a look at reindex, at the BulkByScrollTask and BulkByScrollTask.Status classes.

@@ -170,9 +179,12 @@ protected RollupIndexerAction.Response newResponse(
private class Async extends AsyncBroadcastAction {
private final RollupIndexerAction.Request request;
private final ActionListener<RollupIndexerAction.Response> listener;
private final Task task;
private boolean hasCancelled = false;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think I fully understand that the changes to the Async inner class.
It seems to always cancel a shard level rollup before actually performing it?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think I understand now, why this logic is added and how it works. This for cancelling other shard level operations in case one fails. Downsampling may take a long time to complete and if a downsample fails in one shard, this can speed up the response to the client.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should the hasCancelled field be volatile? I think it will read / updated from different threads.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I also don't see this change. Maybe you forgot to push it?

@@ -90,11 +92,15 @@ class RollupShardIndexer {
private final String[] metricFields;
private final String[] labelFields;
private final Map<String, FieldValueFetcher> fieldValueFetchers;
private final AtomicLong numReceived = new AtomicLong();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe these counters should be moved to the RollupShardTask class? If this class has access to the task then it can just increments these counters that are part of the task. I think makes managing these counters easier.

@martijnvg
Copy link
Member

the RollupShardStatus is referenced by RollupShardIndexer. I'm not sure if RollupShardStatus will change the reference.
If the reference changes, the new RollupShardStatus changed to ABORT, but the old RollupShardStatus reference by RollupShardIndexer don't changed, so the task can not be stopped.

I think status classes aren't meant to be passed around to other components and keep track of state. This what the task class is for. When an action executes there should be only one task instance and that should exist until the operation completed (either successfully or failed). In this case for downsampling, a task (created by RollupIndexerAction) will be created for each shard and also a task instance when TransportRollupAction invokes the rollup indexer action action.

Copy link
Member

@martijnvg martijnvg left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @weizijun. I did another review round.


private final ShardId shardId;
private final long rollupStart;
private final AtomicLong numReceived;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Change the num* fields to type long?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These fields are still AtomicLong, can you change the type of the fields field long?


public class RollupShardTask extends CancellableTask {
private String rollupIndex;
private DownsampleConfig config;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think rollupIndex and config fields can also be final fields?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These fields are still not final?

private final AtomicLong numSent = new AtomicLong();
private final AtomicLong numIndexed = new AtomicLong();
private final AtomicLong numFailed = new AtomicLong();
private final AtomicLong numReceived;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe use the use getters on the task object directly? Instead of also defining these fields here.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is a reference to the task here. So I think the atomic long fields can be removed here and
instead the atomic longs from the RollupShardTask can be used directly in this class.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you also make this (^) change?


@Override
public Status getStatus() {
return status;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of creating a status object in the constructor. The status object should be created in this method.
This way a read only copy of the status of the task is created. And then there is no need to pass the atomic longs to the status.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I change the code.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method should return a new status instance:

new RollupShardStatus(shardId, numReceived, numSent, numIndexed, numFailed);

and then the status field in this class can be removed.

@@ -170,9 +179,12 @@ protected RollupIndexerAction.Response newResponse(
private class Async extends AsyncBroadcastAction {
private final RollupIndexerAction.Request request;
private final ActionListener<RollupIndexerAction.Response> listener;
private final Task task;
private boolean hasCancelled = false;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think I understand now, why this logic is added and how it works. This for cancelling other shard level operations in case one fails. Downsampling may take a long time to complete and if a downsample fails in one shard, this can speed up the response to the client.

@@ -170,9 +179,12 @@ protected RollupIndexerAction.Response newResponse(
private class Async extends AsyncBroadcastAction {
private final RollupIndexerAction.Request request;
private final ActionListener<RollupIndexerAction.Response> listener;
private final Task task;
private boolean hasCancelled = false;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should the hasCancelled field be volatile? I think it will read / updated from different threads.

@@ -50,6 +55,7 @@ public class TransportRollupIndexerAction extends TransportBroadcastAction<
private final Client client;
private final ClusterService clusterService;
private final IndicesService indicesService;
private final TransportCancelTasksAction cancelTasksAction;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe use the client field here instead of adding cancelTasksAction here? The cancel task action is available in ClusterAdminClient.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the cancelTasksAction field can be removed here and just use client here to invoke the cancel action.

@Override
protected void onOperation(@Nullable ShardRouting shard, final ShardIterator shardIt, int shardIndex, Exception e) {
// when this shard operation failed, cancel other shard operations
cancelOtherShardIndexers();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In case a shard has replicas the behaviour of the super class is to retry on the replica shard.
With the cancel logic, I don't think we want this?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When do you want cancelOtherShardIndexers to be called? I change the code, when the shard failed, to avoid retry, I set the shardIt to the last shard.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I change the code, when the shard failed, to avoid retry, I set the shardIt to the last shard.

👍

* main: (163 commits)
  [DOCS] Edits frequent items aggregation (elastic#91564)
  Handle providers of optional services in ubermodule classloader (elastic#91217)
  Add `exportDockerImages` lifecycle task for exporting docker tarballs (elastic#91571)
  Fix CSV dependency report output file location in DRA CI job
  Fix variable placeholder for Strings.format calls (elastic#91531)
  Fix output dir creation in ConcatFileTask (elastic#91568)
  Fix declaration of dependencies in DRA snapshots CI job (elastic#91569)
  Upgrade Gradle Enterprise plugin to 3.11.4 (elastic#91435)
  Ingest DateProcessor (small) speedup, optimize collections code in DateFormatter.forPattern (elastic#91521)
  Fix inter project handling of generateDependenciesReport (elastic#91555)
  [Synthetics] Add synthetics-* read to fleet-server (elastic#91391)
  [ML] Copy more settings when creating DF analytics destination index (elastic#91546)
  Reduce CartesianCentroidIT flakiness (elastic#91553)
  Propagate last node to reinitialized routing tables (elastic#91549)
  Forecast write load during rollovers (elastic#91425)
  [DOCS] Warn about potential overhead of named queries (elastic#91512)
  Datastream unavailable exception metadata (elastic#91461)
  Generate docker images and dependency report in DRA ci job (elastic#91545)
  Support cartesian_bounds aggregation on point and shape (elastic#91298)
  Add support for EQL samples queries (elastic#91312)
  ...

# Conflicts:
#	x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/RollupShardIndexer.java
@kingherc kingherc added v8.7.0 and removed v8.6.0 labels Nov 16, 2022
Copy link
Member

@martijnvg martijnvg left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did another review round. I think some changes were not pushed? I don't see some of the changes you said you've done.


@Override
public Status getStatus() {
return status;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method should return a new status instance:

new RollupShardStatus(shardId, numReceived, numSent, numIndexed, numFailed);

and then the status field in this class can be removed.


public class RollupShardTask extends CancellableTask {
private String rollupIndex;
private DownsampleConfig config;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These fields are still not final?

@@ -50,6 +55,7 @@ public class TransportRollupIndexerAction extends TransportBroadcastAction<
private final Client client;
private final ClusterService clusterService;
private final IndicesService indicesService;
private final TransportCancelTasksAction cancelTasksAction;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the cancelTasksAction field can be removed here and just use client here to invoke the cancel action.

@@ -170,9 +179,12 @@ protected RollupIndexerAction.Response newResponse(
private class Async extends AsyncBroadcastAction {
private final RollupIndexerAction.Request request;
private final ActionListener<RollupIndexerAction.Response> listener;
private final Task task;
private boolean hasCancelled = false;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I also don't see this change. Maybe you forgot to push it?

task,
new CancelTasksRequest().setTargetParentTaskId(new TaskId(clusterService.localNode().getId(), task.getId())),
ActionListener.wrap(r -> {
logger.info("[{}] rollup cancel other shard indexers", request.getRollupRequest().getSourceIndex());
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I also don't see this change. Maybe you forgot to push it?

private final AtomicLong numSent = new AtomicLong();
private final AtomicLong numIndexed = new AtomicLong();
private final AtomicLong numFailed = new AtomicLong();
private final AtomicLong numReceived;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is a reference to the task here. So I think the atomic long fields can be removed here and
instead the atomic longs from the RollupShardTask can be used directly in this class.

@weizijun
Copy link
Contributor Author

I did another review round. I think some changes were not pushed? I don't see some of the changes you said you've done.

Sorry, I lost some changes, now I have pushed them.


private final ShardId shardId;
private final long rollupStart;
private final AtomicLong numReceived;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These fields are still AtomicLong, can you change the type of the fields field long?

private final AtomicLong numSent = new AtomicLong();
private final AtomicLong numIndexed = new AtomicLong();
private final AtomicLong numFailed = new AtomicLong();
private final AtomicLong numReceived;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you also make this (^) change?

@Override
protected void onOperation(@Nullable ShardRouting shard, final ShardIterator shardIt, int shardIndex, Exception e) {
// when this shard operation failed, cancel other shard operations
cancelOtherShardIndexers();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I change the code, when the shard failed, to avoid retry, I set the shardIt to the last shard.

👍

@@ -170,9 +177,12 @@ protected RollupIndexerAction.Response newResponse(
private class Async extends AsyncBroadcastAction {
private final RollupIndexerAction.Request request;
private final ActionListener<RollupIndexerAction.Response> listener;
private final Task task;
private volatile boolean hasCancelled = false;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe rename to hasCancelledOtherShardOperations?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

Copy link
Member

@martijnvg martijnvg left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I left one smaller comment and a bigger comment around cancelling shard level downsamples in case a single shard level downsample fails. I think it is fragile, and I don't have a good idea how to make this more robust. Maybe leave this out of this PR? So that we can get the rest of the PR merged? If you like then the shard level downsample cancelling in case of failure can be explored/developed in another PR.


@Override
public Status getStatus() {
return new RollupShardStatus(shardId, rollupStartTime, numReceived, numSent, numIndexed, numFailed);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Change return new RollupShardStatus(shardId, rollupStartTime, numReceived, numSent, numIndexed, numFailed); into return new RollupShardStatus(shardId, rollupStartTime.get(), numReceived.get(), numSent.get(), numIndexed.get(), numFailed.get()); and then the fields in RollupShardStatus that are of type AtomicLong should be changed to long.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@@ -186,5 +196,42 @@ protected void finishHim() {
listener.onFailure(e);
}
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

After some more thinking, I'm not comfortable this particular change (cancelling other shard level downsamples incase a shard level downsample fails). I think in the current state it is fragile. I don't have a good suggestion how to make this logic more robust.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

okay, I removed these code.

@martijnvg
Copy link
Member

@elasticmachine test this please

* upstream/main: (209 commits)
  Remove unused methods and classes from HLRC (elastic#92030)
  Clean up on exception while chunking XContent (elastic#92024)
  Add profiling plugin (elastic#91640)
  Remove unused methods and classes from HLRC (elastic#92012)
  Remove IndexerState from HLRC (elastic#92023)
  Ensure cached time elapses in ClusterServiceIT (elastic#91986)
  Chunked encoding for RestGetIndicesAction (elastic#92016)
  Simplify shardsWithState (elastic#91991)
  [DOCS] Updates ML decider docs by mentioning CPU as scaling criterion (elastic#92018)
  Add chunking to ClusterState.Custom impls (elastic#91963)
  Speedup time_series agg by caching current tsid ordinal, parent bucket ordinal and buck ordinal (elastic#91784)
  Drop the ingest listener call count tracking (elastic#92003)
  [DOCS] fixes issue number 91889 - missing [discrete] header (elastic#91976)
  Fix PersistentTasksClusterServiceTests (elastic#92002)
  [docs] Update search-settings documentation to reflect the fact that the indices.query.bool.max_clause_count setting has been deprecated (elastic#91811)
  Clarify writability in Netty4HttpPipeliningHandler (elastic#91982)
  Load stable plugins as synthetic modules (elastic#91869)
  Handle any exception thrown while generating source for an IngestDocument (elastic#91981)
  fixing Apache HttpHost url on java-rest doc (elastic#91945)
  Implement repair functionality for aliases colliding with indices bug (elastic#91887)
  ...
Copy link
Member

@martijnvg martijnvg left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @weizijun! I left one minor comment, but other than that this looks good to me.

return new RollupShardStatus(shardId, rollupStartTime, numReceived.get(), numSent.get(), numIndexed.get(), numFailed.get());
}

public long getNumReceived() {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe just have getter methods for the AtomicLong fields?

@martijnvg
Copy link
Member

@elasticmachine test this please

@martijnvg
Copy link
Member

@elasticmachine update branch

@martijnvg
Copy link
Member

@elasticmachine test this please

@martijnvg
Copy link
Member

@elasticmachine run elasticsearch-ci/packaging-tests-windows-sample

@martijnvg martijnvg merged commit 5e7c417 into elastic:main Dec 6, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
external-contributor Pull request authored by a developer outside the Elasticsearch team >non-issue :StorageEngine/Rollup Turn fine-grained time-based data into coarser-grained data Team:Analytics Meta label for analytical engine team (ESQL/Aggs/Geo) v8.7.0
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants