|
@@ -34,6 +34,11 @@ import org.elasticsearch.action.admin.indices.close.CloseIndexResponse.IndexResu
|
|
|
import org.elasticsearch.action.admin.indices.close.CloseIndexResponse.ShardResult;
|
|
|
import org.elasticsearch.action.admin.indices.close.TransportVerifyShardBeforeCloseAction;
|
|
|
import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest;
|
|
|
+import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockClusterStateUpdateRequest;
|
|
|
+import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockResponse;
|
|
|
+import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockResponse.AddBlockResult;
|
|
|
+import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockResponse.AddBlockShardResult;
|
|
|
+import org.elasticsearch.action.admin.indices.readonly.TransportVerifyShardIndexBlockAction;
|
|
|
import org.elasticsearch.action.support.ActiveShardsObserver;
|
|
|
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
|
|
import org.elasticsearch.client.node.NodeClient;
|
|
@@ -45,6 +50,7 @@ import org.elasticsearch.cluster.ack.OpenIndexClusterStateUpdateResponse;
|
|
|
import org.elasticsearch.cluster.block.ClusterBlock;
|
|
|
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
|
|
import org.elasticsearch.cluster.block.ClusterBlocks;
|
|
|
+import org.elasticsearch.cluster.metadata.IndexMetadata.APIBlock;
|
|
|
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
|
|
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
|
|
import org.elasticsearch.cluster.routing.RoutingTable;
|
|
@@ -92,7 +98,7 @@ import static java.util.Collections.singleton;
|
|
|
import static java.util.Collections.unmodifiableMap;
|
|
|
|
|
|
/**
|
|
|
- * Service responsible for submitting open/close index requests
|
|
|
+ * Service responsible for submitting open/close index requests as well as for adding index blocks
|
|
|
*/
|
|
|
public class MetadataIndexStateService {
|
|
|
private static final Logger logger = LogManager.getLogger(MetadataIndexStateService.class);
|
|
@@ -304,6 +310,176 @@ public class MetadataIndexStateService {
|
|
|
return ClusterState.builder(currentState).blocks(blocks).metadata(metadata).routingTable(routingTable.build()).build();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Updates the cluster state for the given indices with the given index block,
|
|
|
+ * and also returns the updated indices (and their blocks) in a map.
|
|
|
+ * @param indices The indices to add blocks to if needed
|
|
|
+ * @param currentState The current cluster state
|
|
|
+ * @param block The type of block to add
|
|
|
+ * @return a tuple of the updated cluster state, as well as the blocks that got added
|
|
|
+ */
|
|
|
+ static Tuple<ClusterState, Map<Index, ClusterBlock>> addIndexBlock(final Index[] indices, final ClusterState currentState,
|
|
|
+ final APIBlock block) {
|
|
|
+ final Metadata.Builder metadata = Metadata.builder(currentState.metadata());
|
|
|
+
|
|
|
+ final Set<Index> indicesToAddBlock = new HashSet<>();
|
|
|
+ for (Index index : indices) {
|
|
|
+ metadata.getSafe(index); // to check if index exists
|
|
|
+ if (currentState.blocks().hasIndexBlock(index.getName(), block.block)) {
|
|
|
+ logger.debug("index {} already has block {}, ignoring", index, block.block);
|
|
|
+ } else {
|
|
|
+ indicesToAddBlock.add(index);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (indicesToAddBlock.isEmpty()) {
|
|
|
+ return Tuple.tuple(currentState, Collections.emptyMap());
|
|
|
+ }
|
|
|
+
|
|
|
+ final ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks());
|
|
|
+ final RoutingTable.Builder routingTable = RoutingTable.builder(currentState.routingTable());
|
|
|
+ final Map<Index, ClusterBlock> blockedIndices = new HashMap<>();
|
|
|
+
|
|
|
+ for (Index index : indicesToAddBlock) {
|
|
|
+ ClusterBlock indexBlock = null;
|
|
|
+ final Set<ClusterBlock> clusterBlocks = currentState.blocks().indices().get(index.getName());
|
|
|
+ if (clusterBlocks != null) {
|
|
|
+ for (ClusterBlock clusterBlock : clusterBlocks) {
|
|
|
+ if (clusterBlock.id() == block.block.id()) {
|
|
|
+ // Reuse the existing UUID-based block
|
|
|
+ indexBlock = clusterBlock;
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (indexBlock == null) {
|
|
|
+ // Create a new UUID-based block
|
|
|
+ indexBlock = createUUIDBasedBlock(block.block);
|
|
|
+ }
|
|
|
+ assert Strings.hasLength(indexBlock.uuid()) : "Block should have a UUID";
|
|
|
+ blocks.addIndexBlock(index.getName(), indexBlock);
|
|
|
+ blockedIndices.put(index, indexBlock);
|
|
|
+ // update index settings as well to match the block
|
|
|
+ final IndexMetadata indexMetadata = metadata.getSafe(index);
|
|
|
+ if (block.setting().get(indexMetadata.getSettings()) == false) {
|
|
|
+ final Settings updatedSettings = Settings.builder()
|
|
|
+ .put(indexMetadata.getSettings()).put(block.settingName(), true).build();
|
|
|
+
|
|
|
+ metadata.put(IndexMetadata.builder(indexMetadata)
|
|
|
+ .settings(updatedSettings)
|
|
|
+ .settingsVersion(indexMetadata.getSettingsVersion() + 1));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ logger.info("adding block {} to indices {}", block.name,
|
|
|
+ blockedIndices.keySet().stream().map(Object::toString).collect(Collectors.toList()));
|
|
|
+ return Tuple.tuple(ClusterState.builder(currentState).blocks(blocks).metadata(metadata)
|
|
|
+ .routingTable(routingTable.build()).build(), blockedIndices);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Adds an index block based on the given request, and notifies the listener upon completion.
|
|
|
+ * Adding blocks is done in three steps:
|
|
|
+ * - First, a temporary UUID-based block is added to the index
|
|
|
+ * (see {@link #addIndexBlock(Index[], ClusterState, APIBlock)}.
|
|
|
+ * - Second, shards are checked to have properly applied the UUID-based block.
|
|
|
+ * (see {@link WaitForBlocksApplied}).
|
|
|
+ * - Third, the temporary UUID-based block is turned into a full block
|
|
|
+ * (see {@link #finalizeBlock(ClusterState, Map, Map, APIBlock)}.
|
|
|
+ * Using this three-step process ensures non-interference by other operations in case where
|
|
|
+ * we notify successful completion here.
|
|
|
+ */
|
|
|
+ public void addIndexBlock(AddIndexBlockClusterStateUpdateRequest request,
|
|
|
+ ActionListener<AddIndexBlockResponse> listener) {
|
|
|
+ final Index[] concreteIndices = request.indices();
|
|
|
+ if (concreteIndices == null || concreteIndices.length == 0) {
|
|
|
+ throw new IllegalArgumentException("Index name is required");
|
|
|
+ }
|
|
|
+ List<String> writeIndices = new ArrayList<>();
|
|
|
+ SortedMap<String, IndexAbstraction> lookup = clusterService.state().metadata().getIndicesLookup();
|
|
|
+ for (Index index : concreteIndices) {
|
|
|
+ IndexAbstraction ia = lookup.get(index.getName());
|
|
|
+ if (ia != null && ia.getParentDataStream() != null && ia.getParentDataStream().getWriteIndex().getIndex().equals(index)) {
|
|
|
+ writeIndices.add(index.getName());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (writeIndices.size() > 0) {
|
|
|
+ throw new IllegalArgumentException("cannot add a block to the following data stream write indices [" +
|
|
|
+ Strings.collectionToCommaDelimitedString(writeIndices) + "]");
|
|
|
+ }
|
|
|
+
|
|
|
+ clusterService.submitStateUpdateTask("add-index-block-[" + request.getBlock().name + "]-" + Arrays.toString(concreteIndices),
|
|
|
+ new ClusterStateUpdateTask(Priority.URGENT) {
|
|
|
+
|
|
|
+ private Map<Index, ClusterBlock> blockedIndices;
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public ClusterState execute(final ClusterState currentState) {
|
|
|
+ final Tuple<ClusterState, Map<Index, ClusterBlock>> tup =
|
|
|
+ addIndexBlock(concreteIndices, currentState, request.getBlock());
|
|
|
+ blockedIndices = tup.v2();
|
|
|
+ return tup.v1();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void clusterStateProcessed(final String source, final ClusterState oldState, final ClusterState newState) {
|
|
|
+ if (oldState == newState) {
|
|
|
+ assert blockedIndices.isEmpty() : "List of blocked indices is not empty but cluster state wasn't changed";
|
|
|
+ listener.onResponse(new AddIndexBlockResponse(true, false, Collections.emptyList()));
|
|
|
+ } else {
|
|
|
+ assert blockedIndices.isEmpty() == false : "List of blocked indices is empty but cluster state was changed";
|
|
|
+ threadPool.executor(ThreadPool.Names.MANAGEMENT)
|
|
|
+ .execute(new WaitForBlocksApplied(blockedIndices, request,
|
|
|
+ ActionListener.wrap(verifyResults ->
|
|
|
+ clusterService.submitStateUpdateTask("finalize-index-block-[" + request.getBlock().name +
|
|
|
+ "]-[" + blockedIndices.keySet().stream().map(Index::getName)
|
|
|
+ .collect(Collectors.joining(", ")) + "]",
|
|
|
+ new ClusterStateUpdateTask(Priority.URGENT) {
|
|
|
+ private final List<AddBlockResult> indices = new ArrayList<>();
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public ClusterState execute(final ClusterState currentState) throws Exception {
|
|
|
+ Tuple<ClusterState, Collection<AddBlockResult>> addBlockResult =
|
|
|
+ finalizeBlock(currentState, blockedIndices, verifyResults, request.getBlock());
|
|
|
+ assert verifyResults.size() == addBlockResult.v2().size();
|
|
|
+ indices.addAll(addBlockResult.v2());
|
|
|
+ return addBlockResult.v1();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onFailure(final String source, final Exception e) {
|
|
|
+ listener.onFailure(e);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void clusterStateProcessed(final String source,
|
|
|
+ final ClusterState oldState,
|
|
|
+ final ClusterState newState) {
|
|
|
+
|
|
|
+ final boolean acknowledged = indices.stream().noneMatch(
|
|
|
+ AddBlockResult::hasFailures);
|
|
|
+ listener.onResponse(new AddIndexBlockResponse(acknowledged, acknowledged, indices));
|
|
|
+ }
|
|
|
+ }),
|
|
|
+ listener::onFailure)
|
|
|
+ )
|
|
|
+ );
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onFailure(final String source, final Exception e) {
|
|
|
+ listener.onFailure(e);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public TimeValue timeout() {
|
|
|
+ return request.masterNodeTimeout();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Step 2 - Wait for indices to be ready for closing
|
|
|
* <p>
|
|
@@ -429,6 +605,112 @@ public class MetadataIndexStateService {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Helper class that coordinates with shards to ensure that blocks have been properly applied to all shards using
|
|
|
+ * {@link TransportVerifyShardIndexBlockAction}.
|
|
|
+ */
|
|
|
+ class WaitForBlocksApplied extends ActionRunnable<Map<Index, AddBlockResult>> {
|
|
|
+
|
|
|
+ private final Map<Index, ClusterBlock> blockedIndices;
|
|
|
+ private final AddIndexBlockClusterStateUpdateRequest request;
|
|
|
+
|
|
|
+ private WaitForBlocksApplied(final Map<Index, ClusterBlock> blockedIndices,
|
|
|
+ final AddIndexBlockClusterStateUpdateRequest request,
|
|
|
+ final ActionListener<Map<Index, AddBlockResult>> listener) {
|
|
|
+ super(listener);
|
|
|
+ if (blockedIndices == null || blockedIndices.isEmpty()) {
|
|
|
+ throw new IllegalArgumentException("Cannot wait for blocks to be applied, list of blocked indices is empty or null");
|
|
|
+ }
|
|
|
+ this.blockedIndices = blockedIndices;
|
|
|
+ this.request = request;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void doRun() throws Exception {
|
|
|
+ final Map<Index, AddBlockResult> results = ConcurrentCollections.newConcurrentMap();
|
|
|
+ final CountDown countDown = new CountDown(blockedIndices.size());
|
|
|
+ final ClusterState state = clusterService.state();
|
|
|
+ blockedIndices.forEach((index, block) -> {
|
|
|
+ waitForShardsReady(index, block, state, response -> {
|
|
|
+ results.put(index, response);
|
|
|
+ if (countDown.countDown()) {
|
|
|
+ listener.onResponse(unmodifiableMap(results));
|
|
|
+ }
|
|
|
+ });
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
+ private void waitForShardsReady(final Index index,
|
|
|
+ final ClusterBlock clusterBlock,
|
|
|
+ final ClusterState state,
|
|
|
+ final Consumer<AddBlockResult> onResponse) {
|
|
|
+ final IndexMetadata indexMetadata = state.metadata().index(index);
|
|
|
+ if (indexMetadata == null) {
|
|
|
+ logger.debug("index {} has since been deleted, ignoring", index);
|
|
|
+ onResponse.accept(new AddBlockResult(index));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ final IndexRoutingTable indexRoutingTable = state.routingTable().index(index);
|
|
|
+ if (indexRoutingTable == null || indexMetadata.getState() == IndexMetadata.State.CLOSE) {
|
|
|
+ logger.debug("index {} is closed, no need to wait for shards, ignoring", index);
|
|
|
+ onResponse.accept(new AddBlockResult(index));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ final ImmutableOpenIntMap<IndexShardRoutingTable> shards = indexRoutingTable.getShards();
|
|
|
+ final AtomicArray<AddBlockShardResult> results = new AtomicArray<>(shards.size());
|
|
|
+ final CountDown countDown = new CountDown(shards.size());
|
|
|
+
|
|
|
+ for (IntObjectCursor<IndexShardRoutingTable> shard : shards) {
|
|
|
+ final IndexShardRoutingTable shardRoutingTable = shard.value;
|
|
|
+ final int shardId = shardRoutingTable.shardId().id();
|
|
|
+ sendVerifyShardBlockRequest(shardRoutingTable, clusterBlock, new NotifyOnceListener<ReplicationResponse>() {
|
|
|
+ @Override
|
|
|
+ public void innerOnResponse(final ReplicationResponse replicationResponse) {
|
|
|
+ AddBlockShardResult.Failure[] failures = Arrays.stream(replicationResponse.getShardInfo().getFailures())
|
|
|
+ .map(f -> new AddBlockShardResult.Failure(f.index(), f.shardId(), f.getCause(), f.nodeId()))
|
|
|
+ .toArray(AddBlockShardResult.Failure[]::new);
|
|
|
+ results.setOnce(shardId, new AddBlockShardResult(shardId, failures));
|
|
|
+ processIfFinished();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void innerOnFailure(final Exception e) {
|
|
|
+ AddBlockShardResult.Failure failure = new AddBlockShardResult.Failure(index.getName(), shardId, e);
|
|
|
+ results.setOnce(shardId, new AddBlockShardResult(shardId, new AddBlockShardResult.Failure[]{failure}));
|
|
|
+ processIfFinished();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void processIfFinished() {
|
|
|
+ if (countDown.countDown()) {
|
|
|
+ onResponse.accept(new AddBlockResult(index, results.toArray(new AddBlockShardResult[results.length()])));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void sendVerifyShardBlockRequest(final IndexShardRoutingTable shardRoutingTable,
|
|
|
+ final ClusterBlock block,
|
|
|
+ final ActionListener<ReplicationResponse> listener) {
|
|
|
+ final ShardId shardId = shardRoutingTable.shardId();
|
|
|
+ if (shardRoutingTable.primaryShard().unassigned()) {
|
|
|
+ logger.debug("primary shard {} is unassigned, ignoring", shardId);
|
|
|
+ final ReplicationResponse response = new ReplicationResponse();
|
|
|
+ response.setShardInfo(new ReplicationResponse.ShardInfo(shardRoutingTable.size(), shardRoutingTable.size()));
|
|
|
+ listener.onResponse(response);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ final TaskId parentTaskId = new TaskId(clusterService.localNode().getId(), request.taskId());
|
|
|
+ final TransportVerifyShardIndexBlockAction.ShardRequest shardRequest =
|
|
|
+ new TransportVerifyShardIndexBlockAction.ShardRequest(shardId, block, parentTaskId);
|
|
|
+ if (request.ackTimeout() != null) {
|
|
|
+ shardRequest.timeout(request.ackTimeout());
|
|
|
+ }
|
|
|
+ client.executeLocally(TransportVerifyShardIndexBlockAction.TYPE, shardRequest, listener);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Step 3 - Move index states from OPEN to CLOSE in cluster state for indices that are ready for closing.
|
|
|
*/
|
|
@@ -618,6 +900,68 @@ public class MetadataIndexStateService {
|
|
|
return ClusterState.builder(updatedState).routingTable(routingTable.build()).build();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Finalizes the addition of blocks by turning the temporary UUID-based blocks into full blocks.
|
|
|
+ * @param currentState the cluster state to update
|
|
|
+ * @param blockedIndices the indices and their temporary UUID-based blocks to convert
|
|
|
+ * @param verifyResult the index-level results for adding the block
|
|
|
+ * @param block the full block to convert to
|
|
|
+ * @return the updated cluster state, as well as the (failed and successful) index-level results for adding the block
|
|
|
+ */
|
|
|
+ static Tuple<ClusterState, Collection<AddBlockResult>> finalizeBlock(final ClusterState currentState,
|
|
|
+ final Map<Index, ClusterBlock> blockedIndices,
|
|
|
+ final Map<Index, AddBlockResult> verifyResult,
|
|
|
+ final APIBlock block) {
|
|
|
+
|
|
|
+ final Metadata.Builder metadata = Metadata.builder(currentState.metadata());
|
|
|
+ final ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks());
|
|
|
+ final RoutingTable.Builder routingTable = RoutingTable.builder(currentState.routingTable());
|
|
|
+
|
|
|
+ final Set<String> effectivelyBlockedIndices = new HashSet<>();
|
|
|
+ Map<Index, AddBlockResult> blockingResults = new HashMap<>(verifyResult);
|
|
|
+ for (Map.Entry<Index, AddBlockResult> result : verifyResult.entrySet()) {
|
|
|
+ final Index index = result.getKey();
|
|
|
+ final boolean acknowledged = result.getValue().hasFailures() == false;
|
|
|
+ try {
|
|
|
+ if (acknowledged == false) {
|
|
|
+ logger.debug("verification of shards before blocking {} failed [{}]", index, result);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ final IndexMetadata indexMetadata = metadata.getSafe(index);
|
|
|
+ final ClusterBlock tempBlock = blockedIndices.get(index);
|
|
|
+ assert tempBlock != null;
|
|
|
+ assert tempBlock.uuid() != null;
|
|
|
+ final ClusterBlock currentBlock = currentState.blocks().getIndexBlockWithId(index.getName(), tempBlock.id());
|
|
|
+ if (currentBlock != null && currentBlock.equals(block.block)) {
|
|
|
+ logger.debug("verification of shards for {} succeeded, but block finalization already occurred" +
|
|
|
+ " (possibly for another block) [{}]", index, result);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
+ if (currentBlock == null || currentBlock.equals(tempBlock) == false) {
|
|
|
+ // we should report error in this case as the index can be left as open.
|
|
|
+ blockingResults.put(result.getKey(), new AddBlockResult(result.getKey(), new IllegalStateException(
|
|
|
+ "verification of shards before blocking " + index + " succeeded but block has been removed in the meantime")));
|
|
|
+ logger.debug("verification of shards before blocking {} succeeded but block has been removed in the meantime", index);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
+ assert currentBlock != null && currentBlock.equals(tempBlock) && currentBlock.id() == block.block.id();
|
|
|
+
|
|
|
+ blocks.removeIndexBlockWithId(index.getName(), tempBlock.id());
|
|
|
+ blocks.addIndexBlock(index.getName(), block.block);
|
|
|
+
|
|
|
+ logger.debug("add block {} to index {} succeeded", block.block, index);
|
|
|
+ effectivelyBlockedIndices.add(index.getName());
|
|
|
+ } catch (final IndexNotFoundException e) {
|
|
|
+ logger.debug("index {} has been deleted since blocking it started, ignoring", index);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ logger.info("completed adding block {} to indices {}", block.name, effectivelyBlockedIndices);
|
|
|
+ return Tuple.tuple(ClusterState.builder(currentState).blocks(blocks).metadata(metadata).routingTable(routingTable.build()).build(),
|
|
|
+ blockingResults.values());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* @return Generates a {@link ClusterBlock} that blocks read and write operations on soon-to-be-closed indices. The
|
|
|
* cluster block is generated with the id value equals to {@link #INDEX_CLOSED_BLOCK_ID} and a unique UUID.
|
|
@@ -633,4 +977,12 @@ public class MetadataIndexStateService {
|
|
|
&& VERIFIED_BEFORE_CLOSE_SETTING.exists(indexMetadata.getSettings())
|
|
|
&& VERIFIED_BEFORE_CLOSE_SETTING.get(indexMetadata.getSettings());
|
|
|
}
|
|
|
+
|
|
|
+ // Create UUID based block based on non-UUID one
|
|
|
+ public static ClusterBlock createUUIDBasedBlock(ClusterBlock clusterBlock) {
|
|
|
+ assert clusterBlock.uuid() == null : "no UUID expected on source block";
|
|
|
+ return new ClusterBlock(clusterBlock.id(), UUIDs.randomBase64UUID(), "moving to block " + clusterBlock.description(),
|
|
|
+ clusterBlock.retryable(), clusterBlock.disableStatePersistence(), clusterBlock.isAllowReleaseResources(), clusterBlock.status(),
|
|
|
+ clusterBlock.levels());
|
|
|
+ }
|
|
|
}
|