Skip to content
Merged
Show file tree
Hide file tree
Changes from 7 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -2154,7 +2154,7 @@ public void testDeleteIndexWithOutOfOrderFinalization() {
final Map<String, SubscribableListener<Void>> otherIndexSnapshotListeners = indexNames.stream()
.collect(Collectors.toMap(k -> k, k -> new SubscribableListener<>()));
masterTransportService.<UpdateIndexShardSnapshotStatusRequest>addRequestHandlingBehavior(
SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME,
TransportUpdateSnapshotStatusAction.NAME,
(handler, request, channel, task) -> {
final var indexName = request.shardId().getIndexName();
if (indexName.equals(indexToDelete)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1063,7 +1063,7 @@ public void onRequestSent(
TransportRequest request,
TransportRequestOptions finalOptions
) {
if (blocked.get() && action.equals(SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME)) {
if (blocked.get() && action.equals(TransportUpdateSnapshotStatusAction.NAME)) {
throw new AssertionError("Node had no assigned shard snapshots so it shouldn't send out shard state updates");
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ public void testConsistencyCheckerLogging() {
// allow cluster states to go through normally until we see a shard snapshot update
final var shardUpdateSeen = new AtomicBoolean();
MockTransportService.getInstance(masterNode)
.addRequestHandlingBehavior(SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME, (handler, request, channel, task) -> {
.addRequestHandlingBehavior(TransportUpdateSnapshotStatusAction.NAME, (handler, request, channel, task) -> {
shardUpdateSeen.set(true);
handler.messageReceived(request, channel, task);
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,7 @@ public void testRemoveNodeAndFailoverMasterDuringSnapshot() throws Exception {
final var masterName = internalCluster().getMasterName();
final var masterTransportService = MockTransportService.getInstance(masterName);
masterTransportService.addRequestHandlingBehavior(
SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME,
TransportUpdateSnapshotStatusAction.NAME,
(handler, request, channel, task) -> masterTransportService.getThreadPool().generic().execute(() -> {
safeAwait(snapshotStatusUpdateBarrier);
safeAwait(snapshotStatusUpdateBarrier);
Expand Down Expand Up @@ -405,7 +405,7 @@ public void testAbortSnapshotWhileRemovingNode() throws Exception {
final var updateSnapshotStatusBarrier = new CyclicBarrier(2);
final var masterTransportService = MockTransportService.getInstance(internalCluster().getMasterName());
masterTransportService.addRequestHandlingBehavior(
SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME,
TransportUpdateSnapshotStatusAction.NAME,
(handler, request, channel, task) -> masterTransportService.getThreadPool().generic().execute(() -> {
safeAwait(updateSnapshotStatusBarrier);
safeAwait(updateSnapshotStatusBarrier);
Expand Down Expand Up @@ -455,7 +455,7 @@ public void testShutdownWhileSuccessInFlight() throws Exception {
final var clusterService = internalCluster().getCurrentMasterNodeInstance(ClusterService.class);
final var masterTransportService = MockTransportService.getInstance(internalCluster().getMasterName());
masterTransportService.addRequestHandlingBehavior(
SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME,
TransportUpdateSnapshotStatusAction.NAME,
(handler, request, channel, task) -> putShutdownForRemovalMetadata(
clusterService,
primaryNode,
Expand Down Expand Up @@ -559,7 +559,7 @@ public void testSnapshotShutdownProgressTracker() throws Exception {
final CountDownLatch snapshotStatusUpdateLatch = new CountDownLatch(1);
final var masterTransportService = MockTransportService.getInstance(internalCluster().getMasterName());
masterTransportService.addRequestHandlingBehavior(
SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME,
TransportUpdateSnapshotStatusAction.NAME,
(handler, request, channel, task) -> masterTransportService.getThreadPool().generic().execute(() -> {
safeAwait(snapshotStatusUpdateLatch);
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -405,6 +405,7 @@
import org.elasticsearch.rest.action.synonyms.RestGetSynonymsSetsAction;
import org.elasticsearch.rest.action.synonyms.RestPutSynonymRuleAction;
import org.elasticsearch.rest.action.synonyms.RestPutSynonymsAction;
import org.elasticsearch.snapshots.TransportUpdateSnapshotStatusAction;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.telemetry.TelemetryProvider;
import org.elasticsearch.threadpool.ThreadPool;
Expand Down Expand Up @@ -672,6 +673,7 @@ public <Request extends ActionRequest, Response extends ActionResponse> void reg
actions.register(TransportDeleteSnapshotAction.TYPE, TransportDeleteSnapshotAction.class);
actions.register(TransportCreateSnapshotAction.TYPE, TransportCreateSnapshotAction.class);
actions.register(TransportCloneSnapshotAction.TYPE, TransportCloneSnapshotAction.class);
actions.register(TransportUpdateSnapshotStatusAction.TYPE, TransportUpdateSnapshotStatusAction.class);
Copy link
Member

Choose a reason for hiding this comment

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

This is no need to register this action here since it is never invoked by Client. We will not need modify the operator privilege Constants if we don't register the action here.

actions.register(TransportRestoreSnapshotAction.TYPE, TransportRestoreSnapshotAction.class);
actions.register(TransportSnapshotsStatusAction.TYPE, TransportSnapshotsStatusAction.class);
actions.register(SnapshottableFeaturesAction.INSTANCE, TransportSnapshottableFeaturesAction.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1135,7 +1135,6 @@ public Map<String, String> queryFields() {
clusterModule.getIndexNameExpressionResolver(),
repositoriesService,
transportService,
actionModule.getActionFilters(),
systemIndices,
projectResolver.supportsMultipleProjects()
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -905,7 +905,7 @@ public void onFailure(Exception e) {
updateResultListener,
(req, reqListener) -> transportService.sendRequest(
transportService.getLocalNode(),
SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME,
TransportUpdateSnapshotStatusAction.NAME,
req,
new ActionListenerResponseHandler<>(
reqListener.map(res -> null),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,17 +14,14 @@
import org.apache.logging.log4j.Logger;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.admin.cluster.snapshots.clone.CloneSnapshotRequest;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotRequest;
import org.elasticsearch.action.admin.cluster.snapshots.delete.DeleteSnapshotRequest;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.ContextPreservingActionListener;
import org.elasticsearch.action.support.GroupedActionListener;
import org.elasticsearch.action.support.RefCountingRunnable;
import org.elasticsearch.action.support.SubscribableListener;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateApplier;
Expand All @@ -37,7 +34,6 @@
import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.SnapshotsInProgress.ShardSnapshotStatus;
import org.elasticsearch.cluster.SnapshotsInProgress.ShardState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.DataStreamAlias;
Expand Down Expand Up @@ -93,7 +89,6 @@
import org.elasticsearch.repositories.ShardGeneration;
import org.elasticsearch.repositories.ShardGenerations;
import org.elasticsearch.repositories.ShardSnapshotResult;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;

Expand Down Expand Up @@ -150,8 +145,6 @@ public final class SnapshotsService extends AbstractLifecycleComponent implement

private static final Logger logger = LogManager.getLogger(SnapshotsService.class);

public static final String UPDATE_SNAPSHOT_STATUS_ACTION_NAME = "internal:cluster/snapshot/update_snapshot_status";

public static final String NO_FEATURE_STATES_VALUE = "none";

private final ClusterService clusterService;
Expand Down Expand Up @@ -180,10 +173,6 @@ public final class SnapshotsService extends AbstractLifecycleComponent implement
/** Set of currently initializing clone operations */
private final Set<Snapshot> initializingClones = Collections.synchronizedSet(new HashSet<>());

private final UpdateSnapshotStatusAction updateSnapshotStatusHandler;

private final TransportService transportService;

private final OngoingRepositoryOperations repositoryOperations = new OngoingRepositoryOperations();

private final SystemIndices systemIndices;
Expand Down Expand Up @@ -216,7 +205,6 @@ public SnapshotsService(
IndexNameExpressionResolver indexNameExpressionResolver,
RepositoriesService repositoriesService,
TransportService transportService,
ActionFilters actionFilters,
SystemIndices systemIndices,
boolean serializeProjectMetadata
) {
Expand All @@ -225,10 +213,7 @@ public SnapshotsService(
this.indexNameExpressionResolver = indexNameExpressionResolver;
this.repositoriesService = repositoriesService;
this.threadPool = transportService.getThreadPool();
this.transportService = transportService;

// The constructor of UpdateSnapshotStatusAction will register itself to the TransportService.
this.updateSnapshotStatusHandler = new UpdateSnapshotStatusAction(transportService, clusterService, threadPool, actionFilters);
if (DiscoveryNode.isMasterNode(settings)) {
// addLowPriorityApplier to make sure that Repository will be created before snapshot
clusterService.addLowPriorityApplier(this);
Expand Down Expand Up @@ -592,7 +577,7 @@ private void runReadyClone(
repoShardId,
shardStatusBefore.generation(),
ActionListener.wrap(
shardSnapshotResult -> innerUpdateSnapshotState(
shardSnapshotResult -> createAndSubmitRequestToUpdateSnapshotState(
target,
null,
repoShardId,
Expand All @@ -613,7 +598,7 @@ private void runReadyClone(
() -> currentlyCloning.remove(repoShardId)
)
),
e -> innerUpdateSnapshotState(
e -> createAndSubmitRequestToUpdateSnapshotState(
target,
null,
repoShardId,
Expand Down Expand Up @@ -2256,8 +2241,7 @@ private void addListener(Snapshot snapshot, ActionListener<SnapshotInfo> listene

@Override
protected void doStart() {
assert this.updateSnapshotStatusHandler != null;
assert transportService.getRequestHandler(UPDATE_SNAPSHOT_STATUS_ACTION_NAME) != null;

}

@Override
Expand Down Expand Up @@ -2834,7 +2818,7 @@ public String toString() {
}
}

private void innerUpdateSnapshotState(
public void createAndSubmitRequestToUpdateSnapshotState(
Snapshot snapshot,
ShardId shardId,
RepositoryShardId repoShardId,
Expand Down Expand Up @@ -2896,50 +2880,6 @@ private void startExecutableClones(List<SnapshotsInProgress.Entry> entries) {
}
}

private class UpdateSnapshotStatusAction extends TransportMasterNodeAction<
UpdateIndexShardSnapshotStatusRequest,
ActionResponse.Empty> {
UpdateSnapshotStatusAction(
TransportService transportService,
ClusterService clusterService,
ThreadPool threadPool,
ActionFilters actionFilters
) {
super(
UPDATE_SNAPSHOT_STATUS_ACTION_NAME,
false,
transportService,
clusterService,
threadPool,
actionFilters,
UpdateIndexShardSnapshotStatusRequest::new,
in -> ActionResponse.Empty.INSTANCE,
EsExecutors.DIRECT_EXECUTOR_SERVICE
);
}

@Override
protected void masterOperation(
Task task,
UpdateIndexShardSnapshotStatusRequest request,
ClusterState state,
ActionListener<ActionResponse.Empty> listener
) {
innerUpdateSnapshotState(
request.snapshot(),
request.shardId(),
null,
request.status(),
listener.map(v -> ActionResponse.Empty.INSTANCE)
);
}

@Override
protected ClusterBlockException checkBlock(UpdateIndexShardSnapshotStatusRequest request, ClusterState state) {
return null;
}
}

/**
* Cluster state update task that removes all {@link SnapshotsInProgress.Entry} and {@link SnapshotDeletionsInProgress.Entry} for a
* given repository from the cluster state and afterwards fails all relevant listeners in {@link #snapshotCompletionListeners} and
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the "Elastic License
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
* Public License v 1"; you may not use this file except in compliance with, at
* your election, the "Elastic License 2.0", the "GNU Affero General Public
* License v3.0 only", or the "Server Side Public License, v 1".
*/

package org.elasticsearch.snapshots;

import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.injection.guice.Inject;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;

public final class TransportUpdateSnapshotStatusAction extends TransportMasterNodeAction<
UpdateIndexShardSnapshotStatusRequest,
ActionResponse.Empty> {
public static final String NAME = "internal:cluster/snapshot/update_snapshot_status";
public static final ActionType<ActionResponse.Empty> TYPE = new ActionType<>(NAME);

private final SnapshotsService snapshotsService;

@Inject
public TransportUpdateSnapshotStatusAction(
TransportService transportService,
ClusterService clusterService,
ThreadPool threadPool,
SnapshotsService snapshotsService,
ActionFilters actionFilters
) {
super(
NAME,
false,
transportService,
clusterService,
threadPool,
actionFilters,
UpdateIndexShardSnapshotStatusRequest::new,
in -> ActionResponse.Empty.INSTANCE,
EsExecutors.DIRECT_EXECUTOR_SERVICE
);
this.snapshotsService = snapshotsService;
}

@Override
protected void masterOperation(
Task task,
UpdateIndexShardSnapshotStatusRequest request,
ClusterState state,
ActionListener<ActionResponse.Empty> listener
) {
snapshotsService.createAndSubmitRequestToUpdateSnapshotState(
request.snapshot(),
request.shardId(),
null,
request.status(),
listener.map(v -> ActionResponse.Empty.INSTANCE)
);
}

@Override
protected ClusterBlockException checkBlock(UpdateIndexShardSnapshotStatusRequest request, ClusterState state) {
return null;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1050,7 +1050,7 @@ private void sendLocalRequest(long requestId, final String action, final Transpo
@SuppressWarnings("unchecked")
final RequestHandlerRegistry<TransportRequest> reg = (RequestHandlerRegistry<TransportRequest>) getRequestHandler(action);
if (reg == null) {
assert false : action;
assert false : "Action [" + action + "] not found";
throw new ActionNotFoundTransportException("Action [" + action + "] not found");
}
final Executor executor = reg.getExecutor();
Expand Down
Loading