Browse Source

Reduce use of deprecated Metadata builder method (#124290)

This removes all non-test usage of

     Metadata.Builder.put(IndexMetadata.Builder)

And replaces it with appropriate calls to the equivalent method on
`ProjectMetadata.Builder`

In most cases this _does not_ make the code project aware, but does
reduce the number of deprecated methods in use
Tim Vernum 6 months ago
parent
commit
a0dd4e76ec

+ 9 - 5
benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/AllocationBenchmark.java

@@ -14,7 +14,10 @@ import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.metadata.ProjectId;
+import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
+import org.elasticsearch.cluster.routing.GlobalRoutingTable;
 import org.elasticsearch.cluster.routing.RoutingTable;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.allocation.AllocationService;
@@ -126,19 +129,20 @@ public class AllocationBenchmark {
             Settings.builder().put("cluster.routing.allocation.awareness.attributes", "tag").build()
         );
 
-        Metadata.Builder mb = Metadata.builder();
+        final ProjectId projectId = ProjectId.DEFAULT;
+        ProjectMetadata.Builder pmb = ProjectMetadata.builder(projectId);
         for (int i = 1; i <= numIndices; i++) {
-            mb.put(
+            pmb.put(
                 IndexMetadata.builder("test_" + i)
                     .settings(Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, IndexVersion.current()))
                     .numberOfShards(numShards)
                     .numberOfReplicas(numReplicas)
             );
         }
-        Metadata metadata = mb.build();
+        Metadata metadata = Metadata.builder().put(pmb).build();
         RoutingTable.Builder rb = RoutingTable.builder(TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY);
         for (int i = 1; i <= numIndices; i++) {
-            rb.addAsNew(metadata.getProject().index("test_" + i));
+            rb.addAsNew(metadata.getProject(projectId).index("test_" + i));
         }
         RoutingTable routingTable = rb.build();
         DiscoveryNodes.Builder nb = DiscoveryNodes.builder();
@@ -151,7 +155,7 @@ public class AllocationBenchmark {
         }
         initialClusterState = ClusterState.builder(ClusterName.DEFAULT)
             .metadata(metadata)
-            .routingTable(routingTable)
+            .routingTable(GlobalRoutingTable.builder().put(projectId, routingTable).build())
             .nodes(nb)
             .nodeIdsToCompatibilityVersions(compatibilityVersions)
             .build();

+ 4 - 0
server/src/main/java/org/elasticsearch/cluster/ClusterState.java

@@ -983,6 +983,10 @@ public class ClusterState implements ChunkedToXContent, Diffable<ClusterState> {
         return copyAndUpdate(builder -> builder.metadata(metadata().copyAndUpdate(updater)));
     }
 
+    public ClusterState copyAndUpdateProject(ProjectId projectId, Consumer<ProjectMetadata.Builder> updater) {
+        return copyAndUpdate(builder -> builder.putProjectMetadata(metadata().getProject(projectId).copyAndUpdate(updater)));
+    }
+
     @SuppressForbidden(reason = "directly reading ClusterState#clusterFeatures")
     private static Map<String, Set<String>> getNodeFeatures(ClusterFeatures features) {
         return features.nodeFeatures();

+ 6 - 5
server/src/main/java/org/elasticsearch/cluster/coordination/RemoveIndexSettingsCommand.java

@@ -16,7 +16,7 @@ import org.elasticsearch.cli.Terminal;
 import org.elasticsearch.cli.UserException;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
-import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.common.regex.Regex;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.Tuple;
@@ -58,9 +58,10 @@ public class RemoveIndexSettingsCommand extends ElasticsearchNodeCommand {
         terminal.println(Terminal.Verbosity.VERBOSE, "Loading cluster state");
         final Tuple<Long, ClusterState> termAndClusterState = loadTermAndClusterState(persistedClusterStateService, env);
         final ClusterState oldClusterState = termAndClusterState.v2();
-        final Metadata.Builder newMetadataBuilder = Metadata.builder(oldClusterState.metadata());
+        final ProjectMetadata oldProject = oldClusterState.metadata().getProject();
+        final ProjectMetadata.Builder newProjectBuilder = ProjectMetadata.builder(oldProject);
         int changes = 0;
-        for (IndexMetadata indexMetadata : oldClusterState.metadata().getProject()) {
+        for (IndexMetadata indexMetadata : oldProject) {
             Settings oldSettings = indexMetadata.getSettings();
             Settings.Builder newSettings = Settings.builder().put(oldSettings);
             boolean removed = false;
@@ -76,7 +77,7 @@ public class RemoveIndexSettingsCommand extends ElasticsearchNodeCommand {
                 }
             }
             if (removed) {
-                newMetadataBuilder.put(IndexMetadata.builder(indexMetadata).settings(newSettings));
+                newProjectBuilder.put(IndexMetadata.builder(indexMetadata).settings(newSettings));
                 changes++;
             }
         }
@@ -84,7 +85,7 @@ public class RemoveIndexSettingsCommand extends ElasticsearchNodeCommand {
             throw new UserException(ExitCodes.USAGE, "No index setting matching " + settingsToRemove + " were found on this node");
         }
 
-        final ClusterState newClusterState = ClusterState.builder(oldClusterState).metadata(newMetadataBuilder).build();
+        final ClusterState newClusterState = ClusterState.builder(oldClusterState).putProjectMetadata(newProjectBuilder).build();
         terminal.println(
             Terminal.Verbosity.VERBOSE,
             "[old cluster state = " + oldClusterState + ", new cluster state = " + newClusterState + "]"

+ 5 - 1
server/src/main/java/org/elasticsearch/cluster/coordination/UnsafeBootstrapMasterCommand.java

@@ -15,6 +15,7 @@ import org.elasticsearch.cli.Terminal;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.UUIDs;
@@ -104,8 +105,10 @@ public class UnsafeBootstrapMasterCommand extends ElasticsearchNodeCommand {
             .clusterUUIDCommitted(true)
             .persistentSettings(persistentSettings)
             .coordinationMetadata(newCoordinationMetadata);
+
+        final ProjectMetadata.Builder newProject = ProjectMetadata.builder(metadata.getProject());
         for (IndexMetadata indexMetadata : metadata.getProject().indices().values()) {
-            newMetadata.put(
+            newProject.put(
                 IndexMetadata.builder(indexMetadata)
                     .settings(
                         Settings.builder()
@@ -114,6 +117,7 @@ public class UnsafeBootstrapMasterCommand extends ElasticsearchNodeCommand {
                     )
             );
         }
+        newMetadata.put(newProject);
 
         final ClusterState newClusterState = ClusterState.builder(oldClusterState).metadata(newMetadata).build();
 

+ 7 - 0
server/src/main/java/org/elasticsearch/cluster/metadata/ProjectMetadata.java

@@ -66,6 +66,7 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.function.BiConsumer;
 import java.util.function.BiPredicate;
+import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
@@ -1117,6 +1118,12 @@ public class ProjectMetadata implements Iterable<IndexMetadata>, Diffable<Projec
         return new ProjectMetadata.Builder(projectMetadata);
     }
 
+    public ProjectMetadata copyAndUpdate(Consumer<Builder> updater) {
+        var builder = builder(this);
+        updater.accept(builder);
+        return builder.build();
+    }
+
     public static class Builder {
 
         private final ImmutableOpenMap.Builder<String, IndexMetadata> indices;

+ 49 - 17
server/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java

@@ -29,6 +29,7 @@ import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodeUtils;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
+import org.elasticsearch.cluster.routing.GlobalRoutingTable;
 import org.elasticsearch.cluster.routing.GlobalRoutingTableTestHelper;
 import org.elasticsearch.cluster.routing.IndexRoutingTable;
 import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
@@ -149,6 +150,24 @@ public class ClusterStateTests extends ESTestCase {
         assertThat(copy.metadata().clusterUUID(), equalTo(newClusterUuid));
     }
 
+    public void testCopyAndUpdateProject() throws IOException {
+        var projectId = randomProjectIdOrDefault();
+        var state = buildClusterState(projectId);
+        var indexName = getTestName();
+
+        assertThat(state.metadata().getProject(projectId).hasIndex(indexName), equalTo(false));
+
+        var copy = state.copyAndUpdateProject(
+            projectId,
+            project -> project.put(IndexMetadata.builder(indexName).settings(indexSettings(IndexVersion.current(), randomUUID(), 1, 1)))
+        );
+
+        assertThat(copy, not(sameInstance(state)));
+        assertThat(copy.metadata(), not(sameInstance(state.metadata())));
+        assertThat(copy.metadata().getProject(projectId), not(sameInstance(state.metadata().getProject(projectId))));
+        assertThat(copy.metadata().getProject(projectId).hasIndex(indexName), equalTo(true));
+    }
+
     public void testGetNonExistingProjectStateThrows() {
         final List<ProjectMetadata> projects = IntStream.range(0, between(1, 3))
             .mapToObj(i -> MetadataTests.randomProject(ProjectId.fromId("p_" + i), between(0, 5)))
@@ -1862,6 +1881,10 @@ public class ClusterStateTests extends ESTestCase {
     }
 
     private ClusterState buildClusterState() throws IOException {
+        return buildClusterState(ProjectId.DEFAULT);
+    }
+
+    private static ClusterState buildClusterState(ProjectId projectId) throws IOException {
         IndexMetadata indexMetadata = IndexMetadata.builder("index")
             .state(IndexMetadata.State.OPEN)
             .settings(Settings.builder().put(SETTING_VERSION_CREATED, IndexVersion.current()))
@@ -1925,29 +1948,38 @@ public class ClusterStateTests extends ESTestCase {
                     )
                     .persistentSettings(Settings.builder().put(SETTING_VERSION_CREATED, IndexVersion.current()).build())
                     .transientSettings(Settings.builder().put(SETTING_VERSION_CREATED, IndexVersion.current()).build())
-                    .put(indexMetadata, false)
                     .put(
-                        IndexTemplateMetadata.builder("template")
-                            .patterns(List.of("pattern1", "pattern2"))
-                            .order(0)
-                            .settings(Settings.builder().put(SETTING_VERSION_CREATED, IndexVersion.current()))
-                            .putMapping("type", "{ \"key1\": {} }")
+                        ProjectMetadata.builder(projectId)
+                            .put(indexMetadata, false)
+                            .put(
+                                IndexTemplateMetadata.builder("template")
+                                    .patterns(List.of("pattern1", "pattern2"))
+                                    .order(0)
+                                    .settings(Settings.builder().put(SETTING_VERSION_CREATED, IndexVersion.current()))
+                                    .putMapping("type", "{ \"key1\": {} }")
+                                    .build()
+                            )
                             .build()
                     )
             )
             .routingTable(
-                RoutingTable.builder()
-                    .add(
-                        IndexRoutingTable.builder(new Index("index", "indexUUID"))
-                            .addIndexShard(
-                                new IndexShardRoutingTable.Builder(new ShardId("index", "indexUUID", 0)).addShard(
-                                    TestShardRouting.newShardRouting(
-                                        new ShardId("index", "indexUUID", 0),
-                                        "nodeId2",
-                                        true,
-                                        ShardRoutingState.STARTED
+                GlobalRoutingTable.builder()
+                    .put(
+                        projectId,
+                        RoutingTable.builder()
+                            .add(
+                                IndexRoutingTable.builder(new Index("index", "indexUUID"))
+                                    .addIndexShard(
+                                        new IndexShardRoutingTable.Builder(new ShardId("index", "indexUUID", 0)).addShard(
+                                            TestShardRouting.newShardRouting(
+                                                new ShardId("index", "indexUUID", 0),
+                                                "nodeId2",
+                                                true,
+                                                ShardRoutingState.STARTED
+                                            )
+                                        )
                                     )
-                                )
+                                    .build()
                             )
                             .build()
                     )

+ 20 - 0
server/src/test/java/org/elasticsearch/cluster/metadata/ProjectMetadataTests.java

@@ -29,6 +29,9 @@ import static org.elasticsearch.cluster.metadata.MetadataTests.checkChunkSize;
 import static org.elasticsearch.cluster.metadata.MetadataTests.count;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
 import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.sameInstance;
 
 public class ProjectMetadataTests extends ESTestCase {
 
@@ -384,4 +387,21 @@ public class ProjectMetadataTests extends ESTestCase {
         return Math.toIntExact(chunkCount);
     }
 
+    public void testCopyAndUpdate() {
+        var initialIndexUUID = randomUUID();
+        final String indexName = randomAlphaOfLengthBetween(4, 12);
+        final ProjectMetadata before = ProjectMetadata.builder(randomProjectIdOrDefault())
+            .put(IndexMetadata.builder(indexName).settings(indexSettings(IndexVersion.current(), initialIndexUUID, 1, 1)))
+            .build();
+
+        var alteredIndexUUID = randomUUID();
+        assertThat(alteredIndexUUID, not(equalTo(initialIndexUUID)));
+        final ProjectMetadata after = before.copyAndUpdate(
+            builder -> builder.put(IndexMetadata.builder(indexName).settings(indexSettings(IndexVersion.current(), alteredIndexUUID, 1, 1)))
+        );
+
+        assertThat(after, not(sameInstance(before)));
+        assertThat(after.index(indexName).getIndexUUID(), equalTo(alteredIndexUUID));
+    }
+
 }

+ 5 - 1
x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowAction.java

@@ -26,10 +26,12 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
+import org.elasticsearch.cluster.metadata.ProjectId;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
+import org.elasticsearch.core.FixForMultiProject;
 import org.elasticsearch.core.SuppressForbidden;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.IndexNotFoundException;
@@ -287,6 +289,8 @@ public class TransportUnfollowAction extends AcknowledgedTransportMasterNodeActi
         // Remove ccr custom metadata
         newIndexMetadata.removeCustom(Ccr.CCR_CUSTOM_METADATA_KEY);
 
-        return current.copyAndUpdateMetadata(metadata -> metadata.put(newIndexMetadata));
+        @FixForMultiProject
+        final ProjectId projectId = current.metadata().getProject().id();
+        return current.copyAndUpdateProject(projectId, project -> project.put(newIndexMetadata));
     }
 }

+ 7 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotRepository.java

@@ -20,6 +20,7 @@ import org.apache.lucene.store.NIOFSDirectory;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.MappingMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.cluster.metadata.RepositoryMetadata;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.lucene.Lucene;
@@ -110,9 +111,13 @@ public final class SourceOnlySnapshotRepository extends FilterRepository {
     }
 
     private static Metadata metadataToSnapshot(Collection<IndexId> indices, Metadata metadata) {
-        Metadata.Builder builder = Metadata.builder(metadata);
+        return Metadata.builder(metadata).put(projectMetadataToSnapshot(indices, metadata.getProject())).build();
+    }
+
+    private static ProjectMetadata projectMetadataToSnapshot(Collection<IndexId> indices, ProjectMetadata projectMetadata) {
+        ProjectMetadata.Builder builder = ProjectMetadata.builder(projectMetadata);
         for (IndexId indexId : indices) {
-            IndexMetadata index = metadata.getProject().index(indexId.getName());
+            IndexMetadata index = projectMetadata.index(indexId.getName());
             IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(index);
             // for a minimal restore we basically disable indexing on all fields and only create an index
             // that is valid from an operational perspective. ie. it will have all metadata fields like version/

+ 3 - 3
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/CopySettingsStep.java

@@ -11,7 +11,7 @@ import org.apache.logging.log4j.Logger;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
-import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.Index;
@@ -94,11 +94,11 @@ public class CopySettingsStep extends ClusterStateActionStep {
             settings.put(key, value);
         }
 
-        Metadata.Builder newMetaData = Metadata.builder(clusterState.getMetadata())
+        ProjectMetadata.Builder newProject = ProjectMetadata.builder(clusterState.getMetadata().getProject())
             .put(
                 IndexMetadata.builder(targetIndexMetadata).settingsVersion(targetIndexMetadata.getSettingsVersion() + 1).settings(settings)
             );
-        return ClusterState.builder(clusterState).metadata(newMetaData).build();
+        return ClusterState.builder(clusterState).putProjectMetadata(newProject).build();
     }
 
     @Override

+ 4 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/InitializePolicyContextStep.java

@@ -11,7 +11,7 @@ import org.apache.logging.log4j.Logger;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
-import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.IndexSettings;
@@ -79,7 +79,9 @@ public final class InitializePolicyContextStep extends ClusterStateActionStep {
                         .build()
                 );
             builder.putCustom(ILM_CUSTOM_METADATA_KEY, newLifecycleState.asMap());
-            return ClusterState.builder(clusterState).metadata(Metadata.builder(clusterState.metadata()).put(builder).build()).build();
+            return ClusterState.builder(clusterState)
+                .putProjectMetadata(ProjectMetadata.builder(clusterState.metadata().getProject()).put(builder).build())
+                .build();
         }
     }
 

+ 30 - 13
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycleTransition.java

@@ -15,9 +15,10 @@ import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
-import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.core.Nullable;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.license.XPackLicenseState;
 import org.elasticsearch.xcontent.ToXContentObject;
@@ -443,28 +444,44 @@ public final class IndexLifecycleTransition {
      * any lifecycle execution state that may be present in the index metadata
      */
     public static ClusterState removePolicyForIndexes(final Index[] indices, ClusterState currentState, List<String> failedIndexes) {
-        Metadata.Builder newMetadata = Metadata.builder(currentState.getMetadata());
-        boolean clusterStateChanged = false;
+        final ProjectMetadata currentProject = currentState.metadata().getProject();
+        final ProjectMetadata.Builder updatedProject = removePolicyForIndexes(indices, currentProject, failedIndexes);
+
+        if (updatedProject == null) {
+            return currentState;
+        } else {
+            return ClusterState.builder(currentState).putProjectMetadata(updatedProject).build();
+        }
+    }
+
+    /**
+     * @return If one or more policies were removed, then a new builder representing the changed project state.
+     *         Otherwise {@code null} (if no changes were made)
+     */
+    @Nullable
+    private static ProjectMetadata.Builder removePolicyForIndexes(
+        final Index[] indices,
+        ProjectMetadata currentProject,
+        List<String> failedIndexes
+    ) {
+        ProjectMetadata.Builder newProject = null;
         for (Index index : indices) {
-            IndexMetadata indexMetadata = currentState.getMetadata().getProject().index(index);
+            IndexMetadata indexMetadata = currentProject.index(index);
             if (indexMetadata == null) {
                 // Index doesn't exist so fail it
                 failedIndexes.add(index.getName());
             } else {
                 IndexMetadata.Builder newIdxMetadata = removePolicyForIndex(indexMetadata);
                 if (newIdxMetadata != null) {
-                    newMetadata.put(newIdxMetadata);
-                    clusterStateChanged = true;
+                    if (newProject == null) {
+                        newProject = ProjectMetadata.builder(currentProject);
+                    }
+                    newProject.put(newIdxMetadata);
                 }
             }
         }
-        if (clusterStateChanged) {
-            ClusterState.Builder newClusterState = ClusterState.builder(currentState);
-            newClusterState.metadata(newMetadata);
-            return newClusterState.build();
-        } else {
-            return currentState;
-        }
+
+        return newProject;
     }
 
     /**