Skip to content

Commit b7a4586

Browse files
committed
Add dummy methods in RemoteClusterStateService
Signed-off-by: Sooraj Sinha <soosinha@amazon.com>
1 parent f4c2a04 commit b7a4586

14 files changed

+192
-60
lines changed

server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java

+3-4
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@
4040
import org.opensearch.cluster.node.DiscoveryNode;
4141
import org.opensearch.common.settings.Settings;
4242
import org.opensearch.common.util.io.IOUtils;
43+
import org.opensearch.node.remotestore.RemoteStoreNodeAttribute;
4344

4445
import java.io.Closeable;
4546
import java.io.IOException;
@@ -50,10 +51,8 @@
5051
import java.util.Map;
5152
import java.util.Optional;
5253
import java.util.Set;
53-
import org.opensearch.node.remotestore.RemoteStoreNodeAttribute;
5454

5555
import static org.opensearch.cluster.coordination.Coordinator.ZEN1_BWC_TERM;
56-
import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteRoutingTableEnabled;
5756
import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteStoreClusterStateEnabled;
5857

5958
/**
@@ -583,7 +582,7 @@ public void handlePrePublish(ClusterState clusterState) {
583582
// This is to ensure the remote store is the single source of truth for current state. Even if the current node
584583
// goes down after sending the cluster state to other nodes, we should be able to read the remote state and
585584
// recover the cluster.
586-
if (isRemotePublicationEnabled) {
585+
if (isRemoteStateEnabled) {
587586
assert persistedStateRegistry.getPersistedState(PersistedStateType.REMOTE) != null : "Remote state has not been initialized";
588587
persistedStateRegistry.getPersistedState(PersistedStateType.REMOTE).setLastAcceptedState(clusterState);
589588
}
@@ -594,7 +593,7 @@ public void handlePrePublish(ClusterState clusterState) {
594593
*/
595594
public void handlePreCommit() {
596595
// Publishing the committed state to remote store before sending apply commit to other nodes.
597-
if (isRemotePublicationEnabled) {
596+
if (isRemoteStateEnabled) {
598597
assert persistedStateRegistry.getPersistedState(PersistedStateType.REMOTE) != null : "Remote state has not been initialized";
599598
persistedStateRegistry.getPersistedState(PersistedStateType.REMOTE).markLastAcceptedStateAsCommitted();
600599
}

server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java

-1
Original file line numberDiff line numberDiff line change
@@ -49,7 +49,6 @@
4949
import org.opensearch.cluster.coordination.CoordinationState.VoteCollection;
5050
import org.opensearch.cluster.coordination.FollowersChecker.FollowerCheckRequest;
5151
import org.opensearch.cluster.coordination.JoinHelper.InitialJoinAccumulator;
52-
import org.opensearch.cluster.coordination.PersistedStateRegistry.PersistedStateType;
5352
import org.opensearch.cluster.metadata.Metadata;
5453
import org.opensearch.cluster.node.DiscoveryNode;
5554
import org.opensearch.cluster.node.DiscoveryNodes;

server/src/main/java/org/opensearch/cluster/coordination/PublicationTransportHandler.java

+72-23
Original file line numberDiff line numberDiff line change
@@ -31,9 +31,6 @@
3131

3232
package org.opensearch.cluster.coordination;
3333

34-
import java.util.Locale;
35-
import java.util.Optional;
36-
import java.util.function.Supplier;
3734
import org.apache.logging.log4j.LogManager;
3835
import org.apache.logging.log4j.Logger;
3936
import org.apache.logging.log4j.message.ParameterizedMessage;
@@ -43,7 +40,6 @@
4340
import org.opensearch.cluster.ClusterState;
4441
import org.opensearch.cluster.Diff;
4542
import org.opensearch.cluster.IncompatibleClusterStateVersionException;
46-
import org.opensearch.cluster.coordination.CoordinationState.PersistedState;
4743
import org.opensearch.cluster.coordination.PersistedStateRegistry.PersistedStateType;
4844
import org.opensearch.cluster.node.DiscoveryNode;
4945
import org.opensearch.cluster.node.DiscoveryNodes;
@@ -65,9 +61,7 @@
6561

6662
import java.io.IOException;
6763
import java.util.HashMap;
68-
import java.util.Locale;
6964
import java.util.Map;
70-
import java.util.Optional;
7165
import java.util.concurrent.atomic.AtomicLong;
7266
import java.util.concurrent.atomic.AtomicReference;
7367
import java.util.function.BiConsumer;
@@ -238,7 +232,10 @@ private PublishWithJoinResponse handleIncomingRemotePublishRequest(RemotePublish
238232
if (transportService.getLocalNode().equals(request.getSourceNode())) {
239233
return acceptStateOnLocalNode(request);
240234
}
241-
ClusterMetadataManifest manifest = remoteClusterStateService.getClusterMetadataManifestByFileName(request.getClusterUUID(), request.getManifestFile());
235+
ClusterMetadataManifest manifest = remoteClusterStateService.getClusterMetadataManifestByFileName(
236+
request.getClusterUUID(),
237+
request.getManifestFile()
238+
);
242239
boolean applyFullState = false;
243240
final ClusterState lastSeen = lastSeenClusterState.get();
244241
if (lastSeen == null) {
@@ -253,17 +250,40 @@ private PublishWithJoinResponse handleIncomingRemotePublishRequest(RemotePublish
253250
}
254251

255252
if (applyFullState == true) {
256-
logger.debug(() -> new ParameterizedMessage("Downloading full cluster state for term {}, version {}, stateUUID {}", manifest.getClusterTerm(), manifest.getStateVersion(),
257-
manifest.getStateUUID()));
258-
ClusterState clusterState = remoteClusterStateService.getClusterStateForManifest(request.getClusterName(), manifest, transportService.getLocalNode().getId(), true);
253+
logger.debug(
254+
() -> new ParameterizedMessage(
255+
"Downloading full cluster state for term {}, version {}, stateUUID {}",
256+
manifest.getClusterTerm(),
257+
manifest.getStateVersion(),
258+
manifest.getStateUUID()
259+
)
260+
);
261+
ClusterState clusterState = remoteClusterStateService.getClusterStateForManifest(
262+
request.getClusterName(),
263+
manifest,
264+
transportService.getLocalNode().getId(),
265+
true
266+
);
259267
fullClusterStateReceivedCount.incrementAndGet();
260268
final PublishWithJoinResponse response = acceptState(clusterState);
261269
lastSeenClusterState.set(clusterState);
262270
return response;
263271
} else {
264-
logger.debug(() -> new ParameterizedMessage("Downloading diff cluster state for term {}, version {}, previousUUID {}, current UUID {}", manifest.getClusterTerm(),
265-
manifest.getStateVersion(), manifest.getDiffManifest().getFromStateUUID(), manifest.getStateUUID()));
266-
ClusterState clusterState = remoteClusterStateService.getClusterStateUsingDiff(request.getClusterName(), manifest, lastSeen, transportService.getLocalNode().getId());
272+
logger.debug(
273+
() -> new ParameterizedMessage(
274+
"Downloading diff cluster state for term {}, version {}, previousUUID {}, current UUID {}",
275+
manifest.getClusterTerm(),
276+
manifest.getStateVersion(),
277+
manifest.getDiffManifest().getFromStateUUID(),
278+
manifest.getStateUUID()
279+
)
280+
);
281+
ClusterState clusterState = remoteClusterStateService.getClusterStateUsingDiff(
282+
request.getClusterName(),
283+
manifest,
284+
lastSeen,
285+
transportService.getLocalNode().getId()
286+
);
267287
compatibleClusterStateDiffReceivedCount.incrementAndGet();
268288
final PublishWithJoinResponse response = acceptState(clusterState);
269289
lastSeenClusterState.compareAndSet(lastSeen, clusterState);
@@ -286,7 +306,8 @@ private PublishWithJoinResponse acceptState(ClusterState incomingState) {
286306

287307
private PublishWithJoinResponse acceptStateOnLocalNode(RemotePublishRequest remotePublishRequest) {
288308
final PublishRequest publishRequest = currentPublishRequestToSelf.get();
289-
if (publishRequest == null || publishRequest.getAcceptedState().coordinationMetadata().term() != remotePublishRequest.term
309+
if (publishRequest == null
310+
|| publishRequest.getAcceptedState().coordinationMetadata().term() != remotePublishRequest.term
290311
|| publishRequest.getAcceptedState().version() != remotePublishRequest.version) {
291312
throw new IllegalStateException("publication to self failed for " + remotePublishRequest);
292313
}
@@ -295,9 +316,16 @@ private PublishWithJoinResponse acceptStateOnLocalNode(RemotePublishRequest remo
295316
return publishWithJoinResponse;
296317
}
297318

298-
public PublicationContext newPublicationContext(ClusterChangedEvent clusterChangedEvent, boolean isRemotePublicationEnabled,
299-
PersistedStateRegistry persistedStateRegistry) {
300-
final PublicationContext publicationContext = new PublicationContext(clusterChangedEvent, isRemotePublicationEnabled, persistedStateRegistry);
319+
public PublicationContext newPublicationContext(
320+
ClusterChangedEvent clusterChangedEvent,
321+
boolean isRemotePublicationEnabled,
322+
PersistedStateRegistry persistedStateRegistry
323+
) {
324+
final PublicationContext publicationContext = new PublicationContext(
325+
clusterChangedEvent,
326+
isRemotePublicationEnabled,
327+
persistedStateRegistry
328+
);
301329

302330
// Build the serializations we expect to need now, early in the process, so that an error during serialization fails the publication
303331
// straight away. This isn't watertight since we send diffs on a best-effort basis and may fall back to sending a full state (and
@@ -345,7 +373,11 @@ public class PublicationContext {
345373
private final boolean sendRemoteState;
346374
private final PersistedStateRegistry persistedStateRegistry;
347375

348-
PublicationContext(ClusterChangedEvent clusterChangedEvent, boolean isRemotePublicationEnabled, PersistedStateRegistry persistedStateRegistry) {
376+
PublicationContext(
377+
ClusterChangedEvent clusterChangedEvent,
378+
boolean isRemotePublicationEnabled,
379+
PersistedStateRegistry persistedStateRegistry
380+
) {
349381
discoveryNodes = clusterChangedEvent.state().nodes();
350382
newState = clusterChangedEvent.state();
351383
previousState = clusterChangedEvent.previousState();
@@ -462,11 +494,22 @@ public String executor() {
462494
);
463495
}
464496

465-
private void sendRemoteClusterState(final DiscoveryNode destination, final ClusterState clusterState, final ActionListener<PublishWithJoinResponse> listener) {
497+
private void sendRemoteClusterState(
498+
final DiscoveryNode destination,
499+
final ClusterState clusterState,
500+
final ActionListener<PublishWithJoinResponse> listener
501+
) {
466502
try {
467-
final String manifestFileName = ((RemotePersistedState) persistedStateRegistry.getPersistedState(PersistedStateType.REMOTE)).getLastUploadedManifestFile();
468-
final RemotePublishRequest remotePublishRequest = new RemotePublishRequest(discoveryNodes.getLocalNode(), clusterState.term(),
469-
clusterState.getVersion(), clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), manifestFileName);
503+
final String manifestFileName = ((RemotePersistedState) persistedStateRegistry.getPersistedState(PersistedStateType.REMOTE))
504+
.getLastUploadedManifestFile();
505+
final RemotePublishRequest remotePublishRequest = new RemotePublishRequest(
506+
discoveryNodes.getLocalNode(),
507+
clusterState.term(),
508+
clusterState.getVersion(),
509+
clusterState.getClusterName().value(),
510+
clusterState.metadata().clusterUUID(),
511+
manifestFileName
512+
);
470513
final Consumer<TransportException> transportExceptionHandler = exp -> {
471514
logger.debug(() -> new ParameterizedMessage("failed to send remote cluster state to {}", destination), exp);
472515
listener.onFailure(exp);
@@ -493,7 +536,13 @@ public String executor() {
493536
return ThreadPool.Names.GENERIC;
494537
}
495538
};
496-
transportService.sendRequest(destination, PUBLISH_REMOTE_STATE_ACTION_NAME, remotePublishRequest, stateRequestOptions, responseHandler);
539+
transportService.sendRequest(
540+
destination,
541+
PUBLISH_REMOTE_STATE_ACTION_NAME,
542+
remotePublishRequest,
543+
stateRequestOptions,
544+
responseHandler
545+
);
497546
} catch (Exception e) {
498547
logger.warn(() -> new ParameterizedMessage("error sending remote cluster state to {}", destination), e);
499548
listener.onFailure(e);

server/src/main/java/org/opensearch/cluster/coordination/RemotePublishRequest.java

+26-3
Original file line numberDiff line numberDiff line change
@@ -14,13 +14,24 @@
1414

1515
import java.io.IOException;
1616

17+
/**
18+
* Send the publish request with the remote cluster state details
19+
* @opensearch.internal
20+
*/
1721
public class RemotePublishRequest extends TermVersionRequest {
1822

1923
private final String clusterName;
2024
private final String clusterUUID;
2125
private final String manifestFile;
2226

23-
public RemotePublishRequest(DiscoveryNode sourceNode, long term, long version, String clusterName, String clusterUUID, String manifestFile) {
27+
public RemotePublishRequest(
28+
DiscoveryNode sourceNode,
29+
long term,
30+
long version,
31+
String clusterName,
32+
String clusterUUID,
33+
String manifestFile
34+
) {
2435
super(sourceNode, term, version);
2536
this.clusterName = clusterName;
2637
this.clusterUUID = clusterUUID;
@@ -44,8 +55,20 @@ public void writeTo(StreamOutput out) throws IOException {
4455

4556
@Override
4657
public String toString() {
47-
return "RemotePublishRequest{" + "term=" + term + ", version=" + version + ", clusterName=" + clusterName + ", clusterUUID=" + clusterUUID
48-
+ ", sourceNode=" + sourceNode + ", manifestFile=" + manifestFile + '}';
58+
return "RemotePublishRequest{"
59+
+ "term="
60+
+ term
61+
+ ", version="
62+
+ version
63+
+ ", clusterName="
64+
+ clusterName
65+
+ ", clusterUUID="
66+
+ clusterUUID
67+
+ ", sourceNode="
68+
+ sourceNode
69+
+ ", manifestFile="
70+
+ manifestFile
71+
+ '}';
4972
}
5073

5174
public String getClusterName() {

server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java

-2
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,6 @@
3333
package org.opensearch.cluster.node;
3434

3535
import org.opensearch.Version;
36-
import org.opensearch.cluster.metadata.Metadata;
3736
import org.opensearch.common.UUIDs;
3837
import org.opensearch.common.annotation.PublicApi;
3938
import org.opensearch.common.settings.Setting;
@@ -44,7 +43,6 @@
4443
import org.opensearch.core.common.transport.TransportAddress;
4544
import org.opensearch.core.xcontent.ToXContentFragment;
4645
import org.opensearch.core.xcontent.XContentBuilder;
47-
import org.opensearch.core.xcontent.XContentParser;
4846
import org.opensearch.node.Node;
4947

5048
import java.io.IOException;

server/src/main/java/org/opensearch/common/settings/ClusterSettings.java

+3-1
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,6 @@
7777
import org.opensearch.cluster.routing.allocation.decider.SameShardAllocationDecider;
7878
import org.opensearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
7979
import org.opensearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
80-
import org.opensearch.cluster.routing.remote.RemoteRoutingTableService;
8180
import org.opensearch.cluster.service.ClusterApplierService;
8281
import org.opensearch.cluster.service.ClusterManagerService;
8382
import org.opensearch.cluster.service.ClusterManagerTaskThrottler;
@@ -716,6 +715,9 @@ public void apply(Settings value, Settings current, Settings previous) {
716715
// Remote cluster state settings
717716
RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING,
718717
RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING,
718+
RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING,
719+
RemoteClusterStateService.GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING,
720+
RemoteClusterStateService.METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING,
719721
RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING,
720722
RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING,
721723
IndicesService.CLUSTER_REMOTE_INDEX_RESTRICT_ASYNC_DURABILITY_SETTING,

server/src/main/java/org/opensearch/gateway/GatewayMetaState.java

+7-2
Original file line numberDiff line numberDiff line change
@@ -722,9 +722,14 @@ public void setLastAcceptedState(ClusterState clusterState) {
722722
} else {
723723
assert verifyManifestAndClusterState(lastAcceptedManifest, lastAcceptedState) == true
724724
: "Previous manifest and previous ClusterState are not in sync";
725-
manifestDetails = remoteClusterStateService.writeIncrementalMetadata(lastAcceptedState, clusterState, lastAcceptedManifest);
725+
manifestDetails = remoteClusterStateService.writeIncrementalMetadata(
726+
lastAcceptedState,
727+
clusterState,
728+
lastAcceptedManifest
729+
);
726730
}
727-
assert verifyManifestAndClusterState(manifestDetails.getClusterMetadataManifest(), clusterState) == true : "Manifest and ClusterState are not in sync";
731+
assert verifyManifestAndClusterState(manifestDetails.getClusterMetadataManifest(), clusterState) == true
732+
: "Manifest and ClusterState are not in sync";
728733
lastAcceptedManifest = manifestDetails.getClusterMetadataManifest();
729734
lastAcceptedState = clusterState;
730735
lastUploadedManifestFile = manifestDetails.getManifestFileName();

server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java

+12
Original file line numberDiff line numberDiff line change
@@ -332,6 +332,11 @@ public Map<String, UploadedMetadataAttribute> getCustomMetadataMap() {
332332
return uploadedCustomMetadataMap;
333333
}
334334

335+
// TODO https://github.com/opensearch-project/OpenSearch/pull/14089
336+
public ClusterStateDiffManifest getDiffManifest() {
337+
return new ClusterStateDiffManifest();
338+
}
339+
335340
public boolean hasMetadataAttributesFiles() {
336341
return uploadedCoordinationMetadata != null
337342
|| uploadedSettingsMetadata != null
@@ -991,4 +996,11 @@ public String toString() {
991996
+ '}';
992997
}
993998
}
999+
1000+
// TODO https://github.com/opensearch-project/OpenSearch/pull/14089
1001+
public static class ClusterStateDiffManifest {
1002+
public String getFromStateUUID() {
1003+
return null;
1004+
}
1005+
}
9941006
}

server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java

+25
Original file line numberDiff line numberDiff line change
@@ -1092,6 +1092,31 @@ public ClusterState getLatestClusterState(String clusterName, String clusterUUID
10921092
.build();
10931093
}
10941094

1095+
public ClusterState getClusterStateForManifest(
1096+
String clusterName,
1097+
ClusterMetadataManifest manifest,
1098+
String localNodeId,
1099+
boolean includeEphemeral
1100+
) {
1101+
// TODO https://github.com/opensearch-project/OpenSearch/pull/14089
1102+
return null;
1103+
}
1104+
1105+
public ClusterState getClusterStateUsingDiff(
1106+
String clusterName,
1107+
ClusterMetadataManifest manifest,
1108+
ClusterState previousClusterState,
1109+
String localNodeId
1110+
) {
1111+
// TODO https://github.com/opensearch-project/OpenSearch/pull/14089
1112+
return null;
1113+
}
1114+
1115+
public ClusterMetadataManifest getClusterMetadataManifestByFileName(String clusterUUID, String manifestFileName) {
1116+
// TODO https://github.com/opensearch-project/OpenSearch/pull/14089
1117+
return null;
1118+
}
1119+
10951120
private Metadata getGlobalMetadata(String clusterName, String clusterUUID, ClusterMetadataManifest clusterMetadataManifest) {
10961121
String globalMetadataFileName = clusterMetadataManifest.getGlobalMetadataFileName();
10971122
try {

server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java

+2-1
Original file line numberDiff line numberDiff line change
@@ -202,7 +202,8 @@ private static boolean isRemoteRoutingTableAttributePresent(Settings settings) {
202202
}
203203

204204
public static boolean isRemotePublicationEnabled(Settings settings) {
205-
return FeatureFlags.isEnabled(REMOTE_PUBLICATION_EXPERIMENTAL) && isRemoteRoutingTableAttributePresent(settings)
205+
return FeatureFlags.isEnabled(REMOTE_PUBLICATION_EXPERIMENTAL)
206+
&& isRemoteRoutingTableAttributePresent(settings)
206207
&& isRemoteStoreClusterStateEnabled(settings);
207208
}
208209

0 commit comments

Comments
 (0)