From 1575e947fddd636cc0d1b037fda0ad956199af0e Mon Sep 17 00:00:00 2001 From: vikasvb90 Date: Tue, 7 Jan 2025 12:51:55 +0530 Subject: [PATCH] Child replica recovery --- .../shardsplit/InPlaceShardSplitIT.java | 53 ++- .../segments/TransportPitSegmentsAction.java | 1 - .../action/bulk/TransportShardBulkAction.java | 14 +- .../replication/ReplicationOperation.java | 9 - .../replication/TransportWriteAction.java | 2 +- .../action/shard/ShardStateAction.java | 314 +++++++++++------- .../cluster/metadata/IndexMetadata.java | 34 +- .../MetadataInPlaceShardSplitService.java | 1 + .../cluster/metadata/SplitShardsMetadata.java | 11 +- .../cluster/routing/AllocationId.java | 4 +- .../cluster/routing/IndexRoutingTable.java | 44 ++- .../routing/IndexShardRoutingTable.java | 2 +- .../routing/RoutingChangesObserver.java | 34 ++ .../cluster/routing/RoutingNodes.java | 229 +++++++++---- .../cluster/routing/RoutingTable.java | 24 +- .../cluster/routing/ShardRouting.java | 300 +++++++++++------ .../cluster/routing/UnassignedInfo.java | 6 +- .../routing/allocation/AllocationService.java | 4 +- .../allocation/IndexMetadataUpdater.java | 61 +++- .../RoutingNodesChangedObserver.java | 14 + .../allocator/BalancedShardsAllocator.java | 1 + .../allocator/LocalShardsBalancer.java | 131 +++++++- .../allocator/RemoteShardsBalancer.java | 4 + .../allocation/allocator/ShardsBalancer.java | 5 + .../InPlaceShardSplitAllocationDecider.java | 18 +- ...caAfterPrimaryActiveAllocationDecider.java | 4 +- .../decider/SameShardAllocationDecider.java | 7 +- .../decider/ThrottlingAllocationDecider.java | 25 +- .../index/engine/InternalEngine.java | 14 +- .../index/seqno/ReplicationTracker.java | 22 +- .../opensearch/index/shard/IndexShard.java | 22 +- .../opensearch/index/translog/Translog.java | 20 +- .../opensearch/indices/IndicesService.java | 14 +- .../cluster/IndicesClusterStateService.java | 77 ++++- .../LocalStorePeerRecoverySourceHandler.java | 108 +++--- .../recovery/PeerRecoverySourceService.java | 29 +- .../recovery/RecoverySourceHandler.java | 102 +++++- .../RecoverySourceHandlerFactory.java | 9 +- .../RemoteStorePeerRecoverySourceHandler.java | 14 +- .../InPlaceShardSplitRecoveryService.java | 193 +++++++---- ...nPlaceShardSplitRecoverySourceHandler.java | 92 ++++- ...nPlaceShardSplitRecoveryTargetHandler.java | 97 ++---- .../InPlaceShardsRecoveryFailedException.java | 10 +- .../SegmentReplicationTargetService.java | 2 +- .../replication/common/ReplicationTimer.java | 3 + .../main/java/org/opensearch/node/Node.java | 8 +- ...rdFailedClusterStateTaskExecutorTests.java | 16 +- ...dStartedClusterStateTaskExecutorTests.java | 20 +- .../action/shard/ShardStateActionTests.java | 6 +- .../cluster/routing/ShardRoutingTests.java | 25 +- .../allocation/InSyncAllocationIdTests.java | 8 +- ...actIndicesClusterStateServiceTestCase.java | 18 +- .../indices/cluster/ClusterStateChanges.java | 4 +- .../PeerRecoverySourceServiceTests.java | 8 +- .../snapshots/SnapshotResiliencyTests.java | 3 +- .../cluster/routing/ShardRoutingHelper.java | 10 +- .../cluster/routing/TestShardRouting.java | 6 - .../index/shard/IndexShardTestCase.java | 3 +- 58 files changed, 1611 insertions(+), 708 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/shardsplit/InPlaceShardSplitIT.java b/server/src/internalClusterTest/java/org/opensearch/shardsplit/InPlaceShardSplitIT.java index 7627836c119b4..ab827667cc791 100644 --- a/server/src/internalClusterTest/java/org/opensearch/shardsplit/InPlaceShardSplitIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/shardsplit/InPlaceShardSplitIT.java @@ -17,6 +17,8 @@ import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.ShardRange; +import org.opensearch.cluster.routing.RoutingNode; +import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.Priority; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -55,20 +57,34 @@ private Set triggerSplitAndGetChildShardIds(int parentShardId, int numb return Arrays.stream(shards).map(ShardRange::getShardId).collect(Collectors.toSet()); } - private void waitForSplit(int numberOfSplits, Set childShardIds, int parentShardId) throws Exception { + private void waitForSplit(int numberOfSplits, Set childShardIds, int parentShardId, int replicaCount) throws Exception { final long maxWaitTimeMs = Math.max(190 * 1000, 200 * numberOfSplits); assertBusy(() -> { ShardStats[] shardStats = client().admin().indices().prepareStats("test").get().getShards(); int startedChildShards = 0; for (ShardStats shardStat : shardStats) { - if (shardStat.getShardRouting().shardId().id() == parentShardId && shardStat.getShardRouting().started()) { + ShardRouting shardRouting = shardStat.getShardRouting(); + if (shardRouting.primary() && shardRouting.shardId().id() == parentShardId && shardStat.getShardRouting().started()) { throw new Exception("Splitting of shard id " + parentShardId + " failed "); } else if (childShardIds.contains(shardStat.getShardRouting().shardId().id())) { - startedChildShards ++; + startedChildShards++; } } - assertEquals(numberOfSplits, startedChildShards); + if (numberOfSplits + (replicaCount + 1) == startedChildShards) { + System.out.println(); + } + assertEquals(numberOfSplits * (replicaCount + 1), startedChildShards); +// ClusterState state = client().admin().cluster().prepareState().get().getState(); +// int startedChildReplicas = 0; +// for (RoutingNode routingNode : state.getRoutingNodes()) { +// for (ShardRouting shardRouting : routingNode) { +// if (shardRouting.isStartedChildReplica()) { +// startedChildReplicas++; +// } +// } +// } +// assertEquals(numberOfSplits * (replicaCount), startedChildReplicas); }, maxWaitTimeMs, TimeUnit.MILLISECONDS); assertClusterHealth(); @@ -105,11 +121,11 @@ private void verifyAfterSplit(long totalIndexedDocs, Set ids, int parent assertEquals(childShardIds, newServingChildShardIds); refresh("test"); - ShardStats[] stats = client().admin().indices().prepareStats("test").get().getShards(); - for (ShardStats shardStat : stats) { - logger.info("Shard stat after first indexing of shard " + shardStat.getShardRouting().shardId().id() + " docs: " - + shardStat.getStats().indexing.getTotal().getIndexCount() + " seq no: " + shardStat.getSeqNoStats().getMaxSeqNo()); - } +// ShardStats[] stats = client().admin().indices().prepareStats("test").get().getShards(); +// for (ShardStats shardStat : stats) { +// logger.info("Shard stat after first indexing of shard " + shardStat.getShardRouting().shardId().id() + " docs: " +// + shardStat.getStats().indexing.getTotal().getIndexCount() + " seq no: " + shardStat.getSeqNoStats().getMaxSeqNo()); +// } SearchHits hits = client().prepareSearch("test") .setQuery(matchAllQuery()) @@ -129,22 +145,23 @@ private void verifyAfterSplit(long totalIndexedDocs, Set ids, int parent public void testShardSplit() throws Exception { internalCluster().startNodes(2); + int replicaCount = 2; prepareCreate("test", Settings.builder().put("index.number_of_shards", 3) - .put("index.number_of_replicas", 0)).get(); + .put("index.number_of_replicas", replicaCount)).get(); ensureGreen(); - int numDocs = scaledRandomIntBetween(200, 2500); + int numDocs = scaledRandomIntBetween(1500, 2400); try (BackgroundIndexer indexer = new BackgroundIndexer("test", MapperService.SINGLE_MAPPING_NAME, client(), numDocs, 4)) { logger.info("--> waiting for {} docs to be indexed ...", numDocs); waitForDocs(numDocs, indexer); logger.info("--> {} docs indexed", numDocs); - numDocs = scaledRandomIntBetween(200, 1000); + numDocs = scaledRandomIntBetween(5000, 7500); logger.info("--> Allow indexer to index [{}] more documents", numDocs); indexer.continueIndexing(numDocs); int numberOfSplits = 3, parentShardId = 0; logger.info("--> starting split..."); Set childShardIds = triggerSplitAndGetChildShardIds(parentShardId, numberOfSplits); logger.info("--> waiting for shards to be split ..."); - waitForSplit(numberOfSplits, childShardIds, parentShardId); + waitForSplit(numberOfSplits, childShardIds, parentShardId, replicaCount); logger.info("--> Shard split completed ..."); logger.info("--> Verifying after split ..."); indexer.pauseIndexing(); @@ -155,8 +172,9 @@ public void testShardSplit() throws Exception { public void testSplittingShardHavingNonEmptyCommit() throws Exception { internalCluster().startNodes(2); + int replicaCount = 0; prepareCreate("test", Settings.builder().put("index.number_of_shards", 1) - .put("index.number_of_replicas", 0)).get(); + .put("index.number_of_replicas", replicaCount)).get(); ensureGreen(); int numDocs = scaledRandomIntBetween(200, 2500); try (BackgroundIndexer indexer = new BackgroundIndexer("test", MapperService.SINGLE_MAPPING_NAME, client(), numDocs, 4)) { @@ -176,7 +194,7 @@ public void testSplittingShardHavingNonEmptyCommit() throws Exception { logger.info("--> starting split..."); Set childShardIds = triggerSplitAndGetChildShardIds(parentShardId, numberOfSplits); logger.info("--> waiting for shards to be split ..."); - waitForSplit(numberOfSplits, childShardIds, parentShardId); + waitForSplit(numberOfSplits, childShardIds, parentShardId, replicaCount); logger.info("--> Shard split completed ..."); logger.info("--> Verifying after split ..."); indexer.pauseIndexing(); @@ -187,8 +205,9 @@ public void testSplittingShardHavingNonEmptyCommit() throws Exception { public void testSplittingShardWithNoTranslogReplay() throws Exception { internalCluster().startNodes(2); + int replicaCount = 0; prepareCreate("test", Settings.builder().put("index.number_of_shards", 1) - .put("index.number_of_replicas", 0)).get(); + .put("index.number_of_replicas", replicaCount)).get(); ensureGreen(); int numDocs = scaledRandomIntBetween(200, 2500); try (BackgroundIndexer indexer = new BackgroundIndexer("test", MapperService.SINGLE_MAPPING_NAME, client(), numDocs, 4)) { @@ -205,7 +224,7 @@ public void testSplittingShardWithNoTranslogReplay() throws Exception { logger.info("--> starting split..."); Set childShardIds = triggerSplitAndGetChildShardIds(parentShardId, numberOfSplits); logger.info("--> waiting for shards to be split ..."); - waitForSplit(numberOfSplits, childShardIds, parentShardId); + waitForSplit(numberOfSplits, childShardIds, parentShardId, replicaCount); logger.info("--> Shard split completed ..."); logger.info("--> Verifying after split ..."); verifyAfterSplit(indexer.totalIndexedDocs(), indexer.getIds(), parentShardId, childShardIds); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/segments/TransportPitSegmentsAction.java b/server/src/main/java/org/opensearch/action/admin/indices/segments/TransportPitSegmentsAction.java index 23cd48ae09341..15a1db4cff45f 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/segments/TransportPitSegmentsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/segments/TransportPitSegmentsAction.java @@ -241,7 +241,6 @@ public PitAwareShardRouting( allocationId, expectedShardSize, null, - null, null ); this.pitId = pitId; diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java index 997d0f60192bf..9f0f9f663c710 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java @@ -668,6 +668,9 @@ static boolean executeBulkItemRequest( request.isRetry() ); } +// if (primary.isPrimaryMode() && primary.shardId().id() == 0) { +// logger.info("Indexing operation sequence " + result.getSeqNo() + " on shard 0."); +// } if (result.getResultType() == Engine.Result.Type.MAPPING_UPDATE_REQUIRED) { // if (context.getBulkShardRequest().shardId().id() == 0) { // logger.info("Executing bulk item mapping update"); @@ -873,7 +876,7 @@ public static Translog.Location performOnReplica(BulkShardRequest request, Index final BulkItemResponse response = item.getPrimaryResponse(); final Engine.Result operationResult; boolean discardOperation = false; - if (replica.routingEntry().isSplitTarget() == true) { + if (replica.getParentShardId() != null) { IndexMetadata indexMetadata = replica.indexSettings().getIndexMetadata(); // Discard operations belonging to a different child shard. This can happen during in-place shard // split recovery where after all child shards are added to replication tracker, bulk @@ -881,6 +884,13 @@ public static Translog.Location performOnReplica(BulkShardRequest request, Index int computedShardId = OperationRouting.generateShardId(indexMetadata, item.request().id(), item.request().routing(), true); discardOperation = computedShardId != replica.shardId().id(); +// if (replica.routingEntry().isStartedChildReplica()) { +// logger.info("Processing seq no." + response.getResponse().getSeqNo() + " on replica child " +// + replica.shardId().id() + ", discarding " + discardOperation); +// } else if (replica.routingEntry().isSplitTarget()) { +// logger.info("Processing seq no. on child primary" + response.getResponse().getSeqNo() + " on replica child " +// + replica.shardId().id() + ", discarding " + discardOperation); +// } } if (item.getPrimaryResponse().isFailed()) { @@ -909,7 +919,7 @@ public static Translog.Location performOnReplica(BulkShardRequest request, Index operationResult = replica.markSeqNoAsNoop( response.getResponse().getSeqNo(), response.getResponse().getPrimaryTerm(), - "op belongs to another child shard" + Translog.NoOp.FILLING_GAPS ); } else { operationResult = performOpOnReplica(response.getResponse(), item.request(), replica); diff --git a/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java index d70d97a0f8482..1680bfad722e9 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java @@ -169,9 +169,6 @@ private void handlePrimaryResult(final PrimaryResultT primaryResult) { if (TransportShardBulkAction.debugRequest.get() && request.shardId().id() == 0) { logger.info("Handling primary result for id: " + id); } - if (request.shardId.id() == 3) { - System.out.println(); - } if (replicaRequest != null) { if (logger.isTraceEnabled()) { logger.trace("[{}] op [{}] completed on primary for request [{}]", primary.routingEntry().shardId(), opType, request); @@ -391,12 +388,6 @@ public boolean shouldRetry(Exception e) { private void updateCheckPoints(ShardRouting shard, LongSupplier localCheckpointSupplier, LongSupplier globalCheckpointSupplier) { try { - if (TransportShardBulkAction.debugRequest.get() == true) { - logger.info("Updating checkpoint for shard " + shard.shardId().id() - + " on primary shard allocation " + primary.routingEntry().allocationId().getId() - + " for replica shard allocation " + shard.allocationId().getId() - + " with checkpoint " + localCheckpointSupplier.getAsLong()); - } primary.updateLocalCheckpointForShard(shard.allocationId().getId(), localCheckpointSupplier.getAsLong()); primary.updateGlobalCheckpointForShard(shard.allocationId().getId(), globalCheckpointSupplier.getAsLong()); } catch (final AlreadyClosedException e) { diff --git a/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java index 589dcf29fc0ba..91c887c4bde73 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java @@ -577,7 +577,7 @@ public void failShardIfNeeded( // If a write action fails due to the closure of the primary shard // then the replicas should not be marked as failed since they are // still up-to-date with the (now closed) primary shard - if (exception instanceof PrimaryShardClosedException == false && replica.isSplitTarget() == false) { + if (exception instanceof PrimaryShardClosedException == false) { shardStateAction.remoteShardFailed( replica.shardId(), replica.allocationId().getId(), diff --git a/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java b/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java index 71244d544d2e4..ca38754a5bb88 100644 --- a/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java +++ b/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java @@ -255,9 +255,20 @@ public void remoteShardFailed( @Nullable final Exception failure, ActionListener listener ) { + ShardRouting shardRouting = clusterService.state().getRoutingNodes().getByAllocationId(shardId, allocationId); + final FailedShardEntry failedShardEntry; + if (shardRouting == null) { + logger.debug("Remote shard [{}] not known to current node [{}] ", shardId, clusterService.localNode().getId()); + failedShardEntry = new FailedShardEntry(shardId, allocationId, primaryTerm, message, failure, markAsStale, + null, null, null); + } else { + failedShardEntry = new FailedShardEntry(shardId, allocationId, primaryTerm, message, failure, markAsStale, + null, shardRouting.getParentShardId(), shardRouting.allocationId().getParentAllocationId()); + } + assert primaryTerm > 0L : "primary term should be strictly positive"; remoteFailedShardsDeduplicator.executeOnce( - new FailedShardEntry(shardId, allocationId, primaryTerm, message, failure, markAsStale, null), + failedShardEntry, listener, (req, reqListener) -> sendShardAction(SHARD_FAILED_ACTION_NAME, clusterService.state(), req, reqListener) ); @@ -288,20 +299,6 @@ public void localShardFailed( @Nullable final Exception failure, ActionListener listener, final ClusterState currentState - ) { - localShardFailed(shardRouting, message, failure, listener, currentState, null); - } - - /** - * Send a shard failed request to the cluster-manager node to update the cluster state when a shard on the local node failed. - */ - public void localShardFailed( - final ShardRouting shardRouting, - final String message, - @Nullable final Exception failure, - ActionListener listener, - final ClusterState currentState, - final Boolean childShardsFailedEvent ) { FailedShardEntry shardEntry = new FailedShardEntry( shardRouting.shardId(), @@ -310,7 +307,9 @@ public void localShardFailed( message, failure, true, - childShardsFailedEvent + null, + shardRouting.getParentShardId(), + shardRouting.allocationId().getParentAllocationId() ); sendShardAction(SHARD_FAILED_ACTION_NAME, currentState, shardEntry, listener); } @@ -476,72 +475,89 @@ public ClusterTasksResult execute(ClusterState currentState, L logger.debug("{} ignoring shard failed task [{}] (unknown index {})", task.shardId, task, task.shardId.getIndex()); batchResultBuilder.success(task); } else { - // The primary term is 0 if the shard failed itself. It is > 0 if a write was done on a primary but was failed to be - // replicated to the shard copy with the provided allocation id. In case where the shard failed itself, it's ok to just - // remove the corresponding routing entry from the routing table. In case where a write could not be replicated, - // however, it is important to ensure that the shard copy with the missing write is considered as stale from that point - // on, which is implemented by removing the allocation id of the shard copy from the in-sync allocations set. - // We check here that the primary to which the write happened was not already failed in an earlier cluster state update. - // This prevents situations where a new primary has already been selected and replication failures from an old stale - // primary unnecessarily fail currently active shards. - if (task.primaryTerm > 0 && Boolean.TRUE.equals(task.childShardsFailedEvent) == false) { - long currentPrimaryTerm = indexMetadata.primaryTerm(task.shardId.id()); - if (currentPrimaryTerm != task.primaryTerm) { - assert currentPrimaryTerm > task.primaryTerm : "received a primary term with a higher term than in the " - + "current cluster state (received [" - + task.primaryTerm - + "] but current is [" - + currentPrimaryTerm - + "])"; - logger.debug( - "{} failing shard failed task [{}] (primary term {} does not match current term {})", - task.shardId, - task, - task.primaryTerm, - indexMetadata.primaryTerm(task.shardId.id()) - ); - batchResultBuilder.failure( - task, - new NoLongerPrimaryShardException( - task.shardId, - "primary term [" - + task.primaryTerm - + "] did not match current primary term [" - + currentPrimaryTerm - + "]" - ) - ); + if (task.parentShardId != null) { + ShardRouting parentShard = currentState.getRoutingNodes().getByAllocationId(task.parentShardId, task.parentAllocationId); + if (parentShard == null || parentShard.splitting() == false) { + batchResultBuilder.success(task); continue; } - } - - ShardRouting matched = currentState.getRoutingTable().getByAllocationId(task.shardId, task.allocationId); - if (matched == null && Boolean.TRUE.equals(task.childShardsFailedEvent) == false) { - Set inSyncAllocationIds = indexMetadata.inSyncAllocationIds(task.shardId.id()); - // mark shard copies without routing entries that are in in-sync allocations set only as stale if the reason why - // they were failed is because a write made it into the primary but not to this copy (which corresponds to - // the check "primaryTerm > 0"). - if (task.primaryTerm > 0 && (inSyncAllocationIds != null || inSyncAllocationIds.contains(task.allocationId))) { - logger.debug("{} marking shard {} as stale (shard failed task: [{}])", task.shardId, task.allocationId, task); + if (Boolean.TRUE.equals(task.splitFailed)) { tasksToBeApplied.add(task); - staleShardsToBeApplied.add(new StaleShard(task.shardId, task.allocationId)); + logger.debug("{} failing split {} (shard failed task: [{}])", task.shardId, parentShard, task); + for (ShardRouting childShard : parentShard.getRecoveringChildShards()) { + failedShardsToBeApplied.add(new FailedShard(childShard, task.message, task.failure, task.markAsStale)); + } } else { - // tasks that correspond to non-existent shards are marked as successful - logger.debug("{} ignoring shard failed task [{}] (shard does not exist anymore)", task.shardId, task); - batchResultBuilder.success(task); - } - } else if (Boolean.TRUE.equals(task.childShardsFailedEvent)) { - logger.debug("{} failing child shards {} (shard failed task: [{}])", task.shardId, matched, task); - tasksToBeApplied.add(task); - for (ShardRouting childShard : matched.getRecoveringChildShards()) { - numberOfFailedChildShards++; + ShardRouting childShard = currentState.getRoutingNodes().getByAllocationId(task.shardId, task.allocationId); + if (childShard == null) { + // We already cancelled split + batchResultBuilder.success(task); + continue; + } + tasksToBeApplied.add(task); failedShardsToBeApplied.add(new FailedShard(childShard, task.message, task.failure, task.markAsStale)); } } else { - // failing a shard also possibly marks it as stale (see IndexMetadataUpdater) - logger.debug("{} failing shard {} (shard failed task: [{}])", task.shardId, matched, task); - tasksToBeApplied.add(task); - failedShardsToBeApplied.add(new FailedShard(matched, task.message, task.failure, task.markAsStale)); + // The primary term is 0 if the shard failed itself. It is > 0 if a write was done on a primary but was failed to be + // replicated to the shard copy with the provided allocation id. In case where the shard failed itself, it's ok to just + // remove the corresponding routing entry from the routing table. In case where a write could not be replicated, + // however, it is important to ensure that the shard copy with the missing write is considered as stale from that point + // on, which is implemented by removing the allocation id of the shard copy from the in-sync allocations set. + // We check here that the primary to which the write happened was not already failed in an earlier cluster state update. + // This prevents situations where a new primary has already been selected and replication failures from an old stale + // primary unnecessarily fail currently active shards. + if (task.primaryTerm > 0) { + long currentPrimaryTerm = indexMetadata.primaryTerm(task.shardId.id()); + if (currentPrimaryTerm != task.primaryTerm) { + assert currentPrimaryTerm > task.primaryTerm : "received a primary term with a higher term than in the " + + "current cluster state (received [" + + task.primaryTerm + + "] but current is [" + + currentPrimaryTerm + + "])"; + logger.debug( + "{} failing shard failed task [{}] (primary term {} does not match current term {})", + task.shardId, + task, + task.primaryTerm, + indexMetadata.primaryTerm(task.shardId.id()) + ); + batchResultBuilder.failure( + task, + new NoLongerPrimaryShardException( + task.shardId, + "primary term [" + + task.primaryTerm + + "] did not match current primary term [" + + currentPrimaryTerm + + "]" + ) + ); + continue; + } + } + + ShardRouting matched = currentState.getRoutingTable().getByAllocationId(task.shardId, task.allocationId); + if (matched == null) { + Set inSyncAllocationIds = indexMetadata.inSyncAllocationIds(task.shardId.id()); + // mark shard copies without routing entries that are in in-sync allocations set only as stale if the reason why + // they were failed is because a write made it into the primary but not to this copy (which corresponds to + // the check "primaryTerm > 0"). + if (task.primaryTerm > 0 && (inSyncAllocationIds != null || inSyncAllocationIds.contains(task.allocationId))) { + logger.debug("{} marking shard {} as stale (shard failed task: [{}])", task.shardId, task.allocationId, task); + tasksToBeApplied.add(task); + staleShardsToBeApplied.add(new StaleShard(task.shardId, task.allocationId)); + } else { + // tasks that correspond to non-existent shards are marked as successful + logger.debug("{} ignoring shard failed task [{}] (shard does not exist anymore)", task.shardId, task); + batchResultBuilder.success(task); + } + } else { + // failing a shard also possibly marks it as stale (see IndexMetadataUpdater) + logger.debug("{} failing shard {} (shard failed task: [{}])", task.shardId, matched, task); + tasksToBeApplied.add(task); + failedShardsToBeApplied.add(new FailedShard(matched, task.message, task.failure, task.markAsStale)); + } } } } @@ -603,7 +619,11 @@ public static class FailedShardEntry extends TransportRequest { final boolean markAsStale; @Nullable - final Boolean childShardsFailedEvent; + final Boolean splitFailed; + @Nullable + final ShardId parentShardId; + @Nullable + final String parentAllocationId; FailedShardEntry(StreamInput in) throws IOException { @@ -615,9 +635,13 @@ public static class FailedShardEntry extends TransportRequest { failure = in.readException(); markAsStale = in.readBoolean(); if (in.getVersion().onOrAfter(Version.V_3_0_0)) { - childShardsFailedEvent = in.readOptionalBoolean(); + splitFailed = in.readOptionalBoolean(); + parentShardId = in.readOptionalWriteable(ShardId::new); + parentAllocationId = in.readOptionalString(); } else { - childShardsFailedEvent = null; + splitFailed = null; + parentShardId = null; + parentAllocationId = null; } } @@ -628,7 +652,9 @@ public FailedShardEntry( String message, @Nullable Exception failure, boolean markAsStale, - Boolean childShardsFailedEvent + final Boolean splitFailed, + final ShardId parentShardId, + final String parentAllocationId ) { this.shardId = shardId; this.allocationId = allocationId; @@ -636,7 +662,9 @@ public FailedShardEntry( this.message = message; this.failure = failure; this.markAsStale = markAsStale; - this.childShardsFailedEvent = childShardsFailedEvent; + this.splitFailed = splitFailed; + this.parentShardId = parentShardId; + this.parentAllocationId = parentAllocationId; } public ShardId getShardId() { @@ -657,12 +685,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeException(failure); out.writeBoolean(markAsStale); if (out.getVersion().onOrAfter(Version.V_3_0_0)) { - out.writeOptionalBoolean(childShardsFailedEvent); + out.writeOptionalBoolean(splitFailed); + out.writeOptionalWriteable(parentShardId); + out.writeOptionalString(parentAllocationId); } else { - if (childShardsFailedEvent != null) { - // In-progress shard split is not allowed in a mixed cluster where node(s) with an unsupported split - // version is present. Hence, we also don't want to allow a node with an unsupported version - // to get this state while shard split is in-progress. + if (parentShardId != null) { throw new IllegalStateException("In-place split not allowed on older versions."); } } @@ -679,7 +706,13 @@ public String toString() { components.add("failure [" + ExceptionsHelper.detailedMessage(failure) + "]"); } components.add("markAsStale [" + markAsStale + "]"); - components.add("childShardsFailedEvent [" + Boolean.TRUE.equals(childShardsFailedEvent) + "]"); + if (parentShardId != null) { + if (splitFailed != null) { + components.add("splitFailed [" + splitFailed + "]"); + } + components.add("parentShardId [" + parentShardId + "]"); + components.add("parentAllocationId [" + parentAllocationId + "]"); + } return String.join(", ", components); } @@ -693,12 +726,15 @@ public boolean equals(Object o) { && Objects.equals(this.allocationId, that.allocationId) && primaryTerm == that.primaryTerm && markAsStale == that.markAsStale - && childShardsFailedEvent == that.childShardsFailedEvent; + && splitFailed == that.splitFailed + && Objects.equals(this.parentShardId, that.parentShardId) + && Objects.equals(this.parentAllocationId, that.parentAllocationId); } + @Override public int hashCode() { - return Objects.hash(shardId, allocationId, primaryTerm, markAsStale, childShardsFailedEvent); + return Objects.hash(shardId, allocationId, primaryTerm, markAsStale, splitFailed, parentShardId, parentAllocationId); } } @@ -723,23 +759,27 @@ public void shardStarted( shardRouting.allocationId().getId(), primaryTerm, message, - null + null, + shardRouting.getParentShardId(), + shardRouting.allocationId().getParentAllocationId() ); sendShardAction(SHARD_STARTED_ACTION_NAME, currentState, entry, listener); } public void childShardsStarted( - final ShardRouting sourceShardRouting, + final ShardRouting parentShardRouting, final long primaryTerm, final String message, final ActionListener listener ) { StartedShardEntry entry = new StartedShardEntry( - sourceShardRouting.shardId(), - sourceShardRouting.allocationId().getId(), + parentShardRouting.shardId(), + parentShardRouting.allocationId().getId(), primaryTerm, message, - true + true, + parentShardRouting.shardId(), + parentShardRouting.allocationId().getId() ); sendShardAction(SHARD_STARTED_ACTION_NAME, clusterService.state(), entry, listener); } @@ -812,7 +852,39 @@ public ClusterTasksResult execute(ClusterState currentState, Set seenShardRoutings = new HashSet<>(); // to prevent duplicates for (StartedShardEntry task : tasks) { final ShardRouting matched = currentState.getRoutingTable().getByAllocationId(task.shardId, task.allocationId); - if (matched == null) { + if (task.parentShardId != null) { + if (seenShardRoutings.contains(matched) == true) { + logger.info("Ignoring child event for shard " + matched); + } else { + logger.info("Processing child event for shard " + matched); + } + ShardRouting parentShard = currentState.getRoutingTable().getByAllocationId(task.parentShardId, task.parentAllocationId); + if (parentShard == null || parentShard.splitting() == false) { + builder.success(task); + continue; + } +// seenShardRoutings.add(matched); + if (Boolean.TRUE.equals(task.allChildPrimariesStarted)) { + logger.debug("{} starting child shards of {} (shard started task: [{}])", task.shardId, parentShard, task); + tasksToBeApplied.add(task); + for (ShardRouting childShard : parentShard.getRecoveringChildShards()) { + assert childShard.primary() == true || childShard.started(); + if (childShard.primary() == true) { + shardRoutingsToBeApplied.add(childShard); + } + } + } else { + logger.debug("{} starting replica child shard of {} (shard started task: [{}])", task.shardId, parentShard, task); + ShardRouting childReplica = currentState.getRoutingNodes().getByAllocationId(task.shardId, task.allocationId); + if (childReplica == null) { + // This means we have cancelled ongoing split earlier. + builder.success(task); + continue; + } + tasksToBeApplied.add(task); + shardRoutingsToBeApplied.add(childReplica); + } + } else if (matched == null) { // tasks that correspond to non-existent shards are marked as successful. The reason is that we resend shard started // events on every cluster state publishing that does not contain the shard as started yet. This means that old stale // requests might still be in flight even after the shard has already been started or failed on the cluster-manager. We @@ -843,7 +915,7 @@ public ClusterTasksResult execute(ClusterState currentState, continue; } } - if (matched.initializing() == false && Boolean.FALSE.equals(task.childShardsStartedEvent)) { + if (matched.initializing() == false) { assert matched.active() : "expected active shard routing for task " + task + " but found " + matched; // same as above, this might have been a stale in-flight request, so we just ignore. logger.debug( @@ -856,7 +928,7 @@ public ClusterTasksResult execute(ClusterState currentState, } else { // remove duplicate actions as allocation service expects a clean list without duplicates if (seenShardRoutings.contains(matched)) { - logger.trace( + logger.info( "{} ignoring shard started task [{}] (already scheduled to start {})", task.shardId, task, @@ -865,15 +937,8 @@ public ClusterTasksResult execute(ClusterState currentState, tasksToBeApplied.add(task); } else { tasksToBeApplied.add(task); - if (Boolean.TRUE.equals(task.childShardsStartedEvent)) { - logger.debug("{} starting child shards of {} (shard started task: [{}])", task.shardId, matched, task); - // matched shard is source shard in this case. - assert matched.splitting(); - shardRoutingsToBeApplied.addAll(Arrays.asList(matched.getRecoveringChildShards())); - } else { - logger.debug("{} starting shard {} (shard started task: [{}])", task.shardId, matched, task); - shardRoutingsToBeApplied.add(matched); - } + logger.debug("{} starting shard {} (shard started task: [{}])", task.shardId, matched, task); + shardRoutingsToBeApplied.add(matched); seenShardRoutings.add(matched); } } @@ -926,7 +991,12 @@ public static class StartedShardEntry extends TransportRequest { final String message; @Nullable - final Boolean childShardsStartedEvent; + final Boolean allChildPrimariesStarted; + @Nullable + final ShardId parentShardId; + @Nullable + final String parentAllocationId; + StartedShardEntry(StreamInput in) throws IOException { super(in); @@ -935,9 +1005,13 @@ public static class StartedShardEntry extends TransportRequest { primaryTerm = in.readVLong(); this.message = in.readString(); if (in.getVersion().onOrAfter(Version.V_3_0_0)) { - childShardsStartedEvent = in.readOptionalBoolean(); + allChildPrimariesStarted = in.readOptionalBoolean(); + parentShardId = in.readOptionalWriteable(ShardId::new); + parentAllocationId = in.readOptionalString(); } else { - childShardsStartedEvent = null; + allChildPrimariesStarted = null; + parentShardId = null; + parentAllocationId = null; } } @@ -946,13 +1020,17 @@ public StartedShardEntry( final String allocationId, final long primaryTerm, final String message, - Boolean childShardsStartedEvent + final Boolean allChildPrimariesStarted, + final ShardId parentShardId, + final String parentAllocationId ) { this.shardId = shardId; this.allocationId = allocationId; this.primaryTerm = primaryTerm; this.message = message; - this.childShardsStartedEvent = childShardsStartedEvent; + this.allChildPrimariesStarted = allChildPrimariesStarted; + this.parentShardId = parentShardId; + this.parentAllocationId = parentAllocationId; } @Override @@ -963,12 +1041,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(primaryTerm); out.writeString(message); if (out.getVersion().onOrAfter(Version.V_3_0_0)) { - out.writeOptionalBoolean(childShardsStartedEvent); + out.writeOptionalBoolean(allChildPrimariesStarted); + out.writeOptionalWriteable(parentShardId); + out.writeOptionalString(parentAllocationId); } else { - if (childShardsStartedEvent != null) { - // In-progress shard split is not allowed in a mixed cluster where node(s) with an unsupported split - // version is present. Hence, we also don't want to allow a node with an unsupported version - // to get this state while shard split is in-progress. + if (parentShardId != null) { throw new IllegalStateException("In-place split not allowed on older versions."); } } @@ -978,12 +1055,15 @@ public void writeTo(StreamOutput out) throws IOException { public String toString() { return String.format( Locale.ROOT, - "StartedShardEntry{shardId [%s], allocationId [%s], primary term [%d], " + "message [%s], Child shards started event [%s]}", + "StartedShardEntry{shardId [%s], allocationId [%s], primary term [%d], " + + "message [%s], allChildPrimariesStarted [%s], parentShardId [%s], parentAllocationId [%s]}", shardId, allocationId, primaryTerm, message, - Boolean.TRUE.equals(childShardsStartedEvent) + allChildPrimariesStarted, + parentShardId, + parentAllocationId ); } } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java index fa82faffb5674..7e836c4865ed3 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -76,6 +76,7 @@ import java.time.ZonedDateTime; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; @@ -1151,6 +1152,7 @@ public IndexMetadata apply(IndexMetadata part) { builder.aliases.putAll(aliases.apply(part.aliases)); builder.customMetadata.putAll(customData.apply(part.customData)); builder.inSyncAllocationIds.putAll(inSyncAllocationIds.apply(part.inSyncAllocationIds)); + builder.rolloverInfos.putAll(rolloverInfos.apply(part.rolloverInfos)); builder.system(part.isSystem); builder.splitShardsMetadata(splitMetadata.apply(part.splitShardsMetadata)); @@ -1353,20 +1355,27 @@ public int numberOfShards() { return settings.getAsInt(SETTING_NUMBER_OF_SHARDS, -1); } - public Builder updateMetadataForNewChildShards(Map newAllocationIds, int sourceShardId) { + public Builder updateMetadataForNewChildShards(Map newChildAllocationIds, int sourceShardId) { // Now update primary terms against child shard ids - int numOfFinalPrimaryTerms = this.primaryTerms.length + newAllocationIds.size(); + int numOfFinalPrimaryTerms = this.primaryTerms.length + newChildAllocationIds.size(); long []finalPrimaryTerms = Arrays.copyOf(this.primaryTerms, numOfFinalPrimaryTerms); long parentPrimaryTerm = this.primaryTerms[sourceShardId]; Arrays.fill(finalPrimaryTerms, this.primaryTerms.length, numOfFinalPrimaryTerms, parentPrimaryTerm); + finalPrimaryTerms[sourceShardId] = -1; this.primaryTerms = finalPrimaryTerms; - // Add in-sync allocations of child shards - newAllocationIds.forEach((shardId, newAllocationId) -> this.inSyncAllocationIds.put( - shardId, Sets.newHashSet(newAllocationId))); + // Add in-sync allocations of primary child shards + newChildAllocationIds.forEach((shardId, newAllocationId) -> { + Collection curAllocIds = this.inSyncAllocationIds.get(shardId); + Set shardAllocIds = curAllocIds == null ? new HashSet<>() : new HashSet<>(curAllocIds); + shardAllocIds.add(newAllocationId); + putInSyncAllocationIds(shardId, shardAllocIds); + }); + this.inSyncAllocationIds.remove(sourceShardId); + SplitShardsMetadata.Builder splitShardsMetadata = new SplitShardsMetadata.Builder(this.splitShardsMetadata); - splitShardsMetadata.updateSplitMetadataForChildShards(sourceShardId, newAllocationIds.keySet()); + splitShardsMetadata.updateSplitMetadataForChildShards(sourceShardId, newChildAllocationIds.keySet()); this.splitShardsMetadata = splitShardsMetadata.build(); numberOfShards(this.splitShardsMetadata.getNumberOfShards()); @@ -1376,6 +1385,9 @@ public Builder updateMetadataForNewChildShards(Map newAllocatio } public Builder cancelSplit(int sourceShardId) { + for (ShardRange child : splitShardsMetadata.getChildShardsOfParent(sourceShardId)) { + inSyncAllocationIds.remove(child.getShardId()); + } SplitShardsMetadata.Builder splitShardsMetadata = new SplitShardsMetadata.Builder(this.splitShardsMetadata); splitShardsMetadata.cancelSplit(sourceShardId); this.splitShardsMetadata = splitShardsMetadata.build(); @@ -1602,9 +1614,17 @@ public IndexMetadata build() { for (int i = 0; i < numberOfShards; i++) { if (inSyncAllocationIds.containsKey(i)) { filledInSyncAllocationIds.put(i, Collections.unmodifiableSet(new HashSet<>(inSyncAllocationIds.get(i)))); - } else { + } else if (splitShardsMetadata.isEmptyParentShard(i) == false) { filledInSyncAllocationIds.put(i, Collections.emptySet()); } + + if (splitShardsMetadata.isSplitOfShardInProgress(i) == true) { + for (ShardRange child : splitShardsMetadata.getChildShardsOfParent(i)) { + if (inSyncAllocationIds.containsKey(child.getShardId())) { + filledInSyncAllocationIds.put(child.getShardId(), inSyncAllocationIds.get(child.getShardId())); + } + } + } } final Map requireMap = INDEX_ROUTING_REQUIRE_GROUP_SETTING.getAsMap(settings); final DiscoveryNodeFilters requireFilters; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataInPlaceShardSplitService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataInPlaceShardSplitService.java index ed1636721e13b..d1f3a17217394 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataInPlaceShardSplitService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataInPlaceShardSplitService.java @@ -157,6 +157,7 @@ public ClusterState applyShardSplitRequest( RoutingTable routingTable = routingTableBuilder.build(); metadataBuilder.put(indexMetadataBuilder); + ClusterState updatedState = ClusterState.builder(currentState).metadata(metadataBuilder).routingTable(routingTable).build(); return rerouteRoutingTable.apply(updatedState, "shard [" + request.getShardId() + "] of index [" + request.getIndex() + "] split"); } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/SplitShardsMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/SplitShardsMetadata.java index 7dffb089fa6fb..08334f0b24f8f 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/SplitShardsMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/SplitShardsMetadata.java @@ -35,19 +35,13 @@ public class SplitShardsMetadata extends AbstractDiffable i private static final String KEY_MAX_SHARD_ID = "max_shard_id"; private static final String KEY_IN_PROGRESS_SPLIT_SHARD_ID = "in_progress_split_shard_id"; - - // Root shard id to flat list of all child shards under root. private final ShardRange[][] rootShardsToAllChildren; - // Mapping of a parent shard ID to children. This is a temporary map since a shard id of parent is reused - // in one of its children and triggering a split of a child which is using the shard id of parent can replace - // child shards of its parent with its own child shards. + // Mapping of a parent shard ID to children. private final Map parentToChildShards; private final int maxShardId; private final int inProgressSplitShardId; - - private SplitShardsMetadata(ShardRange[][] rootShardsToAllChildren, Map parentToChildShards, int inProgressSplitShardId, int maxShardId) { @@ -89,7 +83,6 @@ public int getShardIdOfHash(int rootShardId, int hash, boolean includeInProgress return rootShardId; } - ShardRange[] existingChildShards = rootShardsToAllChildren[rootShardId]; ShardRange shardRange = binarySearchShards(existingChildShards, hash); assert shardRange != null; @@ -317,7 +310,7 @@ private static Tuple findRootAndShard(int shardId, ShardRan return new Tuple<>(shardId, new ShardRange(shardId, Integer.MIN_VALUE, Integer.MAX_VALUE)); } - return null; + throw new IllegalArgumentException("Shard ID doesn't exist in the current list of shard ranges"); } public int getInProgressSplitShardId() { diff --git a/server/src/main/java/org/opensearch/cluster/routing/AllocationId.java b/server/src/main/java/org/opensearch/cluster/routing/AllocationId.java index 8c254b34134da..da30dec1680e3 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/AllocationId.java +++ b/server/src/main/java/org/opensearch/cluster/routing/AllocationId.java @@ -192,10 +192,10 @@ public static AllocationId newRelocation(AllocationId allocationId) { * Creates a new allocation id for a shard that is undergoing split, populating * the transient holder for splitChildAllocationIds. */ - public static AllocationId newSplit(AllocationId allocationId, int childShardsSize) { + public static AllocationId newSplit(AllocationId allocationId, int numberOfChildShards) { assert allocationId.getSplitChildAllocationIds() == null && allocationId.getParentAllocationId() == null; List splitChildAllocationIds = new ArrayList<>(); - for (int c = 0; c < childShardsSize; c++) { + for (int c = 0; c < numberOfChildShards; c++) { splitChildAllocationIds.add(UUIDs.randomBase64UUID()); } return new AllocationId(allocationId.getId(), null, splitChildAllocationIds, null); diff --git a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java index b7476e7fc814d..e3402677ea6b1 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java @@ -33,6 +33,7 @@ package org.opensearch.cluster.routing; import org.apache.lucene.util.CollectionUtil; +import org.opensearch.Version; import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; import org.opensearch.cluster.metadata.IndexMetadata; @@ -88,12 +89,15 @@ public class IndexRoutingTable extends AbstractDiffable imple // shards with state set to UNASSIGNED private final Map shards; + private final Map childReplicas; + private final List allActiveShards; - IndexRoutingTable(Index index, final Map shards) { + IndexRoutingTable(Index index, final Map shards, Map childReplicas) { this.index = index; this.shuffler = new RotationShardShuffler(Randomness.get().nextInt()); this.shards = Collections.unmodifiableMap(shards); + this.childReplicas = Collections.unmodifiableMap(childReplicas); List allActiveShards = new ArrayList<>(); for (IndexShardRoutingTable cursor : shards.values()) { for (ShardRouting shardRouting : cursor) { @@ -247,6 +251,10 @@ public IndexShardRoutingTable shard(int shardId) { return shards.get(shardId); } + public IndexShardRoutingTable childReplicaShard(int shardId) { + return childReplicas.get(shardId); + } + /** * Returns true if all shards are primary and active. Otherwise false. */ @@ -329,6 +337,7 @@ public boolean equals(Object o) { if (!index.equals(that.index)) return false; if (!shards.equals(that.shards)) return false; + if (!childReplicas.equals(that.childReplicas)) return false; return true; } @@ -337,6 +346,7 @@ public boolean equals(Object o) { public int hashCode() { int result = index.hashCode(); result = 31 * result + shards.hashCode(); + result = 31 * result + childReplicas.hashCode(); return result; } @@ -348,6 +358,12 @@ public static IndexRoutingTable readFrom(StreamInput in) throws IOException { for (int i = 0; i < size; i++) { builder.addIndexShard(IndexShardRoutingTable.Builder.readFromThin(in, index)); } + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + int noOfChildReplicas = in.readVInt(); + for (int i = 0; i < noOfChildReplicas; i++) { + builder.addChildReplicaShard(IndexShardRoutingTable.Builder.readFromThin(in, index)); + } + } return builder.build(); } @@ -363,6 +379,12 @@ public void writeTo(StreamOutput out) throws IOException { for (IndexShardRoutingTable indexShard : this) { IndexShardRoutingTable.Builder.writeToThin(indexShard, out); } + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeVInt(childReplicas.size()); + for (IndexShardRoutingTable indexShard : childReplicas.values()) { + IndexShardRoutingTable.Builder.writeToThin(indexShard, out); + } + } } public static Builder builder(Index index) { @@ -379,6 +401,7 @@ public static class Builder { private final Index index; private final Map shards = new HashMap<>(); + private final Map childReplicas = new HashMap<>(); public Builder(Index index) { this.index = index; @@ -643,6 +666,11 @@ public Builder addIndexShard(IndexShardRoutingTable indexShard) { return this; } + public Builder addChildReplicaShard(IndexShardRoutingTable indexShard) { + childReplicas.put(indexShard.shardId().id(), indexShard); + return this; + } + /** * Adds a new shard routing (makes a copy of it), with reference data used from the index shard routing table * if it needs to be created. @@ -658,8 +686,20 @@ public Builder addShard(ShardRouting shard) { return this; } + public Builder addChildReplica(ShardRouting childReplica) { + assert childReplica.started() && childReplica.getParentShardId() != null && childReplica.primary() == false; + IndexShardRoutingTable childReplicaShard = childReplicas.get(childReplica.shardId().id()); + if (childReplicaShard == null) { + childReplicaShard = new IndexShardRoutingTable.Builder(childReplica.shardId()).addShard(childReplica).build(); + } else { + childReplicaShard = new IndexShardRoutingTable.Builder(childReplicaShard).addShard(childReplica).build(); + } + childReplicas.put(childReplica.id(), childReplicaShard); + return this; + } + public IndexRoutingTable build() { - return new IndexRoutingTable(index, shards); + return new IndexRoutingTable(index, shards, childReplicas); } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java index 5bb729af0c2be..6009868aab453 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java @@ -147,7 +147,7 @@ public class IndexShardRoutingTable implements Iterable { allAllocationIds.add(targetChild.allocationId().getId()); assert shard.assignedToNode() : "relocating from unassigned " + shard; - assert targetChild.assignedToNode() : "relocating to unassigned " + shard.getTargetRelocatingShard(); + assert targetChild.assignedToNode() : "Child shard not assigned "; assignedShards.add(targetChild); } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingChangesObserver.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingChangesObserver.java index 2ab52b9bf5040..3b3ad9c5c4b1a 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingChangesObserver.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingChangesObserver.java @@ -54,6 +54,16 @@ public interface RoutingChangesObserver { */ void shardStarted(ShardRouting initializingShard, ShardRouting startedShard); + /** + * Called when a child replica is started. + */ + void childReplicaStarted(ShardRouting initializingShard, ShardRouting parentShard, ShardRouting childShard); + + /** + * Called when a child shard fails. + */ + void childShardFailed(ShardRouting parentShard, ShardRouting childShard); + /** * Called when relocation of a started shard is initiated. */ @@ -127,6 +137,16 @@ public void shardStarted(ShardRouting initializingShard, ShardRouting startedSha } + @Override + public void childReplicaStarted(ShardRouting initializingShard, ShardRouting parentShard, ShardRouting childReplica) { + + } + + @Override + public void childShardFailed(ShardRouting parentShard, ShardRouting childShard) { + + } + @Override public void relocationStarted(ShardRouting startedShard, ShardRouting targetRelocatingShard) { @@ -263,6 +283,20 @@ public void relocationSourceRemoved(ShardRouting removedReplicaRelocationSource) } } + @Override + public void childReplicaStarted(ShardRouting initializingShard, ShardRouting parentShard, ShardRouting childReplica) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.childReplicaStarted(initializingShard, parentShard, childReplica); + } + } + + @Override + public void childShardFailed(ShardRouting parentShard, ShardRouting childShard) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.childShardFailed(parentShard, childShard); + } + } + @Override public void splitCompleted(ShardRouting removedSplitSource, IndexMetadata indexMetadata) { for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java index 6fe163a51e32b..6641cf681bf5e 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java @@ -51,7 +51,6 @@ import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -94,6 +93,8 @@ public class RoutingNodes implements Iterable { private final Map> assignedShards = new HashMap<>(); + private final List splittingShards = new ArrayList<>(); + private final boolean readOnly; private int inactivePrimaryCount = 0; @@ -101,7 +102,7 @@ public class RoutingNodes implements Iterable { private int inactiveShardCount = 0; private int relocatingShards = 0; - private int splittingShards = 0; + private int splittingShardsCount = 0; private final Map> nodesPerAttributeNames; private final Map recoveriesPerNode = new HashMap<>(); @@ -154,10 +155,24 @@ public RoutingNodes(ClusterState clusterState, boolean readOnly) { routingNode.add(targetShardRouting); assignedShardsAdd(targetShardRouting); } else if (shard.splitting()) { - splittingShards++; + splittingShardsCount++; for (ShardRouting childShard : shard.getRecoveringChildShards()) { - addInitialRecovery(childShard, indexShard.primary); - routingNode.add(childShard); + // Replication source is parent primary in case of both child shards and their replicas. + if (childShard.started() == false) { + addInitialRecovery(childShard, indexShard.primary); + } else { + // Only replicas can start first if a portion of recovering child shards is seen started. + assert childShard.primary() == false; + } + if (childShard.primary()) { + routingNode.add(childShard); + } else { + RoutingNode replicaRoutingNode = nodesToShards.computeIfAbsent( + childShard.currentNodeId(), + k -> new RoutingNode(childShard.currentNodeId(), clusterState.nodes().get(childShard.currentNodeId())) + ); + replicaRoutingNode.add(childShard); + } assignedShardsAdd(childShard); } updateSplitSourceOutgoingRecovery(shard, true); @@ -167,6 +182,13 @@ public RoutingNodes(ClusterState clusterState, boolean readOnly) { } inactiveShardCount++; addInitialRecovery(shard, indexShard.primary); + } else if (shard.started() && shard.primary()) { + IndexMetadata indexMetadata = metadata.getIndexSafe(indexRoutingTable.getIndex()); + if (indexMetadata.getSplitShardsMetadata().isSplitOfShardInProgress(shard.id())) { + ShardRange[] childShardRanges = indexMetadata.getSplitShardsMetadata().getChildShardsOfParent(shard.shardId().id()); + ShardRouting parentRouting = shard.createRecoveringChildShards(childShardRanges, indexMetadata.getNumberOfReplicas()); + splittingShards.add(parentRouting); + } } } else { unassignedShards.add(shard); @@ -199,7 +221,7 @@ private void updateRecoveryCounts(final ShardRouting routing, final boolean incr // TODO: check primary == null || primary.active() after all tests properly add ReplicaAfterPrimaryActiveAllocationDecider assert primary == null || primary.assignedToNode() : "shard is initializing but its primary is not assigned to a node"; - // Primary shard routing, excluding the relocating/split target primaries. + // Primary shard routing, excluding the relocating/splitting primaries in the following if condition. if (routing.primary() && (primary == null || (primary == routing))) { assert routing.relocatingNodeId() == null : "Routing must be a non relocating primary"; Recoveries.getOrAdd(initialPrimaryRecoveries, routing.currentNodeId()).addIncoming(howMany); @@ -208,7 +230,7 @@ private void updateRecoveryCounts(final ShardRouting routing, final boolean incr Recoveries.getOrAdd(getRecoveries(routing), routing.currentNodeId()).addIncoming(howMany); - if (routing.recoverySource().getType() == RecoverySource.Type.PEER) { + if (routing.recoverySource().getType() == RecoverySource.Type.PEER && !routing.isSplitTarget()) { // add/remove corresponding outgoing recovery on node with primary shard if (primary == null) { throw new IllegalStateException("shard is peer recovering but primary is unassigned"); @@ -299,7 +321,7 @@ private ShardRouting findAssignedPrimaryIfPeerRecoveryOrSplit(ShardRouting routi return shardRouting; } else if (primary == null) { primary = shardRouting; - } else if (primary.getRecoveringChildShardRanges() != null) { + } else if (primary.getRecoveringChildShards() != null) { primary = shardRouting; } } @@ -323,6 +345,10 @@ public UnassignedShards unassigned() { return this.unassignedShards; } + public List splitting() { + return this.splittingShards; + } + public RoutingNode node(String nodeId) { return nodesToShards.get(nodeId); } @@ -369,7 +395,7 @@ public int getRelocatingShardCount() { } public int getSplittingShardCount() { - return splittingShards; + return splittingShardsCount; } /** @@ -585,39 +611,34 @@ public Tuple relocateShard( } /** - * Relocate a shard to another node, adding the target initializing - * shard as well as assigning it. - * - * @return pair of source relocating and target initializing shards. + * Update assigned child shards against the current splitting parent and their respective recoveries. */ - public Tuple> splitShard( + public void assignChildShards( ShardRouting startedShard, - IndexMetadata indexMetadata, - long expectedShardSize, - RoutingChangesObserver changes + ShardRouting parentShard, + RoutingChangesObserver changes, + Map assignedRoutingNodes ) { ensureMutable(); - splittingShards++; - ShardRange[] childShardRanges = indexMetadata.getSplitShardsMetadata().getChildShardsOfParent(startedShard.shardId().id()); - ShardRouting source = startedShard.split(childShardRanges, expectedShardSize); - updateAssigned(startedShard, source); - ShardRouting[] childShards = source.getRecoveringChildShards(); - List childShardsList = Arrays.asList(childShards); - for (ShardRouting childShard : childShards) { - node(source.currentNodeId()).add(childShard); - assignedShardsAdd(childShard); - addRecovery(childShard); + assert parentShard.splitting(); + splittingShardsCount++; + List assignedChildShards = parentShard.assignChildShards(assignedRoutingNodes); + for (ShardRouting assignedChildShard : assignedChildShards) { + node(assignedChildShard.currentNodeId()).add(assignedChildShard); + assignedShardsAdd(assignedChildShard); + addRecovery(assignedChildShard); } - updateSplitSourceOutgoingRecovery(source, true); - changes.splitStarted(startedShard, childShardsList); - return Tuple.tuple(source, childShardsList); + updateSplitSourceOutgoingRecovery(parentShard, true); + updateAssigned(startedShard, parentShard); + changes.splitStarted(startedShard, assignedChildShards); } public void startInPlaceChildShards( Logger logger, List childShards, IndexMetadata indexMetadata, - RoutingChangesObserver routingChangesObserver + RoutingChangesObserver routingChangesObserver, + RoutingTable routingTable ) { ensureMutable(); assert !childShards.isEmpty(); @@ -627,20 +648,18 @@ public void startInPlaceChildShards( ); int validShardEvents = 0, invalidShardEvents = 0; for (ShardRouting childShard : childShards) { - if (childShard.isSplitTargetOf(parentShard) == false) { + if (childShard.isSplitTargetOf(parentShard) == false || + (childShard.isStartedChildReplica() == false && childShard.initializing() == false)) { invalidShardEvents++; } else { validShardEvents++; } } - if (invalidShardEvents != 0 || validShardEvents != parentShard.getRecoveringChildShards().length) { + if (invalidShardEvents != 0) { logger.error( - "Invalid shard started event for child shards received. Unknown child found :[" - + (invalidShardEvents != 0) - + "], Number of missing child shards in started shard event: [" - + (parentShard.getRecoveringChildShards().length - validShardEvents) - + "], Parent shard is valid: [" + "Invalid shard started event for child shards received. Unknown child found." + + ", Parent shard is valid: [" + (indexMetadata.getSplitShardsMetadata().isSplitOfShardInProgress(parentShard.shardId().id()) == true) + "]. Failing all child shards and cancelling split." ); @@ -651,13 +670,46 @@ public void startInPlaceChildShards( return; } - childShards.forEach(childShard -> { - ShardRouting startedShard = started(childShard); - logger.trace("{} marked shard as started (routing: {})", childShard.shardId(), childShard); - routingChangesObserver.shardStarted(childShard, startedShard); - }); - remove(parentShard); - routingChangesObserver.splitCompleted(parentShard, indexMetadata); + int startedChildPrimaries = 0, startedChildReplicas = 0; + ShardRouting updatedParent = parentShard; + for (ShardRouting childShard : childShards) { + if (childShard.primary() == false) { + if (childShard.initializing() == true) { + ShardRouting startedChild = started(childShard); + routingChangesObserver.childReplicaStarted(childShard, parentShard, startedChild); + updatedParent = updatedParent.updatedStartedReplicaOnParent(childShard, startedChild); + startedChildReplicas++; + } + } else { + assert childShard.initializing(); + ShardRouting startedShard = started(childShard); + routingChangesObserver.shardStarted(childShard, startedShard); + startedChildPrimaries++; + } + } + updateAssigned(parentShard, updatedParent); + + if (startedChildPrimaries == 0) { + assert startedChildReplicas > 0; + } else { + assert startedChildPrimaries == indexMetadata.getSplitShardsMetadata(). + getChildShardsOfParent(parentShard.shardId().id()).length; + assert startedChildReplicas == 0; + for (ShardRouting childShard : parentShard.getRecoveringChildShards()) { + ShardRouting assignedChild = getByAllocationId(childShard.shardId(), childShard.allocationId().getId()); + if (assignedChild.isSplitTarget()) { + assert assignedChild.primary() == false; + removeParentFromStartedChild(assignedChild); + assignedChild = getByAllocationId(childShard.shardId(), childShard.allocationId().getId()); + } + assert assignedChild.started() && assignedChild.isSplitTarget() == false; + } + IndexShardRoutingTable shardRoutingTable = routingTable.shardRoutingTable(parentShard.shardId()); + for (ShardRouting parent : shardRoutingTable.getShards()) { + remove(parent); + } + routingChangesObserver.splitCompleted(parentShard, indexMetadata); + } } /** @@ -743,7 +795,7 @@ public void failShard( RoutingChangesObserver routingChangesObserver ) { ensureMutable(); - if (failedShard.isSplitTarget() && getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId()) == null) { + if (failedShard.getParentShardId() != null && getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId()) == null) { // We already removed this child when parent failed. return; } @@ -758,7 +810,7 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId logger.debug("{} failing shard {} with unassigned info ({})", failedShard.shardId(), failedShard, unassignedInfo.shortSummary()); // if this is a primary, fail initializing replicas first (otherwise we move RoutingNodes into an inconsistent state) - if (failedShard.primary()) { + if (failedShard.primary() && failedShard.getParentShardId() == null) { List assignedShards = assignedShards(failedShard.shardId()); if (assignedShards.isEmpty() == false) { // copy list to prevent ConcurrentModificationException @@ -802,7 +854,7 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId } } else if (failedShard.splitting()) { for (ShardRouting childShard : failedShard.getRecoveringChildShards()) { - assert childShard.isSplitTargetOf(failedShard); + assert childShard.getParentShardId() != null; logger.trace("{} is removed due to the failure/cancellation of the source shard", childShard); remove(childShard); } @@ -810,6 +862,21 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId } // fail actual shard + if (failedShard.getParentShardId() != null) { + ShardRouting parentShard = getByAllocationId( + failedShard.getParentShardId(), + failedShard.allocationId().getParentAllocationId() + ); + + if (parentShard.splitting() == true) { + childShardFailed(logger, failedShard, unassignedInfo, indexMetadata, routingChangesObserver, parentShard); + assert node(failedShard.currentNodeId()).getByShardId(failedShard.shardId()) == null : "failedShard " + + failedShard + + " was matched but wasn't removed"; + } + return; + } + if (failedShard.initializing()) { AllocationId failedShardAllocId = failedShard.allocationId(); if (failedShard.relocatingNodeId() == null && failedShardAllocId.getParentAllocationId() == null) { @@ -820,27 +887,6 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId // initializing shard that is not relocation target, just move to unassigned moveToUnassigned(failedShard, unassignedInfo); } - } else if (failedShardAllocId.getParentAllocationId() != null) { - ShardRouting sourceShard = getByAllocationId( - failedShard.getParentShardId(), - failedShard.allocationId().getParentAllocationId() - ); - // If source shard is not splitting then we must have failed it in previous iteration of child shard. - if (sourceShard.splitting() == true) { - assert sourceShard.isSplitSourceOf(failedShard); - logger.trace( - "{}, resolved source to [{}]. canceling split ... ({})", - failedShard.shardId(), - sourceShard, - unassignedInfo.shortSummary() - ); - cancelSplit(sourceShard); - - for (ShardRouting childShard : sourceShard.getRecoveringChildShards()) { - remove(childShard); - } - routingChangesObserver.splitFailed(sourceShard, indexMetadata); - } } else { // The shard is a target or child of a relocating shard. In that case we only need to remove the target/child shard(s) and // cancel the source @@ -880,6 +926,30 @@ assert node(failedShard.currentNodeId()).getByShardId(failedShard.shardId()) == + " was matched but wasn't removed"; } + private void childShardFailed( + Logger logger, + ShardRouting failedShard, + UnassignedInfo unassignedInfo, + IndexMetadata indexMetadata, + RoutingChangesObserver routingChangesObserver, + ShardRouting parentShard + ) { + + assert parentShard.isSplitSourceOf(failedShard); + logger.trace( + "{}, resolved source to [{}]. canceling split ... ({})", + failedShard.shardId(), + parentShard, + unassignedInfo.shortSummary() + ); + cancelSplit(parentShard); + + for (ShardRouting childShard : parentShard.getRecoveringChildShards()) { + remove(childShard); + } + routingChangesObserver.splitFailed(parentShard, indexMetadata); + } + private void unassignPrimaryAndPromoteActiveReplicaIfExists( ShardRouting failedShard, UnassignedInfo unassignedInfo, @@ -923,11 +993,24 @@ private ShardRouting started(ShardRouting shard) { } } removeRecovery(shard); - ShardRouting startedShard = shard.moveToStarted(); + ShardRouting startedShard; + if (shard.isSplitTarget() && shard.primary() == false) { + startedShard = shard.moveChildReplicaToStarted(); + } else { + startedShard = shard.moveToStarted(); + } updateAssigned(shard, startedShard); return startedShard; } + + private ShardRouting removeParentFromStartedChild(ShardRouting shard) { + assert shard.started(); + ShardRouting parentRemoved = shard.removeParentFromReplica(); + updateAssigned(shard, parentRemoved); + return parentRemoved; + } + /** * Cancels a relocation of a shard that shard must relocating. * @@ -946,7 +1029,10 @@ private ShardRouting cancelRelocation(ShardRouting shard) { * @return the shard after cancelling relocation */ private ShardRouting cancelSplit(ShardRouting shard) { - splittingShards--; + splittingShardsCount--; + for (ShardRouting splittingShard : splittingShards) { + assert splittingShard.shardId().equals(shard.shardId()) == false; + } ShardRouting cancelledShard = shard.cancelSplit(); updateAssigned(shard, cancelledShard); updateSplitSourceOutgoingRecovery(shard, false); @@ -1484,7 +1570,6 @@ private static void assertRecoveriesPerNode( if (assigned.initializing() && assigned.recoverySource().getType() == RecoverySource.Type.PEER) { outgoing++; } else if (assigned.splitting()) { - assert assigned.getRecoveringChildShards().length == assigned.getRecoveringChildShardRanges().length; for (ShardRouting childShardRouting : assigned.getRecoveringChildShards()) { assert routingNodes.assignedShards.containsKey(childShardRouting.shardId()); for (ShardRouting assignedChildShard : routingNodes.assignedShards.get(childShardRouting.shardId())) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java index 2fe787b678a1c..acce1e29974c0 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java @@ -171,6 +171,24 @@ public IndexShardRoutingTable shardRoutingTable(ShardId shardId) { return shard; } + /** + * All child replica shards for the provided {@link ShardId} + * @return All the child replica shard routing entries for the given index and shard id + * @throws IndexNotFoundException if provided index does not exist + * @throws ShardNotFoundException if provided shard id is unknown + */ + public IndexShardRoutingTable childReplicaShardRoutingTable(ShardId shardId) { + IndexRoutingTable indexRouting = index(shardId.getIndexName()); + if (indexRouting == null || indexRouting.getIndex().equals(shardId.getIndex()) == false) { + throw new IndexNotFoundException(shardId.getIndex()); + } + IndexShardRoutingTable shard = indexRouting.childReplicaShard(shardId.id()); + if (shard == null) { + throw new ShardNotFoundException(shardId); + } + return shard; + } + @Nullable public ShardRouting getByAllocationId(ShardId shardId, String allocationId) { final IndexRoutingTable indexRoutingTable = index(shardId.getIndexName()); @@ -491,7 +509,11 @@ private static void addShard( indexBuilder = new IndexRoutingTable.Builder(index); indexRoutingTableBuilders.put(index.getName(), indexBuilder); } - indexBuilder.addShard(shardRoutingEntry); + if (shardRoutingEntry.isStartedChildReplica()) { + indexBuilder.addChildReplica(shardRoutingEntry); + } else { + indexBuilder.addShard(shardRoutingEntry); + } } /** diff --git a/server/src/main/java/org/opensearch/cluster/routing/ShardRouting.java b/server/src/main/java/org/opensearch/cluster/routing/ShardRouting.java index ddab0895bb680..e49dd06c617b1 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/ShardRouting.java +++ b/server/src/main/java/org/opensearch/cluster/routing/ShardRouting.java @@ -48,9 +48,12 @@ import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.Objects; /** * {@link ShardRouting} immutably encapsulates information about shard @@ -79,12 +82,9 @@ public class ShardRouting implements Writeable, ToXContentObject { @Nullable private final ShardRouting targetRelocatingShard; @Nullable - private final ShardRange[] recoveringChildShardRanges; - @Nullable private final ShardRouting[] recoveringChildShards; @Nullable private final ShardId parentShardId; - private final ShardRange shardRange; /** * A constructor to internally create shard routing instances, note, the internal flag should only be set to true @@ -100,9 +100,8 @@ protected ShardRouting( UnassignedInfo unassignedInfo, AllocationId allocationId, long expectedShardSize, - ShardRange shardRange, - ShardId parentShardId, - ShardRange[] childShardRanges + ShardRouting[] recoveringChildShards, + ShardId parentShardId ) { this.shardId = shardId; this.currentNodeId = currentNodeId; @@ -114,10 +113,8 @@ protected ShardRouting( this.allocationId = allocationId; this.expectedShardSize = expectedShardSize; this.targetRelocatingShard = initializeTargetRelocatingShard(); - this.recoveringChildShardRanges = childShardRanges; - this.recoveringChildShards = initializeRecoveringChildShards(); + this.recoveringChildShards = recoveringChildShards; this.parentShardId = parentShardId; - this.shardRange = shardRange; this.asList = Collections.singletonList(this); assert expectedShardSize == UNAVAILABLE_EXPECTED_SHARD_SIZE || state == ShardRoutingState.INITIALIZING @@ -130,8 +127,9 @@ protected ShardRouting( assert !(state == ShardRoutingState.UNASSIGNED && unassignedInfo == null) : "unassigned shard must be created with meta"; assert (state == ShardRoutingState.UNASSIGNED || state == ShardRoutingState.INITIALIZING) == (recoverySource != null) : "recovery source only available on unassigned or initializing shard but was " + state; - assert recoverySource == null || recoverySource == PeerRecoverySource.INSTANCE || primary - : "replica shards always recover from primary"; + assert recoverySource == null || recoverySource == PeerRecoverySource.INSTANCE || + recoverySource == RecoverySource.InPlaceShardSplitRecoverySource.INSTANCE || primary + : "replica shards always recover from primary or child shards"; assert (currentNodeId == null) == (state == ShardRoutingState.UNASSIGNED) : "unassigned shard must not be assigned to a node " + this; } @@ -149,7 +147,6 @@ private ShardRouting initializeTargetRelocatingShard() { unassignedInfo, AllocationId.newTargetRelocation(allocationId), expectedShardSize, - shardRange, null, null ); @@ -158,31 +155,57 @@ private ShardRouting initializeTargetRelocatingShard() { } } - @Nullable - private ShardRouting[] initializeRecoveringChildShards() { - if (state == ShardRoutingState.SPLITTING) { - List childShardAllocIds = allocationId.getSplitChildAllocationIds(); - ShardRouting[] childShards = new ShardRouting[childShardAllocIds.size()]; - for (int idx = 0; idx < childShardAllocIds.size(); idx++) { - childShards[idx] = new ShardRouting( - new ShardId(shardId.getIndex(), recoveringChildShardRanges[idx].getShardId()), - currentNodeId, - null, - primary, - ShardRoutingState.INITIALIZING, - RecoverySource.InPlaceShardSplitRecoverySource.INSTANCE, - unassignedInfo, - AllocationId.newTargetSplit(allocationId, childShardAllocIds.get(idx)), - expectedShardSize, - recoveringChildShardRanges[idx], - shardId, - null - ); + public List assignChildShards(Map assignedRoutingNodes) { + List assignedChildShards = new ArrayList<>(); + int idx = 0; + for (ShardRouting childShard : assignedRoutingNodes.keySet()) { + ShardRouting assignedChildShard = new ShardRouting( + childShard.shardId, + assignedRoutingNodes.get(childShard), + null, + childShard.primary, + ShardRoutingState.INITIALIZING, + childShard.recoverySource, + unassignedInfo, + childShard.allocationId, + expectedShardSize, + null, + shardId + ); + assignedChildShards.add(assignedChildShard); + recoveringChildShards[idx++] = assignedChildShard; + } + + return assignedChildShards; + } + + public ShardRouting updatedStartedReplicaOnParent(ShardRouting initializingReplica, ShardRouting startedReplica) { + List updatedChildShards = new ArrayList<>(); + boolean moved = false; + for (ShardRouting childShard : recoveringChildShards) { + if (childShard.equals(initializingReplica)) { + assert startedReplica.primary == false; + moved = true; + updatedChildShards.add(startedReplica); + } else { + updatedChildShards.add(childShard); } - return childShards; - } else { - return null; } + + assert moved; + return new ShardRouting( + shardId, + currentNodeId, + relocatingNodeId, + primary, + state, + recoverySource, + unassignedInfo, + allocationId, + expectedShardSize, + updatedChildShards.toArray(new ShardRouting[0]), + parentShardId + ); } /** @@ -205,7 +228,6 @@ public static ShardRouting newUnassigned( null, UNAVAILABLE_EXPECTED_SHARD_SIZE, null, - null, null ); } @@ -387,15 +409,12 @@ public ShardRouting(ShardId shardId, StreamInput in) throws IOException { expectedShardSize = shardSize; asList = Collections.singletonList(this); targetRelocatingShard = initializeTargetRelocatingShard(); - parentShardId = null; if (in.getVersion().onOrAfter(Version.V_3_0_0)) { - shardRange = in.readOptionalWriteable(ShardRange::new); - recoveringChildShardRanges = in.readOptionalArray(ShardRange::new, ShardRange[]::new); - recoveringChildShards = initializeRecoveringChildShards(); + recoveringChildShards = in.readOptionalArray(ShardRouting::new, ShardRouting[]::new); + parentShardId = in.readOptionalWriteable(ShardId::new); } else { recoveringChildShards = null; - recoveringChildShardRanges = null; - shardRange = null; + parentShardId = null; } } @@ -427,10 +446,10 @@ public void writeToThin(StreamOutput out) throws IOException { out.writeLong(expectedShardSize); } if (out.getVersion().onOrAfter(Version.V_3_0_0)) { - out.writeOptionalWriteable(shardRange); - out.writeOptionalArray(recoveringChildShardRanges); + out.writeOptionalArray(recoveringChildShards); + out.writeOptionalWriteable(parentShardId); } else { - if (recoveringChildShardRanges != null) { + if (recoveringChildShards != null || parentShardId != null) { // In-progress shard split is not allowed in a mixed cluster where node(s) with an unsupported split // version is present. Hence, we also don't want to allow a node with an unsupported version // to get this state while shard split is in-progress. @@ -458,9 +477,8 @@ public ShardRouting updateUnassigned(UnassignedInfo unassignedInfo, RecoverySour unassignedInfo, allocationId, expectedShardSize, - shardRange, null, - null + parentShardId ); } @@ -489,9 +507,8 @@ public ShardRouting moveToUnassigned(UnassignedInfo unassignedInfo) { unassignedInfo, null, UNAVAILABLE_EXPECTED_SHARD_SIZE, - shardRange, null, - null + parentShardId ); } @@ -519,9 +536,8 @@ public ShardRouting initialize(String nodeId, @Nullable String existingAllocatio unassignedInfo, allocationId, expectedShardSize, - shardRange, null, - null + parentShardId ); } @@ -542,18 +558,54 @@ public ShardRouting relocate(String relocatingNodeId, long expectedShardSize) { null, AllocationId.newRelocation(allocationId), expectedShardSize, - shardRange, null, - null + parentShardId ); } /** - * Split the shard. - * + * Create child shard routings for a splitting parent. */ - public ShardRouting split(ShardRange[] shardRanges, long expectedShardSize) { - assert state == ShardRoutingState.STARTED : "current shard has to be started in order to be split " + this; + public ShardRouting createRecoveringChildShards(ShardRange[] recoveringChildShardRanges, int replicaCount) { + int totalShards = recoveringChildShardRanges.length * (replicaCount + 1); + AllocationId allocationId = AllocationId.newSplit(allocationId(), totalShards); + ShardRouting[] childShards = new ShardRouting[totalShards]; + int allocationIdx = 0; + UnassignedInfo childUnassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.CHILD_SHARD_CREATED, + "child_shard_allocation_pending[parent shard " + shardId + "]"); + for (ShardRange recoveringChildShardRange : recoveringChildShardRanges) { + childShards[allocationIdx] = new ShardRouting( + new ShardId(index(), recoveringChildShardRange.getShardId()), + null, + null, + true, + ShardRoutingState.UNASSIGNED, + RecoverySource.InPlaceShardSplitRecoverySource.INSTANCE, + childUnassignedInfo, + AllocationId.newTargetSplit(allocationId, allocationId.getSplitChildAllocationIds().get(allocationIdx)), + UNAVAILABLE_EXPECTED_SHARD_SIZE, + null, + shardId + ); + allocationIdx++; + + for (int replica = 0; replica < replicaCount; replica++) { + childShards[allocationIdx] = new ShardRouting( + new ShardId(index(), recoveringChildShardRange.getShardId()), + null, + null, + false, + ShardRoutingState.UNASSIGNED, + PeerRecoverySource.INSTANCE, + childUnassignedInfo, + AllocationId.newTargetSplit(allocationId, allocationId.getSplitChildAllocationIds().get(allocationIdx)), + UNAVAILABLE_EXPECTED_SHARD_SIZE, + null, + shardId + ); + allocationIdx++; + } + } return new ShardRouting( shardId, @@ -563,11 +615,10 @@ public ShardRouting split(ShardRange[] shardRanges, long expectedShardSize) { ShardRoutingState.SPLITTING, recoverySource, null, - AllocationId.newSplit(allocationId, shardRanges.length), + allocationId, expectedShardSize, - shardRange, - null, - shardRanges + childShards, + null ); } @@ -589,9 +640,8 @@ public ShardRouting cancelRelocation() { null, AllocationId.cancelRelocation(allocationId), UNAVAILABLE_EXPECTED_SHARD_SIZE, - shardRange, null, - null + parentShardId ); } @@ -613,7 +663,6 @@ public ShardRouting cancelSplit() { null, AllocationId.cancelSplit(allocationId), UNAVAILABLE_EXPECTED_SHARD_SIZE, - shardRange, null, null ); @@ -639,9 +688,8 @@ public ShardRouting removeRelocationSource() { unassignedInfo, AllocationId.finishRelocation(allocationId), expectedShardSize, - shardRange, null, - null + parentShardId ); } @@ -662,9 +710,26 @@ public ShardRouting reinitializeReplicaShard() { unassignedInfo, AllocationId.newInitializing(), expectedShardSize, - shardRange, null, - null + parentShardId + ); + } + + public ShardRouting moveChildReplicaToStarted() { + // Note that we don't detach parent info yet. This will be done when primary child shards are started so that + // started child replicas continue to act as child shards. + return new ShardRouting( + shardId, + currentNodeId, + null, + primary, + ShardRoutingState.STARTED, + null, + null, + allocationId, + UNAVAILABLE_EXPECTED_SHARD_SIZE, + null, + parentShardId ); } @@ -693,7 +758,26 @@ public ShardRouting moveToStarted() { null, allocationId, UNAVAILABLE_EXPECTED_SHARD_SIZE, - shardRange, + null, + null + ); + } + + /** + * Removes Parent info from a started replica + */ + public ShardRouting removeParentFromReplica() { + assert isStartedChildReplica(); + return new ShardRouting( + shardId, + currentNodeId, + null, + primary, + ShardRoutingState.STARTED, + null, + null, + allocationId, + UNAVAILABLE_EXPECTED_SHARD_SIZE, null, null ); @@ -719,9 +803,8 @@ public ShardRouting moveActivePrimaryToReplica() { unassignedInfo, allocationId, expectedShardSize, - shardRange, - parentShardId, - recoveringChildShardRanges + recoveringChildShards, + parentShardId ); } @@ -745,9 +828,8 @@ public ShardRouting moveActiveReplicaToPrimary() { unassignedInfo, allocationId, expectedShardSize, - shardRange, - parentShardId, - recoveringChildShardRanges + recoveringChildShards, + parentShardId ); } @@ -771,9 +853,8 @@ public ShardRouting moveUnassignedFromPrimary() { unassignedInfo, allocationId, expectedShardSize, - shardRange, - parentShardId, - recoveringChildShardRanges + recoveringChildShards, + parentShardId ); } @@ -800,10 +881,10 @@ public boolean isRelocationTarget() { /** * Returns true if this shard is a split target for another shard - * (i.e., was created with {@link #initializeRecoveringChildShards()} ()} + * (i.e., was created in {@link #assignChildShards(Map)} ()} ()} */ public boolean isSplitTarget() { - return state == ShardRoutingState.INITIALIZING && getParentShardId() != null; + return getParentShardId() != null; } /** @@ -813,7 +894,6 @@ public boolean isSplitTargetOf(ShardRouting other) { boolean b = this.allocationId != null && other.allocationId != null - && this.state == ShardRoutingState.INITIALIZING && this.allocationId.getParentAllocationId() != null && this.allocationId.getParentAllocationId().equals(other.allocationId.getId()); @@ -838,7 +918,7 @@ public boolean isSplitTargetOf(ShardRouting other) { + other + "]"; - assert b == false || this.primary == other.primary : "ShardRouting is a splitting target but primary flag is different." + assert b == false || other.primary : "ShardRouting is a splitting target but primary flag is different." + " This [" + this + "], target [" @@ -907,9 +987,6 @@ public boolean isSplitSourceOf(ShardRouting other) { && other.allocationId.getParentAllocationId() != null && this.allocationId.getId().equals(other.allocationId.getParentAllocationId()); - assert b == false || other.state == ShardRoutingState.INITIALIZING - : "ShardRouting is a split source but the target shard state isn't initializing. This [" + this + "], other [" + other + "]"; - assert b == false || this.allocationId.getSplitChildAllocationIds() != null && this.allocationId.getSplitChildAllocationIds().contains(other.allocationId.getId()) @@ -928,7 +1005,7 @@ public boolean isSplitSourceOf(ShardRouting other) { + other + "]"; - assert b == false || this.primary == other.primary : "ShardRouting is a splitting source but primary flag is different." + assert b == false || this.primary : "ShardRouting is a splitting source but primary flag is different." + " This [" + this + "], target [" @@ -1012,7 +1089,10 @@ public boolean equalsIgnoringMetadata(ShardRouting other) { if (recoverySource != null ? !recoverySource.equals(other.recoverySource) : other.recoverySource != null) { return false; } - if (Arrays.equals(recoveringChildShardRanges, other.recoveringChildShardRanges) == false) { + if (Arrays.equals(recoveringChildShards, other.recoveringChildShards) == false) { + return false; + } + if (Objects.equals(parentShardId, other.parentShardId) == false) { return false; } return true; @@ -1051,7 +1131,8 @@ public int hashCode() { h = 31 * h + (recoverySource != null ? recoverySource.hashCode() : 0); h = 31 * h + (allocationId != null ? allocationId.hashCode() : 0); h = 31 * h + (unassignedInfo != null ? unassignedInfo.hashCode() : 0); - h = 31 * h + (recoveringChildShardRanges != null ? Arrays.hashCode(recoveringChildShardRanges) : 0); + h = 31 * h + (recoveringChildShards != null ? Arrays.hashCode(recoveringChildShards) : 0); + h = 31 * h + (parentShardId != null ? parentShardId.hashCode() : 0); hashCode = h; } return h; @@ -1090,12 +1171,11 @@ public String shortSummary() { if (expectedShardSize != UNAVAILABLE_EXPECTED_SHARD_SIZE) { sb.append(", expected_shard_size[").append(expectedShardSize).append("]"); } - if (recoveringChildShardRanges != null) { - sb.append(", recovering_child_shards ["); - for (ShardRange childShard : recoveringChildShardRanges) { - sb.append(childShard.toString()); - } - sb.append("]"); + if (recoveringChildShards != null) { + sb.append(", recovering_child_shards [").append(Arrays.toString(recoveringChildShards)).append("]"); + } + if (parentShardId != null) { + sb.append(", parent_shard_id [").append(parentShardId).append("]"); } return sb.toString(); } @@ -1122,13 +1202,18 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (unassignedInfo != null) { unassignedInfo.toXContent(builder, params); } - if (recoveringChildShardRanges != null) { - builder.startArray("recovering_child_shard_ranges"); - for (ShardRange childShardId : recoveringChildShardRanges) { - childShardId.toXContent(builder, params); + if (recoveringChildShards != null) { + builder.startArray("recovering_child_shards"); + for (ShardRouting childShard : recoveringChildShards) { + childShard.toXContent(builder, params); } builder.endArray(); } + if (parentShardId != null) { + builder.field("parent_shard_id"); + parentShardId.toXContent(builder, params); + } + return builder.endObject(); } @@ -1158,21 +1243,24 @@ public boolean unassignedReasonIndexCreated() { return false; } + public boolean unassignedReasonChildShardCreated() { + if (unassignedInfo != null) { + return unassignedInfo.getReason() == UnassignedInfo.Reason.CHILD_SHARD_CREATED; + } + return false; + } + @Nullable public ShardRouting[] getRecoveringChildShards() { // Not returning copy because in RoutingNodes shards are removed based on their object identity. return recoveringChildShards; } - public ShardRange getShardRange() { - return shardRange; - } - - public ShardRange[] getRecoveringChildShardRanges() { - return recoveringChildShardRanges; - } - public ShardId getParentShardId() { return parentShardId; } + + public boolean isStartedChildReplica() { + return primary == false && getParentShardId() != null && started(); + } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/opensearch/cluster/routing/UnassignedInfo.java index cf6dc9cd7306e..17286909ead81 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/opensearch/cluster/routing/UnassignedInfo.java @@ -150,7 +150,11 @@ public enum Reason { /** * Unassigned as a result of closing an index. */ - INDEX_CLOSED + INDEX_CLOSED, + /** + * Unassigned as allocation of this child shard is pending.. + */ + CHILD_SHARD_CREATED } /** diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 16a400e842984..e69efcd211305 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -606,6 +606,7 @@ private void applyStartedShards(RoutingAllocation routingAllocation, List> inPlaceChildShards = new HashMap<>(); for (ShardRouting startedShard : startedShardEntries) { + logger.info("Started shard cluster state event for shard routing: " + startedShard); assert startedShard.initializing() : "only initializing shards can be started"; assert routingAllocation.metadata().index(startedShard.shardId().getIndex()) != null : "shard started for unknown index (shard entry: " + startedShard + ")"; @@ -625,7 +626,8 @@ private void applyStartedShards(RoutingAllocation routingAllocation, List { IndexMetadata indexMetadata = routingAllocation.metadata().getIndexSafe(childShards.get(0).index()); - routingNodes.startInPlaceChildShards(logger, childShards, indexMetadata, routingAllocation.changes()); + routingNodes.startInPlaceChildShards(logger, childShards, indexMetadata, routingAllocation.changes(), + routingAllocation.routingTable()); }); } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java index c1c00a3448787..d16881e11b415 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java @@ -106,7 +106,7 @@ public void shardStarted(ShardRouting initializingShard, ShardRouting startedSha @Override public void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo) { - if (failedShard.active() && failedShard.primary() && failedShard.isSplitTarget() == false) { + if (failedShard.active() && failedShard.primary() && failedShard.getParentShardId() != null) { Updates updates = changes(failedShard.shardId()); if (updates.firstFailedPrimary == null) { // more than one primary can be failed (because of batching, primary can be failed, replica promoted and then failed...) @@ -132,12 +132,34 @@ public void splitCompleted(ShardRouting removedSplitSource, IndexMetadata indexM removeAllocationId(removedSplitSource); Updates updates = changes(removedSplitSource.shardId()); for (ShardRouting childShard : removedSplitSource.getRecoveringChildShards()) { - updates.addedChildShards.put(childShard.shardId(), childShard); - Updates childUpdates = changes(childShard.shardId()); - childUpdates.isNewChildShard = true; + if (childShard.primary()) { + updates.addedPrimaryChildShards.put(childShard.shardId(), childShard); + Updates childUpdates = changes(childShard.shardId()); + childUpdates.isNewPrimaryChild = true; + } } } + @Override + public void childReplicaStarted(ShardRouting initializingShard, ShardRouting parentShard, ShardRouting childReplica) { + assert childReplica.allocationId().getParentAllocationId().equals(parentShard.allocationId().getId()); + assert childReplica.allocationId().equals(initializingShard.allocationId()); + Updates updates = changes(parentShard.shardId()); + updates.addedAllocationIds.add(childReplica.allocationId().getId()); + Updates childUpdates = changes(childReplica.shardId()); + childUpdates.addedAllocationIds.add(childReplica.allocationId().getId()); + childUpdates.isNewReplicaChild = true; + } + + @Override + public void childShardFailed(ShardRouting parentShard, ShardRouting childShard) { + assert childShard.allocationId().getParentAllocationId().equals(parentShard.allocationId().getId()); + assert childShard.allocationId().equals(childShard.allocationId()); + Updates updates = changes(parentShard.shardId()); + updates.removedAllocationIds.add(childShard.allocationId().getId()); + updates.splitFailed = true; + } + @Override public void splitFailed(ShardRouting splitSource, IndexMetadata indexMetadata) { Updates updates = changes(splitSource.shardId()); @@ -167,13 +189,13 @@ public Metadata applyChanges(Metadata oldMetadata, RoutingTable newRoutingTable) ShardId shardId = shardEntry.getKey(); Updates updates = shardEntry.getValue(); // Avoid updating primary term and in-sync allocations for parent and child shards here. - if (updates.isNewChildShard == false && updates.addedChildShards.isEmpty() == true) { + if (updates.isNewPrimaryChild == false && updates.addedPrimaryChildShards.isEmpty() == true) { indexMetadataBuilder = updateInSyncAllocations(newRoutingTable, oldIndexMetadata, indexMetadataBuilder, shardId, updates); indexMetadataBuilder = updatePrimaryTerm(oldIndexMetadata, indexMetadataBuilder, shardId, updates); } // Invoke metadata update of in-place split only for the parent shard. - if (updates.addedChildShards.isEmpty() == false && updates.splitFailed == false) { - indexMetadataBuilder = updateMetadataForInPlaceSplitCompleted(oldIndexMetadata, indexMetadataBuilder, shardId, updates); + if (updates.addedPrimaryChildShards.isEmpty() == false && updates.splitFailed == false) { + indexMetadataBuilder = updateMetadataForCompletedInPlaceSplit(oldIndexMetadata, indexMetadataBuilder, shardId, updates); } if (updates.splitFailed) { updateMetadataForInPlaceSplitFailed(oldIndexMetadata, indexMetadataBuilder, shardId); @@ -242,6 +264,16 @@ private IndexMetadata.Builder updateInSyncAllocations( // forcing a stale primary resets the in-sync allocations to the singleton set with the stale id indexMetadataBuilder.putInSyncAllocationIds(shardId.id(), Collections.singleton(allocationId)); } + } else if (updates.isNewReplicaChild) { + if (oldInSyncAllocationIds == null) { + oldInSyncAllocationIds = new HashSet<>(); + } + if (indexMetadataBuilder == null) { + indexMetadataBuilder = IndexMetadata.builder(oldIndexMetadata); + } + Set inSyncAllocationIds = new HashSet<>(oldInSyncAllocationIds); + inSyncAllocationIds.addAll(updates.addedAllocationIds); + indexMetadataBuilder.putInSyncAllocationIds(shardId.id(), inSyncAllocationIds); } else { // standard path for updating in-sync ids Set inSyncAllocationIds = new HashSet<>(oldInSyncAllocationIds); @@ -269,6 +301,7 @@ private IndexMetadata.Builder updateInSyncAllocations( List assignedShards = newShardRoutingTable.assignedShards() .stream() .filter(s -> s.isRelocationTarget() == false) + .filter(s -> s.getParentShardId() == null) .collect(Collectors.toList()); assert assignedShards.size() <= maxActiveShards : "cannot have more assigned shards " + assignedShards @@ -382,13 +415,13 @@ private IndexMetadata.Builder updatePrimaryTerm( /** * Adds primary terms of child shards and updates number of shards. */ - private IndexMetadata.Builder updateMetadataForInPlaceSplitCompleted( + private IndexMetadata.Builder updateMetadataForCompletedInPlaceSplit( IndexMetadata oldIndexMetadata, IndexMetadata.Builder indexMetadataBuilder, ShardId parentShardId, Updates updates ) { - if (updates.addedChildShards.isEmpty()) { + if (updates.addedPrimaryChildShards.isEmpty()) { return indexMetadataBuilder; } @@ -396,8 +429,9 @@ private IndexMetadata.Builder updateMetadataForInPlaceSplitCompleted( indexMetadataBuilder = IndexMetadata.builder(oldIndexMetadata); } + // We will only have allocation IDs of primary child shards in this map. Map shardIdToAllocationId = new HashMap<>(); - updates.addedChildShards.forEach((shardId, shardRouting) -> { + updates.addedPrimaryChildShards.forEach((shardId, shardRouting) -> { shardIdToAllocationId.put(shardId.id(), shardRouting.allocationId().getId()); }); @@ -428,7 +462,7 @@ private Updates changes(ShardId shardId) { * Remove allocation id of this shard from the set of in-sync shard copies */ void removeAllocationId(ShardRouting shardRouting) { - if (shardRouting.active()) { + if (shardRouting.active() && shardRouting.getParentShardId() == null) { changes(shardRouting.shardId()).removedAllocationIds.add(shardRouting.allocationId().getId()); } } @@ -444,8 +478,9 @@ private static class Updates { private boolean increaseTerm; // whether primary term should be increased // Child shard ids for this shard which is now split. To be added in in-sync, assign primary term of this shard // and update number of current shards. - private final Map addedChildShards = new HashMap<>(); - private boolean isNewChildShard; + private final Map addedPrimaryChildShards = new HashMap<>(); + private boolean isNewPrimaryChild; // Updated against a new primary child shard. Used to skip child shards in applyChanges. + private boolean isNewReplicaChild; private boolean splitFailed; private Set addedAllocationIds = new HashSet<>(); // allocation ids that should be added to the in-sync set private Set removedAllocationIds = new HashSet<>(); // allocation ids that should be removed from the in-sync set diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingNodesChangedObserver.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingNodesChangedObserver.java index 367d66a550174..100885b41b40a 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingNodesChangedObserver.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingNodesChangedObserver.java @@ -83,6 +83,20 @@ public void splitStarted(ShardRouting startedShard, List childSpli setChanged(); } + @Override + public void childReplicaStarted(ShardRouting initializingShard, ShardRouting parentShard, ShardRouting childReplica) { + assert childReplica.started() : "expected started shard " + childReplica; + setChanged(); + } + + @Override + public void childShardFailed(ShardRouting parentShard, ShardRouting childShard) { + assert childShard.getParentShardId() != null : "expected a child shard " + childShard; + assert childShard.getParentShardId().equals(parentShard.shardId()); + assert childShard.allocationId().getParentAllocationId().equals(parentShard.allocationId().getId()); + setChanged(); + } + @Override public void unassignedInfoUpdated(ShardRouting unassignedShard, UnassignedInfo newUnassignedInfo) { assert unassignedShard.unassigned() : "expected unassigned shard " + unassignedShard; diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index 41ace0e7661fe..02c24624d376c 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -226,6 +226,7 @@ public void allocate(RoutingAllocation allocation) { localShardsBalancer.allocateUnassigned(); localShardsBalancer.moveShards(); localShardsBalancer.balance(); + localShardsBalancer.assignChildShardsOfSplittingShards(); final ShardsBalancer remoteShardsBalancer = new RemoteShardsBalancer(logger, allocation); remoteShardsBalancer.allocateUnassigned(); diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java index 2396a620092dc..a158c0c788e0d 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java @@ -13,6 +13,7 @@ import org.apache.lucene.util.IntroSorter; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.metadata.SplitShardsMetadata; import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.RoutingPool; @@ -612,23 +613,6 @@ void moveShards() { if (targetNode != null) { checkAndAddInEligibleTargetNode(targetNode.getRoutingNode()); } - } else if (moveDecision.isDecisionTaken() && moveDecision.canSplit()) { - final BalancedShardsAllocator.ModelNode sourceNode = nodes.get(shardRouting.currentNodeId()); - sourceNode.removeShard(shardRouting); - IndexMetadata indexMetadata = metadata.getIndexSafe(shardRouting.index()); - Tuple> splittingShards = routingNodes.splitShard( - shardRouting, - indexMetadata, - allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE), - allocation.changes() - ); - splittingShards.v2().forEach(sourceNode::addShard); - if (logger.isTraceEnabled()) { - logger.trace("Splitting shard [{}]", shardRouting); - } - - // Verifying if this node can be considered ineligible for further iterations - checkAndAddInEligibleTargetNode(sourceNode.getRoutingNode()); } else if (moveDecision.isDecisionTaken() && moveDecision.canRemain() == false) { logger.trace("[{}][{}] can't move", shardRouting.index(), shardRouting.id()); } @@ -916,6 +900,119 @@ void allocateUnassigned() { // clear everything we have either added it or moved to ignoreUnassigned } + @Override + public void assignChildShardsOfSplittingShards() { + List splittingShards = routingNodes.splitting(); + if (splittingShards.isEmpty()) { + return; + } + + List splittingShardAssignmentInfos = new ArrayList<>(); + for (ShardRouting splittingShard : splittingShards) { + IndexMetadata indexMetadata = allocation.metadata().getIndexSafe(splittingShard.index()); + SplitShardsMetadata splitShardsMetadata = indexMetadata.getSplitShardsMetadata(); + boolean inProgress = splitShardsMetadata.isSplitOfShardInProgress(splittingShard.shardId().id()) + && allocation.changes().isSplitOfShardFailed(splittingShard) == false; + ShardRouting assignedPrimaryParent = routingNodes.activePrimary(splittingShard.shardId()); + if (inProgress && assignedPrimaryParent != null) { + List primaryChildShards = new ArrayList<>(); + List replicaChildShards = new ArrayList<>(); + + for (ShardRouting childShard : splittingShard.getRecoveringChildShards()) { + if (childShard.primary()) { + primaryChildShards.add(childShard); + } else { + replicaChildShards.add(childShard); + } + } + SplittingShardAssignmentInfo splittingShardAssignmentInfo = new SplittingShardAssignmentInfo( + indexMetadata.getIndexUUID(), assignedPrimaryParent, splittingShard, primaryChildShards, + replicaChildShards); + splittingShardAssignmentInfos.add(splittingShardAssignmentInfo); + } + } + + Collections.shuffle(splittingShardAssignmentInfos); + //TODO: build retry and cancel split to come out of failed shard + + for (SplittingShardAssignmentInfo splittingShardAssignmentInfo : splittingShardAssignmentInfos) { + + RoutingNode parentShardNode = routingNodes.node(splittingShardAssignmentInfo.assignedPrimaryParent.currentNodeId()); + ShardRouting primaryChild; + BalancedShardsAllocator.ModelNode parentModelNode = nodes.get(splittingShardAssignmentInfo.assignedPrimaryParent.currentNodeId()); + Map assignedRoutingNodes = new HashMap<>(); + for (int addIdx = 0; addIdx < splittingShardAssignmentInfo.primaryChildShards.size(); addIdx++) { + primaryChild = splittingShardAssignmentInfo.primaryChildShards.get(addIdx); + Decision currentDecision = allocation.deciders().canAllocate(primaryChild, parentShardNode, allocation); + if (currentDecision.type() == Decision.Type.NO) { + break; + } + parentModelNode.addShard(primaryChild); + assignedRoutingNodes.put(primaryChild, parentModelNode.getNodeId()); + } + + if (assignedRoutingNodes.size() != splittingShardAssignmentInfo.primaryChildShards.size()) { + assignedRoutingNodes.keySet().forEach(routing -> { + BalancedShardsAllocator.ModelNode node = nodes.get(assignedRoutingNodes.get(routing)); + node.removeShard(routing); + }); + continue; + } + + for (ShardRouting replicaChild : splittingShardAssignmentInfo.replicaChildShards) { + final AllocateUnassignedDecision allocationDecision = decideAllocateUnassigned(replicaChild); + if (allocationDecision.getAllocationDecision() != AllocationDecision.YES) { + break; + } + String assignedNodeId = allocationDecision.getTargetNode().getId(); + assignedRoutingNodes.put(replicaChild, nodes.get(assignedNodeId).getNodeId()); + nodes.get(assignedNodeId).addShard(replicaChild); + } + + if (assignedRoutingNodes.size() != splittingShardAssignmentInfo.primaryChildShards.size() + + splittingShardAssignmentInfo.replicaChildShards.size()) { + assignedRoutingNodes.keySet().forEach(routing -> { + BalancedShardsAllocator.ModelNode node = nodes.get(assignedRoutingNodes.get(routing)); + node.removeShard(routing); + }); + continue; + } + + if (logger.isTraceEnabled()) { + logger.trace("Splitting shard [{}]", splittingShardAssignmentInfo.assignedPrimaryParent.shardId()); + } + + routingNodes.assignChildShards( + splittingShardAssignmentInfo.assignedPrimaryParent, + splittingShardAssignmentInfo.splittingPrimaryParent, + allocation.changes(), + assignedRoutingNodes + ); + } + + } + + private static class SplittingShardAssignmentInfo { + final String indexUUID; + final ShardRouting assignedPrimaryParent; + final ShardRouting splittingPrimaryParent; + final List primaryChildShards; + final List replicaChildShards; + + public SplittingShardAssignmentInfo( + String indexUUID, + ShardRouting assignedPrimaryParent, + ShardRouting splittingPrimaryParent, + List primaryChildShards, + List replicaChildShards) { + this.indexUUID = indexUUID; + this.assignedPrimaryParent = assignedPrimaryParent; + this.splittingPrimaryParent = splittingPrimaryParent; + this.primaryChildShards = primaryChildShards; + this.replicaChildShards = replicaChildShards; + } + } + /** * Make a decision for allocating an unassigned shard. This method returns a two values in a tuple: the * first value is the {@link Decision} taken to allocate the unassigned shard, the second value is the diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/RemoteShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/RemoteShardsBalancer.java index a05938c176678..2af85b79e29b9 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/RemoteShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/RemoteShardsBalancer.java @@ -266,6 +266,10 @@ void balance() { } } + @Override + public void assignChildShardsOfSplittingShards() { + } + /** * Calculates the total number of primary shards per node. * @param remoteRoutingNodes routing nodes for which the aggregation needs to be performed diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java index ef2dbd34644a7..c49e4e101d98b 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java @@ -37,6 +37,11 @@ public abstract class ShardsBalancer { */ abstract void balance(); + /** + * Allocate child shard against a splitting shards. + */ + abstract void assignChildShardsOfSplittingShards(); + /** * Make a decision for allocating an unassigned shard. * @param shardRouting the shard for which the decision has to be made diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/InPlaceShardSplitAllocationDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/InPlaceShardSplitAllocationDecider.java index 2924efd1781d9..fc45f533ca13b 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/InPlaceShardSplitAllocationDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/InPlaceShardSplitAllocationDecider.java @@ -17,18 +17,16 @@ public class InPlaceShardSplitAllocationDecider extends AllocationDecider { public static final String NAME = "in_place_shard_split"; - @Override - public Decision canRemain(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { - return canRemainDecision(shardRouting, node, allocation); + public Decision canRebalance(ShardRouting shardRouting, RoutingAllocation allocation) { + if (shardRouting.isSplitTarget() || shardRouting.splitting()) { + return Decision.NO; + } + return Decision.ALWAYS; } - public static Decision canRemainDecision(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { - // If shardRouting is a started parent shard and fact that it exists is sufficient to conclude - // that it needs to be split. - SplitShardsMetadata splitShardsMetadata = allocation.metadata().getIndexSafe(shardRouting.index()).getSplitShardsMetadata(); - if (splitShardsMetadata.isSplitOfShardInProgress(shardRouting.shardId().id()) - && shardRouting.started() && allocation.changes().isSplitOfShardFailed(shardRouting) == false) { - return Decision.SPLIT; + public Decision canMoveAway(ShardRouting shardRouting, RoutingAllocation allocation) { + if (shardRouting.isSplitTarget() || shardRouting.splitting()) { + return Decision.NO; } return Decision.ALWAYS; } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ReplicaAfterPrimaryActiveAllocationDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ReplicaAfterPrimaryActiveAllocationDecider.java index 52b89f5c403e8..f6b5d0dc40b5b 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ReplicaAfterPrimaryActiveAllocationDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ReplicaAfterPrimaryActiveAllocationDecider.java @@ -52,8 +52,8 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, Routing @Override public Decision canAllocate(ShardRouting shardRouting, RoutingAllocation allocation) { - if (shardRouting.primary()) { - return allocation.decision(Decision.YES, NAME, "shard is primary and can be allocated"); + if (shardRouting.primary() || shardRouting.getParentShardId() != null) { + return allocation.decision(Decision.YES, NAME, "shard is primary or a split target and can be allocated"); } ShardRouting primary = allocation.routingNodes().activePrimary(shardRouting.shardId()); if (primary == null) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/SameShardAllocationDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/SameShardAllocationDecider.java index 0b8e69f97c645..c2eccdbc6ed26 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/SameShardAllocationDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/SameShardAllocationDecider.java @@ -87,13 +87,8 @@ private void setSameHost(boolean sameHost) { @Override public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { - Decision decision = InPlaceShardSplitAllocationDecider.canRemainDecision(shardRouting, node, allocation); - if (decision == Decision.SPLIT) { - return decision; - } - Iterable assignedShards = allocation.routingNodes().assignedShards(shardRouting.shardId()); - decision = decideSameNode(shardRouting, node, allocation, assignedShards); + Decision decision = decideSameNode(shardRouting, node, allocation, assignedShards); if (decision.type() == Decision.Type.NO || sameHost == false) { // if its already a NO decision looking at the node, or we aren't configured to look at the host, return the decision return decision; diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ThrottlingAllocationDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ThrottlingAllocationDecider.java index 26a04de31ce39..1e1d01b328173 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ThrottlingAllocationDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ThrottlingAllocationDecider.java @@ -43,12 +43,12 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.shard.ShardId; import java.util.Locale; import java.util.function.BiFunction; -import static org.opensearch.cluster.routing.allocation.decider.Decision.THROTTLE; -import static org.opensearch.cluster.routing.allocation.decider.Decision.YES; +import static org.opensearch.cluster.routing.allocation.decider.Decision.*; /** * {@link ThrottlingAllocationDecider} controls the recovery process per node in @@ -194,7 +194,7 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, Routing // Peer recovery assert initializingShard(shardRouting, node.nodeId()).recoverySource().getType() == RecoverySource.Type.PEER; - if (shardRouting.unassignedReasonIndexCreated()) { + if (shardRouting.unassignedReasonIndexCreated() || shardRouting.unassignedReasonChildShardCreated()) { return allocateInitialShardCopies(shardRouting, node, allocation); } else { return allocateNonInitialShardCopies(shardRouting, node, allocation); @@ -203,8 +203,14 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, Routing } private Decision allocateInitialShardCopies(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + if (shardRouting.getParentShardId() != null) { + assert shardRouting.primary() == false; + //TODO: Implement throttling for replica recoveries after primary children synced state. + return allocation.decision(YES, NAME, "Skipping throttling for replica recovery"); + } int currentInRecoveries = allocation.routingNodes().getInitialIncomingRecoveries(node.nodeId()); - assert shardRouting.unassignedReasonIndexCreated() && !shardRouting.primary(); + assert (shardRouting.unassignedReasonIndexCreated() || shardRouting.unassignedReasonChildShardCreated()) + && !shardRouting.primary(); return allocateShardCopies( shardRouting, @@ -263,7 +269,13 @@ private Integer getPrimaryNodeOutgoingRecoveries(ShardRouting shardRouting, Rout } private Integer getInitialPrimaryNodeOutgoingRecoveries(ShardRouting shardRouting, RoutingAllocation allocation) { - ShardRouting primaryShard = allocation.routingNodes().activePrimary(shardRouting.shardId()); + ShardId primaryShardId; + if (shardRouting.getParentShardId() != null && shardRouting.primary() == false) { + primaryShardId = shardRouting.getParentShardId(); + } else { + primaryShardId = shardRouting.shardId(); + } + ShardRouting primaryShard = allocation.routingNodes().activePrimary(primaryShardId); return allocation.routingNodes().getInitialOutgoingRecoveries(primaryShard.currentNodeId()); } @@ -353,6 +365,9 @@ private ShardRouting initializingShard(ShardRouting shardRouting, String current @Override public Decision canMoveAway(ShardRouting shardRouting, RoutingAllocation allocation) { + if (shardRouting.getParentShardId() != null) { + return allocation.decision(NO, NAME, "recovering child shard"); + } int outgoingRecoveries = 0; if (!shardRouting.primary()) { ShardRouting primaryShard = allocation.routingNodes().activePrimary(shardRouting.shardId()); diff --git a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java index 4706e3610a25d..e41fbe44ca5dd 100644 --- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java @@ -919,9 +919,14 @@ public IndexResult index(Index index) throws IOException { } assert index.seqNo() >= 0 : "ops should have an assigned seq no.; origin: " + index.origin(); - if (plan.indexIntoLucene || plan.addStaleOpToLucene) { indexResult = indexIntoLucene(index, plan); +// if (shardId.id() == 0 && index.origin() == Operation.Origin.PRIMARY) { +// logger.info("Sending operation seq " + index.seqNo()); +// } +// if (shardId.id() == 0 && index.origin() == Operation.Origin.PRIMARY) { +// logger.info("Indexing operation seq " + index.seqNo() + " into lucene: " + plan.indexIntoLucene + " index result " + indexResult.getResultType()); +// } } else { indexResult = new IndexResult( plan.versionForIndexing, @@ -936,6 +941,9 @@ public IndexResult index(Index index) throws IOException { final Translog.Location location; if (indexResult.getResultType() == Result.Type.SUCCESS) { location = translogManager.add(new Translog.Index(index, indexResult)); +// if (shardId.id() == 0 && index.origin() == Operation.Origin.PRIMARY) { +// logger.info("Adding operation seq " + index.seqNo() + " to translog generation " + location.generation); +// } } else if (indexResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { // if we have document failure, record it as a no-op in the translog and Lucene with the generated seq_no final NoOp noOp = new NoOp( @@ -999,7 +1007,6 @@ protected final IndexingStrategy planIndexingAsNonPrimary(Index index) throws IO // a delete state and return false for the created flag in favor of code simplicity final long maxSeqNoOfUpdatesOrDeletes = getMaxSeqNoOfUpdatesOrDeletes(); if (hasBeenProcessedBefore(index)) { - logger.info("Has been processed before"); // the operation seq# was processed and thus the same operation was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already // part of the lucene commit (either from a peer recovery or a local translog) @@ -1735,9 +1742,6 @@ private NoOpResult innerNoOp(final NoOp noOp) throws IOException { } } localCheckpointTracker.markSeqNoAsProcessed(noOpResult.getSeqNo()); - if (seqNo == 490 && shardId.id() > 2) { - logger.info("Marking as no op in internal engine "); - } if (noOpResult.getTranslogLocation() == null) { // the op is coming from the translog (and is hence persisted already) or it does not have a sequence number assert noOp.origin().isFromTranslog() || noOpResult.getSeqNo() == SequenceNumbers.UNASSIGNED_SEQ_NO; diff --git a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java index 10d459878a378..91caaf95c120d 100644 --- a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java @@ -960,7 +960,7 @@ private boolean invariant() { // all tracked shard copies have a corresponding peer-recovery retention lease for (final ShardRouting shardRouting : routingTable.assignedShards()) { final CheckpointState cps = checkpoints.get(shardRouting.allocationId().getId()); - if (cps.tracked && cps.replicated) { + if (cps.tracked && cps.replicated && shardRouting.getParentShardId() == null) { assert retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting)) : "no retention lease for tracked shard [" + shardRouting + "] in " + retentionLeases; assert PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals( @@ -1233,10 +1233,10 @@ public ReplicationCheckpoint getLatestReplicationCheckpoint() { return this.latestReplicationCheckpoint; } - private boolean isPrimaryRelocation(String allocationId) { + private boolean isPrimaryRelocationOrChild(String allocationId) { Optional shardRouting = routingTable.shards() .stream() - .filter(routing -> routing.allocationId().getId().equals(allocationId)) + .filter(routing -> routing.allocationId().getId().equals(allocationId) || routing.isSplitTarget()) .findAny(); return shardRouting.isPresent() && shardRouting.get().primary(); } @@ -1250,7 +1250,7 @@ private void createReplicationLagTimers() { // it is possible for a shard to be in-sync but not yet removed from the checkpoints collection after a failover event. if (cps.inSync && replicationGroup.getUnavailableInSyncShards().contains(allocationId) == false - && isPrimaryRelocation(allocationId) == false + && isPrimaryRelocationOrChild(allocationId) == false && latestReplicationCheckpoint.isAheadOf(cps.visibleReplicationCheckpoint)) { cps.checkpointTimers.computeIfAbsent(latestReplicationCheckpoint, ignored -> new SegmentReplicationLagTimer()); logger.trace( @@ -1282,7 +1282,7 @@ public synchronized void startReplicationLagTimers(ReplicationCheckpoint checkpo final CheckpointState cps = e.getValue(); if (cps.inSync && replicationGroup.getUnavailableInSyncShards().contains(allocationId) == false - && isPrimaryRelocation(e.getKey()) == false + && isPrimaryRelocationOrChild(e.getKey()) == false && latestReplicationCheckpoint.isAheadOf(cps.visibleReplicationCheckpoint) && cps.checkpointTimers.containsKey(latestReplicationCheckpoint)) { cps.checkpointTimers.get(latestReplicationCheckpoint).start(); @@ -1307,7 +1307,7 @@ public synchronized Set getSegmentReplicationStats entry -> entry.getKey().equals(this.shardAllocationId) == false && entry.getValue().inSync && replicationGroup.getUnavailableInSyncShards().contains(entry.getKey()) == false - && isPrimaryRelocation(entry.getKey()) == false + && isPrimaryRelocationOrChild(entry.getKey()) == false ) .map(entry -> buildShardStats(entry.getKey(), entry.getValue())) .collect(Collectors.toUnmodifiableSet()); @@ -1625,12 +1625,14 @@ public synchronized void markAllocationIdAsInSync(final String allocationId, fin + getGlobalCheckpoint() + " or it's not replicated"; if (cps.replicated && cps.localCheckpoint < getGlobalCheckpoint()) { + logger.info("Waiting for sync as local checkpoint " + cps.localCheckpoint + " is less than global checkpoint " + getGlobalCheckpoint()); pendingInSync.add(allocationId); try { while (true) { if (pendingInSync.contains(allocationId)) { waitForLocalCheckpointToAdvance(); } else { + logger.info("Wait over for sync as local checkpoint " + cps.localCheckpoint + " is less than global checkpoint " + getGlobalCheckpoint()); break; } } @@ -1679,9 +1681,6 @@ public synchronized void updateLocalCheckpoint(final String allocationId, final assert invariant(); assert primaryMode; assert handoffInProgress == false; - if (shardId.id() > 2) { - System.out.println(); - } CheckpointState cps = checkpoints.get(allocationId); if (cps == null) { // can happen if replica was removed from cluster but replication process is unaware of it yet @@ -1838,8 +1837,9 @@ public synchronized void activateWithPrimaryContext(PrimaryContext primaryContex if (contextRoutingTable.primaryShard().splitting()) { ShardRouting[] allChildShardRoutings = contextRoutingTable.primaryShard().getRecoveringChildShards(); for (ShardRouting childRouting : allChildShardRoutings) { - // moveToStarted doesn't mutate original routing. So, it's safe to do this. - childRouting = childRouting.moveToStarted(); + if (childRouting.primary()) { + childRouting = childRouting.moveToStarted(); + } if (childRouting.shardId().equals(shardId)) { if (childRoutingTable == null) { childRoutingTable = new IndexShardRoutingTable.Builder(shardId); diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 431ecd3e7e053..be7d1d95ed444 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -1675,7 +1675,8 @@ public void finalizeReplication(SegmentInfos infos) throws IOException { public GatedCloseable acquireSafeIndexCommit() throws EngineException { final IndexShardState state = this.state; // one time volatile read // we allow snapshot on closed index shard, since we want to do one after we close the shard and before we close the engine - if (state == IndexShardState.STARTED || state == IndexShardState.CLOSED) { + // We already asserted earlier that source child shard is in synced state for us to be able to acquire a commit. + if (state == IndexShardState.STARTED || state == IndexShardState.CLOSED || routingEntry().isSplitTarget()) { return getEngine().acquireSafeIndexCommit(); } else { throw new IllegalIndexShardStateException(shardId, state, "snapshot is not allowed"); @@ -2378,6 +2379,10 @@ private Engine.Result applyTranslogOperation(Engine engine, Translog.Operation o default: throw new IllegalStateException("No operation defined for [" + operation + "]"); } + if (shardRouting.isSplitTarget() && shardRouting.primary()) { +// logger.info("Applied seq no. " + operation.seqNo() + " on child shard replica " + shardId.id() +// + ". Local checkpoint now " + getLocalCheckpoint()); + } return result; } @@ -2399,7 +2404,7 @@ private Translog.Operation overrideToNoOpIfOperationNotForShard(Translog.Operati int computedShardId = OperationRouting.generateShardId(indexSettings().getIndexMetadata(), index.id(), index.routing(), true); if (computedShardId != shardId().id()) { - return new Translog.NoOp(index.seqNo(), index.primaryTerm(), "op belongs to another child shard"); + return new Translog.NoOp(index.seqNo(), index.primaryTerm(), Translog.NoOp.FILLING_GAPS); } return operation; @@ -3294,6 +3299,19 @@ public void removeRetentionLease(final String id, final ActionListener acquireRetentionLockWithMinGen() { - try (ReleasableLock lock = readLock.acquire()) { + try (ReleasableLock ignore = readLock.acquire()) { ensureOpen(); final long viewGen = getMinFileGeneration(); - logger.info("Minimum translog with gen " + viewGen + " seq number " + current.getCheckpoint().minSeqNo); Closeable closeable = acquireTranslogGenFromDeletionPolicy(viewGen); return new GatedCloseable<>(viewGen, closeable::close); } @@ -795,7 +807,6 @@ public static String getCommitCheckpointFileName(long generation) { */ public void trimOperations(long belowTerm, long aboveSeqNo) throws IOException { assert aboveSeqNo >= SequenceNumbers.NO_OPS_PERFORMED : "aboveSeqNo has to a valid sequence number"; - try (ReleasableLock lock = writeLock.acquire()) { ensureOpen(); if (current.getPrimaryTerm() < belowTerm) { @@ -1508,6 +1519,7 @@ public String toString() { * @opensearch.internal */ public static class NoOp implements Operation { + public static final String FILLING_GAPS = "filling gaps"; private final long seqNo; private final long primaryTerm; diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index fe953597e39eb..7fcb50c04a11d 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -1036,8 +1036,9 @@ public IndexShard createShard( retentionLeaseSyncer, checkpointPublisher, remoteStoreStatsTrackerFactory, - null + shardRouting.getParentShardId() ); + indexShard.addShardFailureCallback(onShardFailure); indexShard.startRecovery(recoveryState, recoveryTargetService, recoveryListener, repositoriesService, mapping -> { assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.LOCAL_SHARDS @@ -1072,7 +1073,6 @@ public void createChildShardsForSplit( IndexShard parentShard = indexService.getShard(parentShardId.id()); List recoveryContexts = new ArrayList<>(); - List shardIds = new ArrayList<>(); for (ShardRouting shardRouting : shardRoutings) { RecoveryState recoveryState = indexService.createRecoveryState(shardRouting, node, node); IndexShard indexShard = indexService.createShard( @@ -1085,7 +1085,6 @@ public void createChildShardsForSplit( ); indexShard.addShardFailureCallback(onShardFailure); recoveryContexts.add(new InPlaceShardRecoveryContext(recoveryState, indexShard, parentShard)); - shardIds.add(indexShard.shardId()); } for (InPlaceShardRecoveryContext recoveryContext : recoveryContexts) { @@ -1093,8 +1092,13 @@ public void createChildShardsForSplit( recoveryContext.getIndexShard().markAsRecovering("from in-place shard split", recoveryContext.getRecoveryState()); } - threadPool.generic().execute(() -> inPlaceShardSplitRecoveryService.addAndStartRecovery( - recoveryContexts, node, parentShard, recoveryListener, shardIds, request)); + threadPool.generic().execute(() -> inPlaceShardSplitRecoveryService.addAndStartRecovery(recoveryContexts, node, + parentShard, recoveryListener, request, indexService.getMetadata())); + } + + public void moveChildShardsToStarted(ShardId parentShardId, + InPlaceShardSplitRecoveryService inPlaceShardSplitRecoveryService) { + threadPool.generic().execute(() -> inPlaceShardSplitRecoveryService.startChildShards(parentShardId)); } @Override diff --git a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java index 094a95414c4c3..1aa9ce35e9ebd 100644 --- a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java @@ -102,7 +102,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -655,7 +654,7 @@ private void createOrUpdateShards(final ClusterState state) { DiscoveryNodes nodes = state.nodes(); RoutingTable routingTable = state.routingTable(); - Map>> childShardRoutings = new HashMap<>(); + Map>> childShardsToProcess = new HashMap<>(); for (final ShardRouting shardRouting : localRoutingNode) { ShardId shardId = shardRouting.shardId(); if (failedShardsCache.containsKey(shardId) == false) { @@ -663,12 +662,12 @@ private void createOrUpdateShards(final ClusterState state) { assert indexService != null : "index " + shardId.getIndex() + " should have been created by createIndices"; Shard shard = indexService.getShardOrNull(shardId.id()); if (shard == null) { - if (shardRouting.isSplitTarget()) { + if (shardRouting.isSplitTarget() && shardRouting.primary()) { Shard sourceShard = indexService.getShardOrNull(shardRouting.getParentShardId().id()); - assert sourceShard!= null : "Source shard not found for shard id " + shardRouting.getParentShardId(); - childShardRoutings.computeIfAbsent(shardRouting.getParentShardId(), k -> + assert sourceShard != null : "parent shard not found for shard id " + shardRouting.getParentShardId(); + childShardsToProcess.computeIfAbsent(shardRouting.getParentShardId(), k -> new Tuple<>(sourceShard.routingEntry(), new ArrayList<>())); - childShardRoutings.get(shardRouting.getParentShardId()).v2().add(shardRouting); + childShardsToProcess.get(shardRouting.getParentShardId()).v2().add(shardRouting); } else { assert shardRouting.initializing() : shardRouting + " should have been removed by failMissingShards"; createShard(nodes, routingTable, shardRouting, state); @@ -679,8 +678,9 @@ private void createOrUpdateShards(final ClusterState state) { } } - if (!childShardRoutings.isEmpty()) { - createChildShardsForSplit(nodes, childShardRoutings, state); + if (!childShardsToProcess.isEmpty()) { + // Recovery flow always starts with the recovery of primary child shards first and then replica. + createChildShardsForSplit(nodes, childShardsToProcess, state); } } @@ -747,7 +747,12 @@ private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardR } try { - final long primaryTerm = state.metadata().index(shardRouting.index()).primaryTerm(shardRouting.id()); + final long primaryTerm; + if (shardRouting.isSplitTarget() && shardRouting.primary() == false) { + primaryTerm = state.metadata().index(shardRouting.index()).primaryTerm(shardRouting.getParentShardId().id()); + } else { + primaryTerm = state.metadata().index(shardRouting.index()).primaryTerm(shardRouting.id()); + } logger.debug("{} creating shard with primary term [{}]", shardRouting.shardId(), primaryTerm); indicesService.createShard( shardRouting, @@ -781,17 +786,28 @@ private void updateShard( + shardRouting + " local: " + currentRoutingEntry; - if (currentRoutingEntry.isSplitTarget() && shardRouting.isSplitTarget()) { + if (shardRouting.initializing() && shardRouting.getParentShardId() != null) { // Nothing to update yet on recovering child shard. + assert currentRoutingEntry.initializing(); return; } final long primaryTerm; try { final IndexMetadata indexMetadata = clusterState.metadata().index(shard.shardId().getIndex()); - primaryTerm = indexMetadata.primaryTerm(shard.shardId().id()); + final IndexShardRoutingTable indexShardRoutingTable; + if (shardRouting.isStartedChildReplica()) { + if (currentRoutingEntry.isStartedChildReplica() == false) { + logger.info("Starting child replica {} on node {}", shardRouting.shardId().id(), + clusterState.nodes().getLocalNode().getName()); + } + indexShardRoutingTable = routingTable.childReplicaShardRoutingTable(shardRouting.shardId()); + primaryTerm = indexMetadata.primaryTerm(shard.getParentShardId().id()); + } else { + indexShardRoutingTable = routingTable.shardRoutingTable(shardRouting.shardId()); + primaryTerm = indexMetadata.primaryTerm(shard.shardId().id()); + } final Set inSyncIds = indexMetadata.inSyncAllocationIds(shard.shardId().id()); - final IndexShardRoutingTable indexShardRoutingTable = routingTable.shardRoutingTable(shardRouting.shardId()); shard.updateShardState( shardRouting, primaryTerm, @@ -832,6 +848,22 @@ private void updateShard( ); } } + + if (shardRouting.splitting()) { + final IndexMetadata indexMetadata = clusterState.metadata().index(shard.shardId().getIndex()); + if (indexMetadata.getNumberOfReplicas() > 0) { + boolean allChildReplicasRecovered = true; + for (ShardRouting childShard : shardRouting.getRecoveringChildShards()) { + if (childShard.primary() == false && childShard.isStartedChildReplica() == false) { + allChildReplicasRecovered = false; + break; + } + } + if (allChildReplicasRecovered == true) { + indicesService.moveChildShardsToStarted(shardRouting.shardId(), inPlaceShardSplitRecoveryService); + } + } + } } /** @@ -846,7 +878,12 @@ private static DiscoveryNode findSourceNodeForPeerRecovery( ) { DiscoveryNode sourceNode = null; if (!shardRouting.primary()) { - ShardRouting primary = routingTable.shardRoutingTable(shardRouting.shardId()).primaryShard(); + ShardRouting primary; + if (shardRouting.isSplitTarget()) { + primary = routingTable.shardRoutingTable(shardRouting.getParentShardId()).primaryShard(); + } else { + primary = routingTable.shardRoutingTable(shardRouting.shardId()).primaryShard(); + } // only recover from started primary, if we can't find one, we will do it next round if (primary.active()) { sourceNode = nodes.get(primary.currentNodeId()); @@ -887,9 +924,8 @@ public void handleRecoveryDone(ReplicationState state, ShardRouting shardRouting shardStateAction.shardStarted(shardRouting, primaryTerm, "after " + recoveryState.getRecoverySource(), SHARD_STATE_ACTION_LISTENER); } - public void handleChildRecoveriesDone(ShardRouting sourceShardRouting, long primaryTerm, RecoverySource recoverySource) { - - shardStateAction.childShardsStarted(sourceShardRouting, primaryTerm, "after " + recoverySource, SHARD_STATE_ACTION_LISTENER); + public void handleChildRecoveriesDone(ShardRouting parentShardRouting, long primaryTerm, RecoverySource recoverySource) { + shardStateAction.childShardsStarted(parentShardRouting, primaryTerm, "after " + recoverySource, SHARD_STATE_ACTION_LISTENER); } private void failAndRemoveShard( @@ -980,7 +1016,7 @@ private void sendFailChildShards(ShardRouting shardRouting, String message, @Nul for (ShardRouting childShard : shardRouting.getRecoveringChildShards()) { failedShardsCache.put(childShard.shardId(), childShard); } - shardStateAction.localShardFailed(shardRouting, message, failure, SHARD_STATE_ACTION_LISTENER, state, true); + shardStateAction.localShardFailed(shardRouting, message, failure, SHARD_STATE_ACTION_LISTENER, state); } catch (Exception inner) { if (failure != null) inner.addSuppressed(failure); logger.warn( @@ -1215,6 +1251,13 @@ void createChildShardsForSplit( StartRecoveryRequest request ) throws IOException; + /** + * Start child shards in place of parent shard after replicas have started. + * @param parentShardId Shard ID of parent + * @param inPlaceShardSplitRecoveryService recovery service for in-place shard recovery + */ + void moveChildShardsToStarted(ShardId parentShardId, InPlaceShardSplitRecoveryService inPlaceShardSplitRecoveryService); + /** * Returns shard for the specified id if it exists otherwise returns null. */ diff --git a/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java index 7066f8bd3cf7f..7cd6d689cec8a 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java +++ b/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java @@ -34,7 +34,6 @@ import java.io.Closeable; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.function.Consumer; @@ -56,29 +55,52 @@ public LocalStorePeerRecoverySourceHandler( int maxConcurrentOperations, boolean skipSegmentFilesTransfer, CancellableThreads cancellableThreads + ) { + this(shard, recoveryTarget, threadPool, request, fileChunkSizeInBytes, maxConcurrentFileChunks, maxConcurrentOperations, + skipSegmentFilesTransfer, cancellableThreads, null); + } + + public LocalStorePeerRecoverySourceHandler( + IndexShard shard, + RecoveryTargetHandler recoveryTarget, + ThreadPool threadPool, + StartRecoveryRequest request, + int fileChunkSizeInBytes, + int maxConcurrentFileChunks, + int maxConcurrentOperations, + boolean skipSegmentFilesTransfer, + CancellableThreads cancellableThreads, + IndexShard parentShard ) { super(shard, recoveryTarget, threadPool, request, fileChunkSizeInBytes, maxConcurrentFileChunks, maxConcurrentOperations, - skipSegmentFilesTransfer, cancellableThreads); + skipSegmentFilesTransfer, cancellableThreads, parentShard); } @Override protected void innerRecoveryToTarget(ActionListener listener, Consumer onFailure) throws IOException { final SetOnce retentionLeaseRef = new SetOnce<>(); - RunUnderPrimaryPermit.run(() -> { - final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); - ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); - if (targetShardRouting == null) { - logger.debug( - "delaying recovery of {} as it is not listed as assigned to target node {}", - request.shardId(), - request.targetNode() - ); - throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); - } - assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; - retentionLeaseRef.set(shard.getRetentionLeases().get(ReplicationTracker.getPeerRecoveryRetentionLeaseId(targetShardRouting))); - }, shardId + " validating recovery target [" + request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); + // We neither add retention lease on child primary nor it is available. This is because we never resume + // peer recovery and always start again from the beginning in case of failure. + if (shard.routingEntry().isSplitTarget() == false) { + RunUnderPrimaryPermit.run(() -> { + final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); + ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); + if (targetShardRouting == null) { + logger.debug( + "delaying recovery of {} as it is not listed as assigned to target node {}", + request.shardId(), + request.targetNode() + ); + throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); + } + assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; + retentionLeaseRef.set(shard.getRetentionLeases().get(ReplicationTracker.getPeerRecoveryRetentionLeaseId(targetShardRouting))); + }, shardId + " validating recovery target [" + request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); + } else { + assert parentShard != null; + } + final Closeable retentionLock = acquireRetentionLock(); resources.add(retentionLock); final long startingSeqNo; @@ -145,26 +167,31 @@ && isTargetSameHistory() onSendFileStepComplete(sendFileStep, wrappedSafeCommit, releaseStore); final StepListener deleteRetentionLeaseStep = new StepListener<>(); - RunUnderPrimaryPermit.run(() -> { - try { - // If the target previously had a copy of this shard then a file-based recovery might move its global - // checkpoint backwards. We must therefore remove any existing retention lease so that we can create a - // new one later on in the recovery. - shard.removePeerRecoveryRetentionLease( - request.targetNode().getId(), - new ThreadedActionListener<>( - logger, - shard.getThreadPool(), - ThreadPool.Names.GENERIC, - deleteRetentionLeaseStep, - false - ) - ); - } catch (RetentionLeaseNotFoundException e) { - logger.debug("no peer-recovery retention lease for " + request.targetAllocationId()); - deleteRetentionLeaseStep.onResponse(null); - } - }, shardId + " removing retention lease for [" + request.targetAllocationId() + "]", shard, cancellableThreads, logger); + if (shard.routingEntry().isSplitTarget() == true) { + // For the same reason above, we don't have any retention lease for a child primary to delete. + deleteRetentionLeaseStep.onResponse(null); + } else { + RunUnderPrimaryPermit.run(() -> { + try { + // If the target previously had a copy of this shard then a file-based recovery might move its global + // checkpoint backwards. We must therefore remove any existing retention lease so that we can create a + // new one later on in the recovery. + shard.removePeerRecoveryRetentionLease( + request.targetNode().getId(), + new ThreadedActionListener<>( + logger, + shard.getThreadPool(), + ThreadPool.Names.GENERIC, + deleteRetentionLeaseStep, + false + ) + ); + } catch (RetentionLeaseNotFoundException e) { + logger.debug("no peer-recovery retention lease for " + request.targetAllocationId()); + deleteRetentionLeaseStep.onResponse(null); + } + }, shardId + " removing retention lease for [" + request.targetAllocationId() + "]", shard, cancellableThreads, logger); + } deleteRetentionLeaseStep.whenComplete(ignored -> { logger.debug("deleteRetentionLeaseStep completed"); @@ -194,13 +221,15 @@ && isTargetSameHistory() * make sure to do this before sampling the max sequence number in the next step, to ensure that we send * all documents up to maxSeqNo in phase2. */ + IndexShard primaryTracker = replicationTrackingShard(); RunUnderPrimaryPermit.run( - () -> shard.initiateTracking(request.targetAllocationId()), - shardId + " initiating tracking of " + request.targetAllocationId(), - shard, + () -> primaryTracker.initiateTracking(request.targetAllocationId()), + primaryTracker.shardId() + " initiating tracking of " + request.targetAllocationId(), + primaryTracker, cancellableThreads, logger ); + Thread.sleep(1000); final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); if (logger.isTraceEnabled()) { @@ -216,6 +245,7 @@ && isTargetSameHistory() final long maxSeqNoOfUpdatesOrDeletes = shard.getMaxSeqNoOfUpdatesOrDeletes(); final RetentionLeases retentionLeases = shard.getRetentionLeases(); final long mappingVersionOnPrimary = shard.indexSettings().getIndexMetadata().getMappingVersion(); + logger.info("Phase 1 completed till seq no. " + (startingSeqNo - 1) + " phase2 starting from seq no " + startingSeqNo + " and ending seq no " + endingSeqNo); phase2( startingSeqNo, endingSeqNo, diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java index 7425c15641079..aac891f8d1eeb 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java @@ -56,6 +56,7 @@ import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.inplacesplit.InPlaceShardSplitRecoveryService; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportChannel; @@ -93,14 +94,17 @@ public static class Actions { private final TransportService transportService; private final IndicesService indicesService; private final RecoverySettings recoverySettings; + private final InPlaceShardSplitRecoveryService splitRecoveryService; final OngoingRecoveries ongoingRecoveries = new OngoingRecoveries(); @Inject - public PeerRecoverySourceService(TransportService transportService, IndicesService indicesService, RecoverySettings recoverySettings) { + public PeerRecoverySourceService(TransportService transportService, IndicesService indicesService, + RecoverySettings recoverySettings, InPlaceShardSplitRecoveryService splitRecoveryService) { this.transportService = transportService; this.indicesService = indicesService; this.recoverySettings = recoverySettings; + this.splitRecoveryService = splitRecoveryService; // When the target node wants to start a peer recovery it sends a START_RECOVERY request to the source // node. Upon receiving START_RECOVERY, the source node will initiate the peer recovery. transportService.registerRequestHandler( @@ -163,6 +167,18 @@ private void recover(StartRecoveryRequest request, ActionListener initiateRecoveryListener = ActionListener.delegateFailure(listener, + (ex, ignore) -> { + final IndexShard parentShard = indexService.getShard(routingEntry.getParentShardId().id()); + assert parentShard.getReplicationGroup().getInSyncAllocationIds().contains(routingEntry.allocationId().getId()); + initiateRecovery(request, shard, listener); + }); + logger.info("Adding child replica recovery for node " + request.targetNode().getName() + " on parent shard node " + indicesService.clusterService().localNode().getName()); + splitRecoveryService.addReplicaRecoveryAfterChildPrimariesSync(routingEntry.getParentShardId(), initiateRecoveryListener); + return; + } + if (routingEntry.primary() == false || routingEntry.active() == false) { throw new DelayRecoveryException("source shard [" + routingEntry + "] is not an active primary"); } @@ -177,6 +193,10 @@ private void recover(StartRecoveryRequest request, ActionListener listener) { RecoverySourceHandler handler = ongoingRecoveries.addNewRecovery(request, shard); logger.trace( "[{}][{}] starting recovery to {}", @@ -380,8 +400,13 @@ private Tuple createRecovery throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime), shard.isRemoteTranslogEnabled() ); + IndexShard parentShard = null; + if (shard.routingEntry().isSplitTarget()) { + parentShard = indicesService.getShardOrNull(shard.getParentShardId()); + assert parentShard != null; + } handler = RecoverySourceHandlerFactory.create(shard, recoveryTarget, request, recoverySettings, false, - new CancellableThreads()); + new CancellableThreads(), parentShard); return Tuple.tuple(handler, recoveryTarget); } } diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java index 530b9eeb4e90c..819c9445c698e 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java @@ -58,7 +58,9 @@ import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.RecoveryEngineException; +import org.opensearch.index.seqno.ReplicationTracker; import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.seqno.RetentionLeaseNotFoundException; import org.opensearch.index.seqno.RetentionLeases; @@ -77,6 +79,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashSet; @@ -121,6 +124,7 @@ public abstract class RecoverySourceHandler { protected final ListenableFuture future = new ListenableFuture<>(); public static final String PEER_RECOVERY_NAME = "peer-recovery"; private final SegmentFileTransferHandler transferHandler; + protected final IndexShard parentShard; protected RecoverySourceHandler( IndexShard shard, @@ -131,7 +135,8 @@ protected RecoverySourceHandler( int maxConcurrentFileChunks, int maxConcurrentOperations, boolean skipSegmentFilesTransfer, - CancellableThreads cancellableThreads + CancellableThreads cancellableThreads, + IndexShard parentShard ) { this.cancellableThreads = cancellableThreads; this.logger = Loggers.getLogger(RecoverySourceHandler.class, request.shardId(), "recover to " + request.targetNode().getName()); @@ -161,6 +166,7 @@ protected RecoverySourceHandler( resources.addAll(getAdditionalResourcesToClose()); IOUtils.close(resources); }; + this.parentShard = parentShard; } public StartRecoveryRequest getRequest() { @@ -191,6 +197,7 @@ public void recoverToTarget(ActionListener listener) { throw e; }); final Consumer onFailure = e -> { + e.printStackTrace(); assert Transports.assertNotTransportThread(this + "[onFailure]"); IOUtils.closeWhileHandlingException(resourcesReleasable, () -> future.onFailure(e)); }; @@ -200,6 +207,13 @@ public void recoverToTarget(ActionListener listener) { } } + protected IndexShard replicationTrackingShard() { + if (shard.routingEntry().isSplitTarget() == false) { + return shard; + } + return parentShard; + } + public List getAdditionalResourcesToClose() { return new ArrayList<>(); } @@ -405,7 +419,6 @@ protected void phase1( final Store.MetadataSnapshot recoverySourceMetadata; try { recoverySourceMetadata = store.getMetadata(snapshot); - logger.info("Metadata docs " + recoverySourceMetadata.getNumDocs()); } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { shard.failShard("recovery", ex); throw ex; @@ -502,6 +515,12 @@ protected void phase1( logger.debug("sendFilesStep completed"); createRetentionLeaseStep.onResponse(null); }, listener::onFailure); + } else if (shard.routingEntry().isSplitTarget() == true) { + sendFilesStep.whenComplete(r -> { + logger.debug("sendFilesStep completed"); + addRetentionLeaseForChildReplica(startingSeqNo); + createRetentionLeaseStep.onResponse(null); + }, listener::onFailure); } else { sendFilesStep.whenComplete(r -> { logger.debug("sendFilesStep completed"); @@ -620,6 +639,28 @@ void createRetentionLease(final long startingSeqNo, ActionListener curLeases = retentionLeases.leases(); + Collection updatedLeases = new ArrayList<>(curLeases); + updatedLeases.add(retentionLease); + + // Primary mode on replication tracker of child primary will also be false and hence, + // we need to update retention leases as replica. + shard.updateRetentionLeasesOnChildPrimary(new RetentionLeases( + retentionLeases.primaryTerm(), retentionLeases.version() + 1, updatedLeases + )); + } + boolean canSkipPhase1(Store.MetadataSnapshot source, Store.MetadataSnapshot target) { if (source.getSyncId() == null || source.getSyncId().equals(target.getSyncId()) == false) { return false; @@ -742,7 +783,8 @@ protected OperationBatchSender createSender( maxSeqNoOfUpdatesOrDeletes, retentionLeases, mappingVersion, - sendListener + sendListener, + replicationTrackingShard() == parentShard ); } @@ -780,6 +822,7 @@ public List getOperations() { protected class OperationBatchSender extends MultiChunkTransfer { protected final long startingSeqNo; protected final long endingSeqNo; + private long lastSeenSeqNo; protected final Translog.Snapshot snapshot; protected final long maxSeenAutoIdTimestamp; protected final long maxSeqNoOfUpdatesOrDeletes; @@ -789,6 +832,7 @@ protected class OperationBatchSender extends MultiChunkTransfer listener + ActionListener listener, + boolean fillMissingSeqNos ) { super(logger, threadPool.getThreadContext(), listener, maxConcurrentOperations, Collections.singletonList(snapshot)); this.startingSeqNo = startingSeqNo; @@ -808,6 +853,8 @@ protected OperationBatchSender( this.maxSeqNoOfUpdatesOrDeletes = maxSeqNoOfUpdatesOrDeletes; this.retentionLeases = retentionLeases; this.mappingVersion = mappingVersion; + this.fillMissingSeqNos = fillMissingSeqNos; + this.lastSeenSeqNo = startingSeqNo - 1; } @Override @@ -828,7 +875,9 @@ protected synchronized OperationChunkRequest nextChunkRequest(Translog.Snapshot skippedOps.incrementAndGet(); continue; } + fillMissingSeqNos(operation, ops); ops.add(operation); + lastSeenSeqNo = operation.seqNo(); batchSizeInBytes += operation.estimateSize(); sentOps.incrementAndGet(); @@ -838,15 +887,40 @@ protected synchronized OperationChunkRequest nextChunkRequest(Translog.Snapshot } } lastBatchCount = ops.size(); + fillMissingSeqNos(operation, ops); return new OperationChunkRequest(ops, operation == null); } + /** + * Needed in recovery of child replica where recovering replica may not find all consecutive operations on + * their primary child counterparts. + */ + private void fillMissingSeqNos(Translog.Operation op, List ops) { + if (fillMissingSeqNos == false) { + return; + } + + if (op == null) { + for (long seqNo = lastSeenSeqNo + 1; seqNo <= endingSeqNo; seqNo++) { + ops.add(new Translog.NoOp(seqNo, shard.getOperationPrimaryTerm(), Translog.NoOp.FILLING_GAPS)); + } + } else { + final long expectedSeqNo = lastSeenSeqNo + 1; + if (op.seqNo() != expectedSeqNo) { + for (long seqNo = expectedSeqNo; seqNo < op.seqNo(); seqNo++) { + ops.add(new Translog.NoOp(seqNo, shard.getOperationPrimaryTerm(), Translog.NoOp.FILLING_GAPS)); + } + } + } + } + @Override protected void executeChunkRequest(OperationChunkRequest request, ActionListener listener) { cancellableThreads.checkForCancel(); + int totalOperations = fillMissingSeqNos ? (int) (endingSeqNo - startingSeqNo + 1) : snapshot.totalOperations(); recoveryTarget.indexTranslogOperations( request.operations, - snapshot.totalOperations(), + totalOperations, maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases, @@ -875,9 +949,11 @@ void finalizeRecovery(List sendSnapshotResults, long trimAbo } cancellableThreads.checkForCancel(); StopWatch stopWatch = new StopWatch().start(); - logger.info("finalizing recovery"); markAllocationIdAsInSync(sendSnapshotResults); + finalizeRecovery(stopWatch, trimAboveSeqNo, listener); + } + protected void finalizeRecovery(StopWatch stopWatch, long trimAboveSeqNo, ActionListener listener) { final long globalCheckpoint = shard.getLastKnownGlobalCheckpoint(); // this global checkpoint is persisted in finalizeRecovery final StepListener finalizeListener = new StepListener<>(); cancellableThreads.checkForCancel(); @@ -920,10 +996,11 @@ protected void markAllocationIdAsInSync(List sendSnapshotRes * the permit then the state of the shard will be relocated and this recovery will fail. */ sendSnapshotResults.forEach(snapshotResult -> { + IndexShard primaryTracker = replicationTrackingShard(); RunUnderPrimaryPermit.run( - () -> shard.markAllocationIdAsInSync(snapshotResult.targetAllocationId, snapshotResult.targetLocalCheckpoint), - shardId + " marking " + snapshotResult.targetAllocationId+ " as in sync", - shard, + () -> primaryTracker.markAllocationIdAsInSync(snapshotResult.targetAllocationId, snapshotResult.targetLocalCheckpoint), + primaryTracker.shardId() + " marking " + snapshotResult.targetAllocationId+ " as in sync", + primaryTracker, cancellableThreads, logger ); @@ -932,10 +1009,11 @@ protected void markAllocationIdAsInSync(List sendSnapshotRes } protected void updateGlobalCheckpointForShard(long globalCheckpoint) { + IndexShard primaryTracker = replicationTrackingShard(); RunUnderPrimaryPermit.run( - () -> shard.updateGlobalCheckpointForShard(request.targetAllocationId(), globalCheckpoint), - shardId + " updating " + request.targetAllocationId() + "'s global checkpoint", - shard, + () -> primaryTracker.updateGlobalCheckpointForShard(request.targetAllocationId(), globalCheckpoint), + primaryTracker.shardId() + " updating " + request.targetAllocationId() + "'s global checkpoint", + primaryTracker, cancellableThreads, logger ); diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java index 813eb3b5bedfc..eae3137236cb9 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java @@ -24,7 +24,8 @@ public static RecoverySourceHandler create( StartRecoveryRequest request, RecoverySettings recoverySettings, boolean skipSegmentFilesTransfer, - CancellableThreads cancellableThreads + CancellableThreads cancellableThreads, + IndexShard parentShard ) { boolean isReplicaRecoveryWithRemoteTranslog = request.isPrimaryRelocation() == false && shard.isRemoteTranslogEnabled(); if (isReplicaRecoveryWithRemoteTranslog) { @@ -37,7 +38,8 @@ public static RecoverySourceHandler create( recoverySettings.getMaxConcurrentFileChunks(), recoverySettings.getMaxConcurrentOperations(), skipSegmentFilesTransfer, - cancellableThreads + cancellableThreads, + parentShard ); } else { return new LocalStorePeerRecoverySourceHandler( @@ -49,7 +51,8 @@ public static RecoverySourceHandler create( recoverySettings.getMaxConcurrentFileChunks(), recoverySettings.getMaxConcurrentOperations(), skipSegmentFilesTransfer, - cancellableThreads + cancellableThreads, + parentShard ); } } diff --git a/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java index d64f44936ba09..d40b717f54549 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java @@ -45,10 +45,11 @@ public RemoteStorePeerRecoverySourceHandler( int maxConcurrentFileChunks, int maxConcurrentOperations, boolean skipSegmentFilesTransfer, - CancellableThreads cancellableThreads + CancellableThreads cancellableThreads, + IndexShard parentShard ) { super(shard, recoveryTarget, threadPool, request, fileChunkSizeInBytes, maxConcurrentFileChunks, maxConcurrentOperations, - skipSegmentFilesTransfer, cancellableThreads); + skipSegmentFilesTransfer, cancellableThreads, parentShard); } @Override @@ -97,10 +98,11 @@ protected void innerRecoveryToTarget(ActionListener listener, prepareEngineStep.whenComplete(prepareEngineTime -> { logger.debug("prepareEngineStep completed"); assert Transports.assertNotTransportThread(this + "[phase2]"); + IndexShard primaryTracker = replicationTrackingShard(); RunUnderPrimaryPermit.run( - () -> shard.initiateTracking(request.targetAllocationId()), - shardId + " initiating tracking of " + request.targetAllocationId(), - shard, + () -> primaryTracker.initiateTracking(request.targetAllocationId()), + primaryTracker + " initiating tracking of " + request.targetAllocationId(), + primaryTracker, cancellableThreads, logger ); @@ -108,6 +110,8 @@ protected void innerRecoveryToTarget(ActionListener listener, sendSnapshotStep.onResponse(Collections.singletonList(new SendSnapshotResult(endingSeqNo, 0, TimeValue.ZERO, request.targetAllocationId()))); }, onFailure); + + finalizeStepAndCompleteFuture(startingSeqNo, sendSnapshotStep, sendFileStep, prepareEngineStep, new StepListener<>(), onFailure); } diff --git a/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoveryService.java b/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoveryService.java index 85226e7c15949..c15296d0ff5b5 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoveryService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoveryService.java @@ -14,9 +14,11 @@ import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.ClusterChangedEvent; import org.opensearch.cluster.ClusterStateListener; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.CheckedConsumer; import org.opensearch.common.Nullable; import org.opensearch.common.inject.Inject; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; @@ -28,6 +30,7 @@ import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.DelayRecoveryException; import org.opensearch.indices.recovery.RecoveryResponse; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoverySourceHandler; @@ -38,11 +41,13 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Consumer; public class InPlaceShardSplitRecoveryService extends AbstractLifecycleComponent implements IndexEventListener, ClusterStateListener { private static final Logger logger = LogManager.getLogger(InPlaceShardSplitRecoveryService.class); @@ -102,8 +107,8 @@ public void addAndStartRecovery(List recoveryContex DiscoveryNode node, IndexShard sourceShard, InPlaceShardSplitRecoveryListener replicationListener, - List shardIds, - StartRecoveryRequest request) { + StartRecoveryRequest request, + IndexMetadata indexMetadata) { if (ongoingRecoveries.isRecoveryOfShardOnGoing(sourceShard.shardId())) { return; } @@ -118,7 +123,7 @@ public void addAndStartRecovery(List recoveryContex replicationListener, request, timers, ongoingRecoveries, sourceShard); InPlaceShardSplitRecoverySourceHandler handler = ongoingRecoveries.addNewRecovery(sourceShard, node, - recoveryContexts, request, shardIds, childShardAllocationIds, replicationListener); + recoveryContexts, request, childShardAllocationIds, replicationListener, indexMetadata); logger.trace( "[{}] starting in-place recovery from [{}]", sourceShard.shardId().getIndex().getName(), @@ -128,8 +133,27 @@ public void addAndStartRecovery(List recoveryContex handler.recoverToTarget(recoveryResponseListener); } + public void addReplicaRecoveryAfterChildPrimariesSync(ShardId parentShardId, ActionListener listener) { + ongoingRecoveries.addReplicaRecovery(parentShardId, listener); + } + + public void startChildShards(ShardId parentShardId) { + synchronized (this) { + OngoingRecoveries.Recovery recovery = ongoingRecoveries.recoveries.get(parentShardId); + if (recovery != null) { + recovery.sourceHandler.performHandoff(); + } + } + } + public class OngoingRecoveries { private final Map recoveries = new HashMap<>(); + private final Set failedRecoveries = new HashSet<>(); + private final Consumer onSync = shardId -> { + synchronized (this) { + recoveries.get(shardId).notifyAllWaitingReplicaRecoveries(); + } + }; @Nullable private List> emptyListeners; @@ -138,6 +162,7 @@ private class Recovery { private final InPlaceShardSplitRecoveryTargetHandler targetHandler; private final InPlaceShardSplitRecoverySourceHandler sourceHandler; private final InPlaceShardSplitRecoveryListener replicationListener; + private final List> replicaRecoveryListeners = new ArrayList<>(); public Recovery(InPlaceShardSplitRecoveryTargetHandler targetHandler, InPlaceShardSplitRecoverySourceHandler sourceHandler, @@ -146,37 +171,51 @@ public Recovery(InPlaceShardSplitRecoveryTargetHandler targetHandler, this.sourceHandler = sourceHandler; this.replicationListener = replicationListener; } + + private synchronized void notifyAllWaitingReplicaRecoveries() { + replicaRecoveryListeners.forEach(listener -> { + listener.onResponse(null); + }); + } + + private synchronized void waitForPrimaryChildShardsSynced(ActionListener listener) { + replicaRecoveryListeners.add(listener); + } } private boolean isRecoveryOfShardOnGoing(ShardId shardId) { return recoveries.get(shardId) != null; } - synchronized InPlaceShardSplitRecoverySourceHandler addNewRecovery( + InPlaceShardSplitRecoverySourceHandler addNewRecovery( IndexShard sourceShard, DiscoveryNode node, List recoveryContexts, - StartRecoveryRequest request, List shardIds, Set childShardsAllocationIds, - InPlaceShardSplitRecoveryListener replicationListener) { - assert lifecycle.started(); - if (recoveries.containsKey(sourceShard.shardId())) { - throw new IllegalStateException("In-place shard recovery from shard " + sourceShard.shardId() + " already already in progress"); - } - CancellableThreads cancellableThreads = new CancellableThreads(); - List targetShards = new ArrayList<>(); - recoveryContexts.forEach(context -> targetShards.add(context.getIndexShard())); - - InPlaceShardSplitRecoveryTargetHandler targetHandler = createSplitTargetHandler(targetShards, - node, cancellableThreads, recoveryContexts, childShardsAllocationIds ,sourceShard); - RecoverySourceHandler delegatingRecoveryHandler = RecoverySourceHandlerFactory.create( - sourceShard, targetHandler, request, - recoverySettings, true, cancellableThreads); - - InPlaceShardSplitRecoverySourceHandler sourceHandler = createSourceHandler(sourceShard, - targetHandler, delegatingRecoveryHandler, request, cancellableThreads, recoveryContexts, - shardIds, childShardsAllocationIds); - - recoveries.put(sourceShard.shardId(), new Recovery(targetHandler, sourceHandler, replicationListener)); - sourceShard.recoveryStats().incCurrentAsSource(); - return sourceHandler; + StartRecoveryRequest request, Set childShardsAllocationIds, + InPlaceShardSplitRecoveryListener replicationListener, IndexMetadata indexMetadata + ) { + synchronized (this) { + assert lifecycle.started(); + if (recoveries.containsKey(sourceShard.shardId())) { + throw new IllegalStateException("In-place shard recovery from shard " + sourceShard.shardId() + " already already in progress"); + } + failedRecoveries.remove(sourceShard.shardId()); + CancellableThreads cancellableThreads = new CancellableThreads(); + List targetShards = new ArrayList<>(); + recoveryContexts.forEach(context -> targetShards.add(context.getIndexShard())); + + InPlaceShardSplitRecoveryTargetHandler targetHandler = createSplitTargetHandler(targetShards, + node, cancellableThreads, recoveryContexts, childShardsAllocationIds, sourceShard); + RecoverySourceHandler delegatingRecoveryHandler = RecoverySourceHandlerFactory.create(sourceShard, + targetHandler, request, recoverySettings, true, cancellableThreads, null); + + InPlaceShardSplitRecoverySourceHandler sourceHandler = createSourceHandler(sourceShard, + targetHandler, delegatingRecoveryHandler, request, cancellableThreads, recoveryContexts, + childShardsAllocationIds, replicationListener, indexMetadata); + + recoveries.put(sourceShard.shardId(), new Recovery(targetHandler, sourceHandler, replicationListener)); + sourceShard.recoveryStats().incCurrentAsSource(); + logger.info("Adding child primary recovery on node " + indicesService.clusterService().localNode().getName()); + return sourceHandler; + } } protected InPlaceShardSplitRecoveryTargetHandler createSplitTargetHandler( @@ -191,6 +230,29 @@ protected InPlaceShardSplitRecoveryTargetHandler createSplitTargetHandler( node, cancellableThreads, recoveryContexts, childShardsAllocationIds ,sourceShard); } + public void addReplicaRecovery(ShardId parentShardId, ActionListener listener) { + synchronized (this) { + if (failedRecoveries.contains(parentShardId)) { + listener.onFailure(new InPlaceShardsRecoveryFailedException(parentShardId)); + return; + } + + Recovery recovery = recoveries.get(parentShardId); + if (recovery == null) { + logger.info("Delaying replica on node " + indicesService.clusterService().localNode().getName()); + throw new DelayRecoveryException("parent shard [" + parentShardId + "] is not yet added for split recovery"); + } + + if (recovery.sourceHandler.isRecoveryStateInSync()) { + logger.info("Parent in sync on node " + indicesService.clusterService().localNode().getName()); + listener.onResponse(null); + return; + } + + recovery.waitForPrimaryChildShardsSynced(listener); + } + } + protected InPlaceShardSplitRecoverySourceHandler createSourceHandler( IndexShard sourceShard, InPlaceShardSplitRecoveryTargetHandler targetHandler, @@ -198,55 +260,64 @@ protected InPlaceShardSplitRecoverySourceHandler createSourceHandler( StartRecoveryRequest request, CancellableThreads cancellableThreads, List recoveryContexts, - List shardIds, - Set childShardsAllocationIds + Set childShardsAllocationIds, + InPlaceShardSplitRecoveryListener replicationListener, + IndexMetadata indexMetadata ) { - return new InPlaceShardSplitRecoverySourceHandler(sourceShard, - targetHandler, delegatingRecoveryHandler, request, - Math.toIntExact(recoverySettings.getChunkSize().getBytes()), + return new InPlaceShardSplitRecoverySourceHandler(sourceShard, targetHandler, delegatingRecoveryHandler, + request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), recoverySettings.getMaxConcurrentFileChunks(), recoverySettings.getMaxConcurrentOperations(), - cancellableThreads, recoveryContexts, shardIds, childShardsAllocationIds); + cancellableThreads, recoveryContexts, childShardsAllocationIds, replicationListener, indexMetadata, onSync); } - synchronized void remove(InPlaceShardSplitRecoverySourceHandler sourceHandler) { - sourceHandler.getSourceShard().recoveryStats().decCurrentAsSource(); - if (recoveries.isEmpty()) { - if (emptyListeners != null) { - final List> onEmptyListeners = emptyListeners; - emptyListeners = null; - ActionListener.onResponse(onEmptyListeners, null); + void remove(InPlaceShardSplitRecoverySourceHandler sourceHandler) { + synchronized (this) { + sourceHandler.getSourceShard().recoveryStats().decCurrentAsSource(); + if (recoveries.isEmpty()) { + if (emptyListeners != null) { + final List> onEmptyListeners = emptyListeners; + emptyListeners = null; + ActionListener.onResponse(onEmptyListeners, null); + } } } } - public synchronized void markAsDone(IndexShard sourceShard) { - Recovery removed = recoveries.remove(sourceShard.shardId()); - if (removed != null) { - assert sourceShard.routingEntry().splitting(); - remove(removed.sourceHandler); - removed.targetHandler.onDone(); - removed.replicationListener.onDone(null); + public void markAsDone(IndexShard sourceShard) { + synchronized (this) { + Recovery removed = recoveries.remove(sourceShard.shardId()); + if (removed != null) { + assert sourceShard.routingEntry().splitting(); + remove(removed.sourceHandler); + removed.targetHandler.onDone(); + removed.replicationListener.onDone(null); + } } } - public synchronized void fail(IndexShard sourceShard, ReplicationFailedException ex, boolean sendShardFailure) { - Recovery removed = recoveries.remove(sourceShard.shardId()); - if (removed != null) { - remove(removed.sourceHandler); - removed.replicationListener.onFailure(null, ex, sendShardFailure); + public void fail(IndexShard sourceShard, ReplicationFailedException ex, boolean sendShardFailure) { + synchronized (this) { + Recovery removed = recoveries.remove(sourceShard.shardId()); + if (removed != null) { + remove(removed.sourceHandler); + removed.replicationListener.onFailure(null, ex, sendShardFailure); + failedRecoveries.add(sourceShard.shardId()); + } } } - synchronized void cancel(IndexShard shard, String reason) { - try { - ShardId sourceShardId = getSplittingSourceShardId(shard); - if (sourceShardId != null && recoveries.containsKey(sourceShardId)) { - recoveries.get(sourceShardId).sourceHandler.cancel(reason); + void cancel(IndexShard shard, String reason) { + synchronized (this) { + try { + ShardId sourceShardId = getSplittingSourceShardId(shard); + if (sourceShardId != null && recoveries.containsKey(sourceShardId)) { + recoveries.get(sourceShardId).sourceHandler.cancel(reason); + } + } catch (Exception ex) { + throw new OpenSearchException(ex); + } finally { + shard.recoveryStats().decCurrentAsSource(); } - } catch (Exception ex) { - throw new OpenSearchException(ex); - } finally { - shard.recoveryStats().decCurrentAsSource(); } } diff --git a/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoverySourceHandler.java index 35e29215d7c7e..47574e485d76f 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoverySourceHandler.java +++ b/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoverySourceHandler.java @@ -13,7 +13,9 @@ import org.apache.lucene.index.IndexCommit; import org.opensearch.action.StepListener; import org.opensearch.action.admin.indices.flush.FlushRequest; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.SetOnce; +import org.opensearch.common.StopWatch; import org.opensearch.common.collect.Tuple; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.lease.Releasable; @@ -24,10 +26,11 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.engine.Engine; +import org.opensearch.index.seqno.ReplicationTracker; +import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.seqno.RetentionLeases; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; @@ -46,10 +49,10 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.IntSupplier; @@ -59,10 +62,14 @@ public class InPlaceShardSplitRecoverySourceHandler extends RecoverySourceHandle private final InPlaceShardSplitRecoveryTargetHandler recoveryTarget; private final IndexShard sourceShard; private final RecoverySourceHandler delegatingRecoveryHandler; - private final List shardIds; private final Set childShardsAllocationIds; private final SetOnce splitCommitMetadata = new SetOnce<>(); private final Logger logger; + private final InPlaceShardSplitRecoveryListener replicationListener; + private final IndexMetadata indexMetadata; + private volatile boolean inSync = false; + private final Consumer onSync; + private volatile Runnable finalizer; public InPlaceShardSplitRecoverySourceHandler( IndexShard sourceShard, @@ -74,21 +81,27 @@ public InPlaceShardSplitRecoverySourceHandler( int maxConcurrentOperations, CancellableThreads cancellableThreads, List recoveryContexts, - List shardIds, - Set childShardsAllocationIds + Set childShardsAllocationIds, + InPlaceShardSplitRecoveryListener replicationListener, + IndexMetadata indexMetadata, + Consumer onSync ) { super(sourceShard, recoveryTarget, sourceShard.getThreadPool(), request, fileChunkSizeInBytes, maxConcurrentFileChunks, - maxConcurrentOperations, true, cancellableThreads); + maxConcurrentOperations, true, cancellableThreads, sourceShard); + List childShardIds = new ArrayList<>(); + recoveryContexts.forEach(context -> childShardIds.add(context.getIndexShard().shardId())); this.logger = Loggers.getLogger(InPlaceShardSplitRecoverySourceHandler.class, request.shardId(), - "splitting to " + shardIds); + "splitting to " + childShardIds); this.resources.add(recoveryTarget); this.recoveryContexts = recoveryContexts; this.sourceShard = sourceShard; this.delegatingRecoveryHandler = delegatingRecoveryHandler; - this.shardIds = shardIds; this.childShardsAllocationIds = childShardsAllocationIds; this.recoveryTarget = recoveryTarget; + this.replicationListener = replicationListener; + this.indexMetadata = indexMetadata; + this.onSync = onSync; recoveryTarget.initStoreAcquirer((requestStore) -> { Releasable releasable = acquireStore(requestStore); @@ -159,14 +172,18 @@ protected void innerRecoveryToTarget(ActionListener listener, prepareEngine(sendFileStep, prepareEngineStep, RecoveryState.Translog.UNKNOWN, onFailure); prepareEngineStep.whenComplete(prepareEngineTime -> { + addRetentionLeases(startingSeqNo); logger.info("prepareEngineStep completed"); assert Transports.assertNotTransportThread(this + "[phase2]"); initiateTracking(); - final long endingSeqNo = sourceShard.seqNoStats().getMaxSeqNo(); // Syncing here because sequence number can be greater than local checkpoint and operations may not yet be // present in translog. sourceShard.sync(); + // Flush because one or more operations in the provided range may still be pending to be indexed into lucene + // and therefore, may not be available yet in translog. This is a best effort to ensure all operations within + // this range are indexed and therefore, available in translog. + sourceShard.flush(new FlushRequest().waitIfOngoing(true).force(true)); final Translog.Snapshot phase2Snapshot; if (startingSeqNo > endingSeqNo) { phase2Snapshot = new EmptySnapshot(); @@ -185,7 +202,7 @@ protected void innerRecoveryToTarget(ActionListener listener, phase2Snapshot, sourceShard.getMaxSeenAutoIdTimestamp(), sourceShard.getMaxSeqNoOfUpdatesOrDeletes(), - sourceShard.getRetentionLeases(), + RetentionLeases.EMPTY, mappingVersionOnPrimary, sendSnapshotStep ); @@ -200,6 +217,21 @@ protected void innerRecoveryToTarget(ActionListener listener, finalizeStepAndCompleteFuture(startingSeqNo, sendSnapshotStep, sendFileStepWithEmptyResult(), prepareEngineStep, finalizeStep, onFailure); } + private void addRetentionLeases(long startingSeqNo) { + recoveryContexts.forEach(context -> { + RetentionLease primaryRentetionLease = new RetentionLease( + ReplicationTracker.getPeerRecoveryRetentionLeaseId(shard.routingEntry()), + startingSeqNo, + shard.getThreadPool().absoluteTimeInMillis(), + ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE + ); + Collection leases = new ArrayList<>(); + leases.add(primaryRentetionLease); + context.getIndexShard().updateRetentionLeasesOnChildPrimary(new RetentionLeases( + sourceShard.getOperationPrimaryTerm(), 1, leases)); + }); + } + private GatedCloseable acquireCommitAndFetchMetadata(GatedCloseable translogRetentionLock) throws IOException { // Make sure that all operations before acquired translog generation are present in the last commit. // In remote store replication mode refreshed but not flushed ops are also trimmed from translog and hence, @@ -318,6 +350,7 @@ protected void updateGlobalCheckpointForShard(long globalCheckpoint) { @Override protected void relocateShard(Runnable forceSegRepRunnable) throws InterruptedException { + // Relocation is splitting in the current context where parent shard will cease to exist. shard.relocated(childShardsAllocationIds, recoveryTarget::handoffPrimaryContext, forceSegRepRunnable); recoveryTarget.flushOnAllChildShards(); } @@ -413,17 +446,25 @@ protected List createSnapshotResult(OperationBatchSender sen protected class AllShardsOperationBatchSender extends OperationBatchSender { private final Map targetLocalCheckpoints = new HashMap<>(); + private final Closeable onClose; protected AllShardsOperationBatchSender( long startingSeqNo, long endingSeqNo, Translog.Snapshot snapshot, long maxSeenAutoIdTimestamp, long maxSeqNoOfUpdatesOrDeletes, RetentionLeases retentionLeases, long mappingVersion, ActionListener listener) { super(startingSeqNo, endingSeqNo, snapshot, maxSeenAutoIdTimestamp, - maxSeqNoOfUpdatesOrDeletes, retentionLeases, mappingVersion, listener); + maxSeqNoOfUpdatesOrDeletes, retentionLeases, mappingVersion, listener, false); childShardsAllocationIds.forEach(childShardsAllocationId -> { targetLocalCheckpoints.put(childShardsAllocationId, new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED)); }); + + AtomicInteger closeCounter = new AtomicInteger(childShardsAllocationIds.size()); + onClose = () -> { + if (closeCounter.decrementAndGet() == 0) { + snapshot.close(); + } + }; } @Override @@ -445,6 +486,35 @@ protected void executeChunkRequest(OperationChunkRequest request, ActionListener }) ); } + + @Override + public void close() throws IOException { + onClose.close(); + } + } + + public boolean isRecoveryStateInSync() { + return inSync; + } + + protected void finalizeRecovery(StopWatch stopWatch, long trimAboveSeqNo, ActionListener listener) { + if (indexMetadata.getNumberOfReplicas() == 0) { + super.finalizeRecovery(stopWatch, trimAboveSeqNo, listener); + return; + } + Set inSyncAllocationIds = sourceShard.getReplicationGroup().getInSyncAllocationIds(); + recoveryContexts.forEach(context -> { + assert inSyncAllocationIds.contains(context.getIndexShard().routingEntry().allocationId().getId()); + }); + + finalizer = () -> super.finalizeRecovery(new StopWatch().start(), trimAboveSeqNo, listener); + inSync = true; + onSync.accept(sourceShard.shardId()); + } + + public void performHandoff() { + assert finalizer != null; + finalizer.run(); } private void cleanUpMaybeRemoteOnFinalize() { diff --git a/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoveryTargetHandler.java b/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoveryTargetHandler.java index 7e39f973d103c..7ed0ff8943a1b 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoveryTargetHandler.java +++ b/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardSplitRecoveryTargetHandler.java @@ -21,6 +21,7 @@ import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.support.GroupedActionListener; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.SetOnce; import org.opensearch.common.collect.Tuple; @@ -54,6 +55,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -178,7 +180,7 @@ public void indexTranslogOperationsOnShards(List operations, * processed only on one of the child shards and other child shards treat it as a NoOp where only local checkpoint * is advanced. In this case local checkpoint is also the global checkpoint since we are creating a new shard * and hence a new replication group. In scenario where one or more of the child shards are relocated before - * next flush gets triggered, translog replay of operations from snapshot taken from lucene in these peer + * next flush gets triggered, translog replay of operations from snapshot in these peer * recoveries will not have no-ops and therefore, peer recovery will fail while waiting for target shard to * catch up to global checkpoint. So, to make sure that operations till global checkpoint are available, we * will need to trigger a flush to create a new commit on all child shards. @@ -207,73 +209,8 @@ public BatchOperationsResult(long checkpoint, String allocationId) { } } -// public void blockOpsAndForceSegmentFileSync() { -// recoveryContexts.forEach(context -> { -// cancellableThreads.checkForCancel(); -// CheckedRunnable forceSegmentSync = () -> internalForceSegmentSync(context.getIndexShard()); -// try { -// context.getIndexShard().blockOperationsAndExecute(forceSegmentSync); -// } catch (Exception ex) { -// throw new RuntimeException(ex); -// } -// }); -// } - -// public void internalForceSegmentSync(IndexShard childShard) { -// SegmentReplicationTarget segmentReplicationTarget = new SegmentReplicationTarget( -// childShard, -// sourceShard, -// childShard.getLatestReplicationCheckpoint(), -// segRepFactory.get(sourceShard), -// null -// ); -// -// CountDownLatch latch = new CountDownLatch(1); -// AtomicReference replicationFailure = new AtomicReference<>(); -// LatchedActionListener latchedActionListener = new LatchedActionListener<>( -// ActionListener.wrap(res -> childShard.resetToWriteableEngine(), -// replicationFailure::set), latch -// ); -// -// segmentReplicationTarget.startReplication(latchedActionListener); -// try { -// latch.await(); -// } catch (InterruptedException e) { -// throw new RuntimeException(e); -// } -// -// if (replicationFailure.get() != null) { -// throw new RuntimeException(replicationFailure.get()); -// } -// } - -// private static class RemoteReplicationSource extends RemoteStoreReplicationSource { -// public RemoteReplicationSource(IndexShard sourceShard) { -// super(sourceShard); -// } -// -// @Override -// protected void syncFromRemote( -// List filesToFetch, -// IndexShard targetShard, -// BiConsumer fileProgressTracker, -// ActionListener listener, -// List toSyncSegmentFiles -// ) throws IOException { -// -// } -// } - @Override public void forceSegmentFileSync() { -// if (sourceShard.indexSettings().isSegRepEnabled() == false) { -// return; -// } -// -// recoveryContexts.forEach(context -> { -// cancellableThreads.checkForCancel(); -// internalForceSegmentSync(context.getIndexShard()); -// }); } @Override @@ -290,7 +227,32 @@ public void finalizeRecovery(long globalCheckpoint, long trimAboveSeqNo, ActionL @Override public void handoffPrimaryContext(ReplicationTracker.PrimaryContext primaryContext) { - recoveryTargets.values().forEach(recoveryTarget -> recoveryTarget.handoffPrimaryContext(primaryContext)); + ShardRouting[] childShards = sourceShard.routingEntry().getRecoveringChildShards(); + + Map> shardIdToAllocationIds = new HashMap<>(); + for (ShardRouting childShard : childShards) { + shardIdToAllocationIds.putIfAbsent(childShard.shardId().id(), new HashSet<>()); + shardIdToAllocationIds.get(childShard.shardId().id()).add(childShard.allocationId().getId()); + } + + recoveryTargets.forEach((shardId, recoveryTarget) -> { + Set childShardsAllocIds = shardIdToAllocationIds.get(shardId.id()); + Map checkpointStates = new HashMap<>(); + for (String allocationId : childShardsAllocIds) { + if (primaryContext.getCheckpointStates().get(allocationId) == null) { + throw new IllegalStateException("Allocation ID " + allocationId + + " not found in synced checkpoint states of parent shard." + sourceShard.shardId()); + } + checkpointStates.put(allocationId, primaryContext.getCheckpointStates().get(allocationId)); + } + ReplicationTracker.PrimaryContext childPrimaryContext = new ReplicationTracker.PrimaryContext( + primaryContext.clusterStateVersion(), + checkpointStates, + primaryContext.getRoutingTable() + ); + + recoveryTarget.handoffPrimaryContext(childPrimaryContext); + }); } @Override @@ -394,7 +356,6 @@ public Directory syncLocalDirectory(Store store, StoreFileMetadata[] files, Inde private void split(long localCheckpoint, long maxSeqNo, long maxUnsafeAutoIdTimestamp, Directory childShardDirectory, InPlaceShardRecoveryContext context) throws IOException { Tuple addIndexSplitDirectory = new Tuple<>(false, childShardDirectory); - System.out.println("Creating commits with max seq number: " + maxSeqNo); StoreRecovery.addIndices( context.getRecoveryState().getIndex(), diff --git a/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardsRecoveryFailedException.java b/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardsRecoveryFailedException.java index 27ce99c46f816..4edbaa2b582bc 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardsRecoveryFailedException.java +++ b/server/src/main/java/org/opensearch/indices/recovery/inplacesplit/InPlaceShardsRecoveryFailedException.java @@ -23,14 +23,8 @@ @ExperimentalApi public class InPlaceShardsRecoveryFailedException extends ReplicationFailedException { - public InPlaceShardsRecoveryFailedException(ShardId sourceShardId, List recoveryContexts, Throwable cause) { - super("Source shard : " + sourceShardId.id() + - ", child shards: " + Arrays.toString(recoveryContexts.stream().map(context -> - context.getIndexShard().shardId().id()).toArray()) - + ", index: " + sourceShardId.getIndexName() - + ": In-place recovery of shards failed. ", - cause - ); + public InPlaceShardsRecoveryFailedException(ShardId sourceShardId) { + super("Split of shard : " + sourceShardId + " failed."); } } diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java index de7a9d63e9109..56b0c819ba61e 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -170,7 +170,7 @@ public void clusterChanged(ClusterChangedEvent event) { for (IndexService indexService : indicesService) { if (indexService.getIndexSettings().isSegRepEnabled() && event.indexRoutingTableChanged(indexService.index().getName())) { for (IndexShard shard : indexService) { - if (shard.routingEntry().primary() == false) { + if (shard.routingEntry().primary() == false && shard.routingEntry().isSplitTarget() == false) { // for this shard look up its primary routing, if it has completed a relocation trigger replication final String previousNode = event.previousState() .routingTable() diff --git a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTimer.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTimer.java index d884e1676f2be..4e31f884b3826 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTimer.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTimer.java @@ -48,6 +48,9 @@ public synchronized void writeTo(StreamOutput out) throws IOException { } public synchronized void start() { + if (startTime != 0) { + System.out.println(); + } assert startTime == 0 : "already started"; startTime = System.currentTimeMillis(); startNanoTime = System.nanoTime(); diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index bda639f6e822e..d00a71de4ddfe 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -1257,8 +1257,11 @@ protected Node( b.bind(Discovery.class).toInstance(discoveryModule.getDiscovery()); { processRecoverySettings(settingsModule.getClusterSettings(), recoverySettings); + InPlaceShardSplitRecoveryService splitRecoveryService = newInPlaceShardSplitRecoveryService( + indicesService, recoverySettings); + b.bind(InPlaceShardSplitRecoveryService.class).toInstance(splitRecoveryService); b.bind(PeerRecoverySourceService.class) - .toInstance(new PeerRecoverySourceService(transportService, indicesService, recoverySettings)); + .toInstance(new PeerRecoverySourceService(transportService, indicesService, recoverySettings, splitRecoveryService)); b.bind(PeerRecoveryTargetService.class) .toInstance(new PeerRecoveryTargetService(threadPool, transportService, recoverySettings, clusterService)); b.bind(SegmentReplicationTargetService.class) @@ -1274,8 +1277,7 @@ protected Node( ); b.bind(SegmentReplicationSourceService.class) .toInstance(new SegmentReplicationSourceService(indicesService, transportService, recoverySettings)); - b.bind(InPlaceShardSplitRecoveryService.class) - .toInstance(newInPlaceShardSplitRecoveryService(indicesService, recoverySettings)); + } b.bind(HttpServerTransport.class).toInstance(httpServerTransport); pluginComponents.stream().forEach(p -> b.bind((Class) p.getClass()).toInstance(p)); diff --git a/server/src/test/java/org/opensearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java b/server/src/test/java/org/opensearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java index 88644c3293eeb..899f5a7bbf781 100644 --- a/server/src/test/java/org/opensearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java +++ b/server/src/test/java/org/opensearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java @@ -181,7 +181,9 @@ public void testIllegalShardFailureRequests() throws Exception { failingTask.message, failingTask.failure, randomBoolean(), - failingTask.childShardsFailedEvent + null, + null, + null ) ); } @@ -221,6 +223,8 @@ public void testMarkAsStaleWhenFailingShard() throws Exception { "dummy", null, false, + null, + null, null ); ClusterState appliedState = executor.execute(clusterState, Collections.singletonList(failShardOnly)).resultingState; @@ -236,6 +240,8 @@ public void testMarkAsStaleWhenFailingShard() throws Exception { "dummy", null, true, + null, + null, null ); ClusterState appliedState = executor.execute(clusterState, Collections.singletonList(failAndMarkAsStale)).resultingState; @@ -297,7 +303,9 @@ private List createNonExistentShards(ClusterS existingShard.message, existingShard.failure, randomBoolean(), - existingShard.childShardsFailedEvent + existingShard.splitFailed, + existingShard.parentShardId, + existingShard.parentAllocationId ) ); } @@ -312,6 +320,8 @@ private List createNonExistentShards(ClusterS reason, new CorruptIndexException("simulated", nonExistentIndexUUID), randomBoolean(), + null, + null, null ) ) @@ -403,6 +413,8 @@ private static List toTasks( message, new CorruptIndexException("simulated", indexUUID), randomBoolean(), + null, + null, null ) ) diff --git a/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java b/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java index 6d037a3c8be8b..8b5a6a0195153 100644 --- a/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java +++ b/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java @@ -95,7 +95,7 @@ public void testEmptyTaskListProducesSameClusterState() throws Exception { public void testNonExistentIndexMarkedAsSuccessful() throws Exception { final ClusterState clusterState = stateWithNoShard(); - final StartedShardEntry entry = new StartedShardEntry(new ShardId("test", "_na", 0), "aId", randomNonNegativeLong(), "test", null); + final StartedShardEntry entry = new StartedShardEntry(new ShardId("test", "_na", 0), "aId", randomNonNegativeLong(), "test", null, null, null); final ClusterStateTaskExecutor.ClusterTasksResult result = executeTasks(clusterState, singletonList(entry)); assertSame(clusterState, result.resultingState); @@ -113,11 +113,11 @@ public void testNonExistentShardsAreMarkedAsSuccessful() throws Exception { // Existent shard id but different allocation id IntStream.range(0, randomIntBetween(1, 5)) .mapToObj( - i -> new StartedShardEntry(new ShardId(indexMetadata.getIndex(), 0), String.valueOf(i), 0L, "allocation id", null) + i -> new StartedShardEntry(new ShardId(indexMetadata.getIndex(), 0), String.valueOf(i), 0L, "allocation id", null, null, null) ), // Non existent shard id IntStream.range(1, randomIntBetween(2, 5)) - .mapToObj(i -> new StartedShardEntry(new ShardId(indexMetadata.getIndex(), i), String.valueOf(i), 0L, "shard id", null)) + .mapToObj(i -> new StartedShardEntry(new ShardId(indexMetadata.getIndex(), i), String.valueOf(i), 0L, "shard id", null, null, null)) ).collect(Collectors.toList()); @@ -145,7 +145,7 @@ public void testNonInitializingShardAreMarkedAsSuccessful() throws Exception { allocationId = shardRoutingTable.replicaShards().iterator().next().allocationId().getId(); } final long primaryTerm = indexMetadata.primaryTerm(shardId.id()); - return new StartedShardEntry(shardId, allocationId, primaryTerm, "test", null); + return new StartedShardEntry(shardId, allocationId, primaryTerm, "test", null, null, null); }).collect(Collectors.toList()); final ClusterStateTaskExecutor.ClusterTasksResult result = executeTasks(clusterState, tasks); @@ -168,11 +168,11 @@ public void testStartedShards() throws Exception { final String primaryAllocationId = primaryShard.allocationId().getId(); final List tasks = new ArrayList<>(); - tasks.add(new StartedShardEntry(shardId, primaryAllocationId, primaryTerm, "test", null)); + tasks.add(new StartedShardEntry(shardId, primaryAllocationId, primaryTerm, "test", null, null, null)); if (randomBoolean()) { final ShardRouting replicaShard = clusterState.routingTable().shardRoutingTable(shardId).replicaShards().iterator().next(); final String replicaAllocationId = replicaShard.allocationId().getId(); - tasks.add(new StartedShardEntry(shardId, replicaAllocationId, primaryTerm, "test", null)); + tasks.add(new StartedShardEntry(shardId, replicaAllocationId, primaryTerm, "test", null, null, null)); } final ClusterStateTaskExecutor.ClusterTasksResult result = executeTasks(clusterState, tasks); assertNotSame(clusterState, result.resultingState); @@ -197,7 +197,7 @@ public void testDuplicateStartsAreOkay() throws Exception { final long primaryTerm = indexMetadata.primaryTerm(shardId.id()); final List tasks = IntStream.range(0, randomIntBetween(2, 10)) - .mapToObj(i -> new StartedShardEntry(shardId, allocationId, primaryTerm, "test", null)) + .mapToObj(i -> new StartedShardEntry(shardId, allocationId, primaryTerm, "test", null, null, null)) .collect(Collectors.toList()); final ClusterStateTaskExecutor.ClusterTasksResult result = executeTasks(clusterState, tasks); @@ -233,6 +233,8 @@ public void testPrimaryTermsMismatch() throws Exception { primaryAllocationId, primaryTerm - 1, "primary terms does not match on primary", + null, + null, null ); @@ -251,6 +253,8 @@ public void testPrimaryTermsMismatch() throws Exception { primaryAllocationId, primaryTerm, "primary terms match on primary", + null, + null, null ); @@ -274,7 +278,7 @@ public void testPrimaryTermsMismatch() throws Exception { .allocationId() .getId(); - final StartedShardEntry task = new StartedShardEntry(shardId, replicaAllocationId, replicaPrimaryTerm, "test on replica", null); + final StartedShardEntry task = new StartedShardEntry(shardId, replicaAllocationId, replicaPrimaryTerm, "test on replica", null, null, null); final ClusterStateTaskExecutor.ClusterTasksResult result = executeTasks(clusterState, singletonList(task)); assertNotSame(clusterState, result.resultingState); diff --git a/server/src/test/java/org/opensearch/cluster/action/shard/ShardStateActionTests.java b/server/src/test/java/org/opensearch/cluster/action/shard/ShardStateActionTests.java index 4b64ca5f82154..110037f68e250 100644 --- a/server/src/test/java/org/opensearch/cluster/action/shard/ShardStateActionTests.java +++ b/server/src/test/java/org/opensearch/cluster/action/shard/ShardStateActionTests.java @@ -543,11 +543,11 @@ public void testFailedShardEntrySerialization() throws Exception { final long primaryTerm = randomIntBetween(0, 100); final String message = randomRealisticUnicodeOfCodepointLengthBetween(10, 100); final Exception failure = randomBoolean() ? null : getSimulatedFailure(); - final Boolean childShardFailedEvent = randomBoolean() ? null : true; + final String[] childShardAllocationIds = new String[]{randomAlphaOfLength(50)}; final boolean markAsStale = randomBoolean(); final Version version = randomFrom(randomCompatibleVersion(random(), Version.CURRENT)); - final FailedShardEntry failedShardEntry = new FailedShardEntry(shardId, allocationId, primaryTerm, message, failure, markAsStale, childShardFailedEvent); + final FailedShardEntry failedShardEntry = new FailedShardEntry(shardId, allocationId, primaryTerm, message, failure, markAsStale, null, null, null); try (StreamInput in = serialize(failedShardEntry, version).streamInput()) { in.setVersion(version); final FailedShardEntry deserialized = new FailedShardEntry(in); @@ -574,7 +574,7 @@ public void testStartedShardEntrySerialization() throws Exception { final String message = randomRealisticUnicodeOfCodepointLengthBetween(10, 100); final Version version = randomFrom(randomCompatibleVersion(random(), Version.CURRENT)); - try (StreamInput in = serialize(new StartedShardEntry(shardId, allocationId, primaryTerm, message, null), version).streamInput()) { + try (StreamInput in = serialize(new StartedShardEntry(shardId, allocationId, primaryTerm, message, null, null, null), version).streamInput()) { in.setVersion(version); final StartedShardEntry deserialized = new StartedShardEntry(in); assertThat(deserialized.shardId, equalTo(shardId)); diff --git a/server/src/test/java/org/opensearch/cluster/routing/ShardRoutingTests.java b/server/src/test/java/org/opensearch/cluster/routing/ShardRoutingTests.java index be3c93cc9d98f..1fd3223a9fde5 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/ShardRoutingTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/ShardRoutingTests.java @@ -162,9 +162,8 @@ public void testEqualsIgnoringVersion() { otherRouting.unassignedInfo(), otherRouting.allocationId(), otherRouting.getExpectedShardSize(), - otherRouting.getShardRange(), - otherRouting.getParentShardId(), - otherRouting.getRecoveringChildShardRanges() + otherRouting.getRecoveringChildShards(), + otherRouting.getParentShardId() ); break; case 1: @@ -179,9 +178,8 @@ public void testEqualsIgnoringVersion() { otherRouting.unassignedInfo(), otherRouting.allocationId(), otherRouting.getExpectedShardSize(), - otherRouting.getShardRange(), - otherRouting.getParentShardId(), - otherRouting.getRecoveringChildShardRanges() + otherRouting.getRecoveringChildShards(), + otherRouting.getParentShardId() ); break; case 2: @@ -199,9 +197,8 @@ public void testEqualsIgnoringVersion() { otherRouting.unassignedInfo(), otherRouting.allocationId(), otherRouting.getExpectedShardSize(), - otherRouting.getShardRange(), - otherRouting.getParentShardId(), - otherRouting.getRecoveringChildShardRanges() + otherRouting.getRecoveringChildShards(), + otherRouting.getParentShardId() ); } break; @@ -220,9 +217,8 @@ public void testEqualsIgnoringVersion() { otherRouting.unassignedInfo(), otherRouting.allocationId(), otherRouting.getExpectedShardSize(), - otherRouting.getShardRange(), - otherRouting.getParentShardId(), - otherRouting.getRecoveringChildShardRanges() + otherRouting.getRecoveringChildShards(), + otherRouting.getParentShardId() ); } break; @@ -246,9 +242,8 @@ public void testEqualsIgnoringVersion() { otherRouting.unassignedInfo(), otherRouting.allocationId(), otherRouting.getExpectedShardSize(), - otherRouting.getShardRange(), - otherRouting.getParentShardId(), - otherRouting.getRecoveringChildShardRanges() + otherRouting.getRecoveringChildShards(), + otherRouting.getParentShardId() ); } break; diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/InSyncAllocationIdTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/InSyncAllocationIdTests.java index 764110faed306..6a44feca7eaab 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/InSyncAllocationIdTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/InSyncAllocationIdTests.java @@ -195,7 +195,7 @@ public void testDeadNodesBeforeReplicaFailed() throws Exception { clusterState = failedClusterStateTaskExecutor.execute( clusterState, Arrays.asList( - new FailedShardEntry(shardRoutingTable.shardId(), replicaShard.allocationId().getId(), primaryTerm, "dummy", null, true, null) + new FailedShardEntry(shardRoutingTable.shardId(), replicaShard.allocationId().getId(), primaryTerm, "dummy", null, true, null, null, null) ) ).resultingState; @@ -219,9 +219,9 @@ public void testPrimaryFailureBatchedWithReplicaFailure() throws Exception { long primaryTerm = clusterState.metadata().index("test").primaryTerm(0); List failureEntries = new ArrayList<>(); - failureEntries.add(new FailedShardEntry(shardRoutingTable.shardId(), primaryShard.allocationId().getId(), 0L, "dummy", null, true, null)); + failureEntries.add(new FailedShardEntry(shardRoutingTable.shardId(), primaryShard.allocationId().getId(), 0L, "dummy", null, true, null, null, null)); failureEntries.add( - new FailedShardEntry(shardRoutingTable.shardId(), replicaShard.allocationId().getId(), primaryTerm, "dummy", null, true, null) + new FailedShardEntry(shardRoutingTable.shardId(), replicaShard.allocationId().getId(), primaryTerm, "dummy", null, true, null, null, null) ); Collections.shuffle(failureEntries, random()); logger.info("Failing {}", failureEntries); @@ -369,7 +369,7 @@ public void testPrimaryAllocationIdNotRemovedFromInSyncSetWhenNoFailOver() throw clusterState = failedClusterStateTaskExecutor.execute( clusterState, Collections.singletonList( - new FailedShardEntry(shardRoutingTable.shardId(), primaryShard.allocationId().getId(), 0L, "dummy", null, true, null) + new FailedShardEntry(shardRoutingTable.shardId(), primaryShard.allocationId().getId(), 0L, "dummy", null, true, null, null, null) ) ).resultingState; diff --git a/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java b/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java index 7079b1e3e3a01..e11c778a40c6e 100644 --- a/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -278,7 +278,23 @@ public MockIndexShard createShard( } @Override - public void createChildShardsForSplit(List shardRoutings, ShardId parentShardId, InPlaceShardSplitRecoveryService inPlaceShardSplitRecoveryService, InPlaceShardSplitRecoveryListener recoveryListener, Consumer onShardFailure, DiscoveryNode node, Consumer globalCheckpointSyncer, RetentionLeaseSyncer retentionLeaseSyncer, SegmentReplicationCheckpointPublisher checkpointPublisher, RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, StartRecoveryRequest request) throws IOException { + public void createChildShardsForSplit( + List shardRoutings, + ShardId parentShardId, + InPlaceShardSplitRecoveryService inPlaceShardSplitRecoveryService, + InPlaceShardSplitRecoveryListener recoveryListener, + Consumer onShardFailure, + DiscoveryNode node, + Consumer globalCheckpointSyncer, + RetentionLeaseSyncer retentionLeaseSyncer, + SegmentReplicationCheckpointPublisher checkpointPublisher, + RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, + StartRecoveryRequest request) throws IOException { + + } + + @Override + public void moveChildShardsToStarted(ShardId parentShardId, InPlaceShardSplitRecoveryService inPlaceShardSplitRecoveryService) { } diff --git a/server/src/test/java/org/opensearch/indices/cluster/ClusterStateChanges.java b/server/src/test/java/org/opensearch/indices/cluster/ClusterStateChanges.java index a9c32dc810d8f..4736ae92010f3 100644 --- a/server/src/test/java/org/opensearch/indices/cluster/ClusterStateChanges.java +++ b/server/src/test/java/org/opensearch/indices/cluster/ClusterStateChanges.java @@ -461,6 +461,8 @@ public ClusterState applyFailedShards(ClusterState clusterState, List new StartedShardEntry(e.getKey().shardId(), e.getKey().allocationId().getId(), e.getValue(), "shard started", null) + e -> new StartedShardEntry(e.getKey().shardId(), e.getKey().allocationId().getId(), e.getValue(), "shard started", null, null, null) ) .collect(Collectors.toList()) ); diff --git a/server/src/test/java/org/opensearch/indices/recovery/PeerRecoverySourceServiceTests.java b/server/src/test/java/org/opensearch/indices/recovery/PeerRecoverySourceServiceTests.java index 1b9a951356f11..3369e44816cd4 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/PeerRecoverySourceServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/PeerRecoverySourceServiceTests.java @@ -40,6 +40,7 @@ import org.opensearch.index.shard.IndexShardTestCase; import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.inplacesplit.InPlaceShardSplitRecoveryService; import org.opensearch.test.NodeRoles; import org.opensearch.transport.TransportService; @@ -57,10 +58,15 @@ public void testDuplicateRecoveries() throws IOException { final ClusterService clusterService = mock(ClusterService.class); when(clusterService.getSettings()).thenReturn(NodeRoles.dataNode()); when(indicesService.clusterService()).thenReturn(clusterService); + InPlaceShardSplitRecoveryService splitRecoveryService = new InPlaceShardSplitRecoveryService( + indicesService, + new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)) + ); PeerRecoverySourceService peerRecoverySourceService = new PeerRecoverySourceService( mock(TransportService.class), indicesService, - new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)) + new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), + splitRecoveryService ); StartRecoveryRequest startRecoveryRequest = new StartRecoveryRequest( diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index d7e14a4d2aa56..31b91fa042958 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -2114,7 +2114,8 @@ public void onFailure(final Exception e) { new NodeMappingRefreshAction(transportService, metadataMappingService), repositoriesService, mock(SearchService.class), - new PeerRecoverySourceService(transportService, indicesService, recoverySettings), + new PeerRecoverySourceService(transportService, indicesService, recoverySettings, + new InPlaceShardSplitRecoveryService(indicesService, recoverySettings)), snapshotShardsService, new PrimaryReplicaSyncer( transportService, diff --git a/test/framework/src/main/java/org/opensearch/cluster/routing/ShardRoutingHelper.java b/test/framework/src/main/java/org/opensearch/cluster/routing/ShardRoutingHelper.java index e09e333b90f25..ebd327b501447 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/routing/ShardRoutingHelper.java +++ b/test/framework/src/main/java/org/opensearch/cluster/routing/ShardRoutingHelper.java @@ -68,9 +68,8 @@ public static ShardRouting initWithSameId(ShardRouting copy, RecoverySource reco new UnassignedInfo(UnassignedInfo.Reason.REINITIALIZED, null), copy.allocationId(), copy.getExpectedShardSize(), - copy.getShardRange(), - copy.getParentShardId(), - copy.getRecoveringChildShardRanges() + copy.getRecoveringChildShards(), + copy.getParentShardId() ); } @@ -89,9 +88,8 @@ public static ShardRouting newWithRestoreSource(ShardRouting routing, RecoverySo routing.unassignedInfo(), routing.allocationId(), routing.getExpectedShardSize(), - routing.getShardRange(), - routing.getParentShardId(), - routing.getRecoveringChildShardRanges() + routing.getRecoveringChildShards(), + routing.getParentShardId() ); } } diff --git a/test/framework/src/main/java/org/opensearch/cluster/routing/TestShardRouting.java b/test/framework/src/main/java/org/opensearch/cluster/routing/TestShardRouting.java index 530232060b647..a44fd0113d7d7 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/routing/TestShardRouting.java +++ b/test/framework/src/main/java/org/opensearch/cluster/routing/TestShardRouting.java @@ -66,7 +66,6 @@ public static ShardRouting newShardRouting(ShardId shardId, String currentNodeId buildAllocationId(state), -1, null, - null, null ); } @@ -89,7 +88,6 @@ public static ShardRouting newShardRouting( buildAllocationId(state), -1, null, - null, null ); } @@ -129,7 +127,6 @@ public static ShardRouting newShardRouting( buildAllocationId(state), -1, null, - null, null ); } @@ -172,7 +169,6 @@ public static ShardRouting newShardRouting( allocationId, -1, null, - null, null ); } @@ -215,7 +211,6 @@ public static ShardRouting newShardRouting( buildAllocationId(state), -1, null, - null, null ); } @@ -240,7 +235,6 @@ public static ShardRouting newShardRouting( buildAllocationId(state), -1, null, - null, null ); } diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index d5002778a91aa..aadf1a967170d 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -1143,7 +1143,8 @@ protected final void recoverUnstartedReplica( request, recoverySettings, false, - new CancellableThreads() + new CancellableThreads(), + null ); primary.updateShardState( primary.routingEntry(),