Skip to content

Do not mark bulk indexing requests as retried after primary relocations#142157

Merged
pxsalehi merged 20 commits intoelastic:mainfrom
pxsalehi:ps260204-relocation-caused-forwarded-index-req-is-not-retry
Feb 23, 2026
Merged

Do not mark bulk indexing requests as retried after primary relocations#142157
pxsalehi merged 20 commits intoelastic:mainfrom
pxsalehi:ps260204-relocation-caused-forwarded-index-req-is-not-retry

Conversation

@pxsalehi
Copy link
Copy Markdown
Member

@pxsalehi pxsalehi commented Feb 9, 2026

I've added a flag to RetryOnPrimaryException which indicates whether the request which is to be retried might have ran or not. If true (default value and default behaviour before this PR) the request is marked as retry. If false, it is safe to run the request as if it is not a retry, and we skip marking it as a retry. This should impact only one specific path of the retries in the TransportReplicationAction.

Assisted by Cursor

Closes #141586.
Relates ES-14121

@pxsalehi pxsalehi added >enhancement :Distributed/CRUD A catch all label for issues around indexing, updating and getting a doc by id. Not search. labels Feb 9, 2026
Comment on lines +1074 to +1076
if (isPrimaryAction
&& cause.getClass() == ReplicationOperation.RetryOnPrimaryException.class
&& request instanceof BulkShardRequest) {
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I've included that last check (that the request is a bulk shard request) just to make this very specific. Otherwise, I think it is implied.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I'd like to omit that - or if we need it, make it a protected method that is overridden for bulk shard action.

@elasticsearchmachine
Copy link
Copy Markdown
Collaborator

Hi @pxsalehi, I've created a changelog YAML for you.

@pxsalehi pxsalehi marked this pull request as ready for review February 10, 2026 09:23
@elasticsearchmachine elasticsearchmachine added the Team:Distributed Meta label for distributed team. label Feb 10, 2026
@elasticsearchmachine
Copy link
Copy Markdown
Collaborator

Pinging @elastic/es-distributed (Team:Distributed)

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Or would it be more obviously correct, if we change the exception we throw here and check for that exception dow in this file (instead of RetryOnPrimaryException and to distinguish between the cases in org.elasticsearch.action.support.replication.ReplicationOperation#onNoLongerPrimary)? This case for example doesn't necessarily mean that the operation didn't do any changes, but then again it also means the shard copy is not authoritative on what is the latest state, AFAIU.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I would find that easier to reason about. We may then incrementally move other exceptions over to be the new "do not mark retry" exception (better name needed).

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

@henningandersen do you have an objection to only addressing this specific case of RetryOnPrimaryException here and not the UnavailableShardsException? We won't change the reroute phase in this case. The latter is I believe a separate issue. That limits this PR to this one change and we don't have to weed out the UnavailableShardsException cases which I think are not the main issue. I can track that in a separate issue. At the very minimum, I think they can be two separate PRs and go in independently.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Yeah, I am good with not doing anything for unavailability, seems like a very different case. We are primarily after improving the happy path.

Copy link
Copy Markdown
Contributor

@henningandersen henningandersen left a comment

Choose a reason for hiding this comment

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

Left a few comments. I did not try to deeply investigate whether all RetryOnPrimaryException should disregard the retry, let me know if you think it is the right route to go.

Comment on lines +1074 to +1076
if (isPrimaryAction
&& cause.getClass() == ReplicationOperation.RetryOnPrimaryException.class
&& request instanceof BulkShardRequest) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I'd like to omit that - or if we need it, make it a protected method that is overridden for bulk shard action.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I would find that easier to reason about. We may then incrementally move other exceptions over to be the new "do not mark retry" exception (better name needed).

setPhase(task, "waiting_for_retry");
request.onRetry();
if (markRequestAsRetry) {
request.onRetry();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Maybe we need to clarify the behavior on ReplicationRequest.onRetry, i.e., that it may not be called if the action did not do any real work yet.

We could also consider adding the markRequestAsRetry (under another name) boolean to the method instead, mainly to make the contract clearer (same functionality).

return List.of(MockTransportService.TestPlugin.class);
}

public void testPrimaryRelocationShouldNotMarkIndexRequestAsRetry() throws Exception {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Do we have the opposite test, i.e., that we still mark as retry when necessary?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

There are also previous unit tests. I've also added and update some in TransportReplicationActionTests. The change set touches fewer patch than the last one though (see here).

bulk.add(prepareIndex("index1").setSource("text", randomAlphaOfLength(10)));
}
final var listener = bulk.execute();
continueRelocationLatch.countDown();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Should we wait for the bulk to arrive and wait for the permit - to ensure we are in the handoff case and not the reroute phase?

We should probably also have tests for the reroute phase though.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

these are addressed i think now.

}
final var listener = bulk.execute();
continueRelocationLatch.countDown();
safeAwait(node1ForwardedTheBulk);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I am not sure waiting here does anything beyond waiting for the listener? We could assert that it got forwarded after the listener invocation instead, but we already check that it was received by node 2, which I think is enough.

final var bulkResponse = listener.get();
assertThat(node2ReceivedTheBulk.get(), equalTo(true));
assertThat(bulkResponse.hasFailures(), equalTo(false));
assertThat(bulkResponse.getItems().length, equalTo(indexRequestsWithId + indexRequestsWithoutId));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Should we verify that the ids returned can be looked up? I imagine that if anything went wrong with the retry mechanism, i.e., we indeed did process the same auto-id insertion twice, the lookup might fail.

try {
// if we got disconnected from the node, or the node / shard is not in the right state (being closed)
final Throwable cause = exp.unwrapCause();
boolean markAsRetry = true;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Can we make the calculation here a method instead? Together with a dedicated exception that seems like it could operate purely on the exception. It can have a default return true in TransportReplicationAction and be overridden in TransportShardBulkAction to ensure we only impact that.

It can also be called inline below instead then.

Copy link
Copy Markdown
Member Author

@pxsalehi pxsalehi left a comment

Choose a reason for hiding this comment

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

Thanks for the review @henningandersen. this is ready for another round.

return List.of(MockTransportService.TestPlugin.class);
}

public void testPrimaryRelocationShouldNotMarkIndexRequestAsRetry() throws Exception {
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

There are also previous unit tests. I've also added and update some in TransportReplicationActionTests. The change set touches fewer patch than the last one though (see here).

bulk.add(prepareIndex("index1").setSource("text", randomAlphaOfLength(10)));
}
final var listener = bulk.execute();
continueRelocationLatch.countDown();
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

these are addressed i think now.

@pxsalehi
Copy link
Copy Markdown
Member Author

CI failures are :x-pack:plugin:esql-datasource-ndjson:qa:javaRestTest and not related.

Copy link
Copy Markdown
Contributor

@henningandersen henningandersen left a comment

Choose a reason for hiding this comment

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

LGTM.

* the first time.
* Called before this replication request is retried.
* <p>
* {@code markAsRetry} controls whether request should be marked as retry or not. For some retry paths (for example
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Can we name this similar to the exception boolean, i.e., possiblyExecuted? I find that slightly better since onRetry with a mark boolean seems slightly confusing.

exp
);
retry(exp);
boolean markAsRetry = true;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

can we keep the possiblyExecuted naming?

retry(failure, true);
}

void retry(Exception failure, boolean markRequestAsRetry) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Also possiblyExecuted naming here.

@pxsalehi pxsalehi enabled auto-merge (squash) February 23, 2026 16:36
@pxsalehi pxsalehi merged commit e0323eb into elastic:main Feb 23, 2026
35 checks passed
jdconrad pushed a commit to jdconrad/elasticsearch that referenced this pull request Feb 24, 2026
…ns (elastic#142157)

I've added a flag to `RetryOnPrimaryException` which indicates whether the request which is to be retried might have ran or not. If true (default value and default behaviour before this PR) the request is marked as retry. If false, it is safe to run the request as if it is not a retry, and we skip marking it as a retry. This should impact only [one specific path](https://github.com/elastic/elasticsearch/blob/f8908057ea3e95374b720c2a5f70d2598220569e/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java#L493) of the retries in the TransportReplicationAction.

Assisted by Cursor

Closes elastic#141586.
Relates ES-14121
szybia added a commit to szybia/elasticsearch that referenced this pull request Feb 24, 2026
…on-sliced-reindex

* upstream/main:
  Update docs for v9.3.1 release (elastic#142887)
  Update docs for v9.2.6 release (elastic#142888)
  Improves visibility of vector index options and inference configuration (elastic#141653)
  Disable CAE in microsoft-graph-authz plugin (elastic#142848)
  Small improvements to `GetSnapshotsIT#testAllFeatures` (elastic#142825)
  Fix IndexSettingsTests synthetic ID tests (elastic#142654)
  [Test] Unmute tests of SnapshotShutdownIT (elastic#142921)
  Fixing metrics_info.json kibana definition file name (elastic#142813)
  [Packaging] Disable glibc 2.43 malloc huge pages in Wolfi images (elastic#142894)
  Mute org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsTSDBSyntheticIdIntegTests testSearchableSnapshot elastic#142918
  Add shard heap usage to ClusterInfo (elastic#139557)
  ESQL: Load script fields row-by-row (elastic#142807)
  ESQL: Consolidate doc values memory tracking (elastic#142816)
  ES-14124  Create Index Count Limit User documentation Page (elastic#142570)
  Add a es819 codec test to verify tryRead returns null if may contain duplicates (elastic#142409)
  Support arithmetic operations for dense_vectors: scalar version (elastic#141060)
  [Transform] Allow project_routing (elastic#142421)
  Refactor query rewrite async actions for knn and sparse_vector queries (elastic#142889)
  Do not mark bulk indexing requests as retried after primary relocations (elastic#142157)
sidosera pushed a commit to sidosera/elasticsearch that referenced this pull request Feb 24, 2026
…ns (elastic#142157)

I've added a flag to `RetryOnPrimaryException` which indicates whether the request which is to be retried might have ran or not. If true (default value and default behaviour before this PR) the request is marked as retry. If false, it is safe to run the request as if it is not a retry, and we skip marking it as a retry. This should impact only [one specific path](https://github.com/elastic/elasticsearch/blob/f8908057ea3e95374b720c2a5f70d2598220569e/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java#L493) of the retries in the TransportReplicationAction.

Assisted by Cursor

Closes elastic#141586.
Relates ES-14121
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

:Distributed/CRUD A catch all label for issues around indexing, updating and getting a doc by id. Not search. >enhancement Team:Distributed Meta label for distributed team. v9.4.0

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Do not mark bulk indexing requests as retried after primary relocations

3 participants