From 478c253f8929682675298cd9e491963090b897b8 Mon Sep 17 00:00:00 2001 From: Igor Motov Date: Sun, 26 Apr 2015 21:04:41 -0400 Subject: [PATCH] Add support for cluster state diffs Adds support for calculating and sending diffs instead of full cluster state of the most frequently changing elements - cluster state, meta data and routing table. Closes #6295 --- .../reroute/ClusterRerouteResponse.java | 2 +- .../cluster/state/ClusterStateResponse.java | 2 +- .../state/TransportClusterStateAction.java | 15 +- .../indices/alias/get/GetAliasesResponse.java | 2 +- .../indices/create/CreateIndexRequest.java | 10 +- .../admin/indices/get/GetIndexResponse.java | 6 +- .../mapping/get/GetMappingsResponse.java | 4 +- .../get/GetIndexTemplatesResponse.java | 2 +- .../template/put/PutIndexTemplateRequest.java | 10 +- .../cluster/AbstractDiffable.java | 108 +++ .../elasticsearch/cluster/ClusterState.java | 272 ++++++-- .../java/org/elasticsearch/cluster/Diff.java | 42 ++ .../org/elasticsearch/cluster/Diffable.java | 42 ++ .../elasticsearch/cluster/DiffableUtils.java | 283 ++++++++ ...ompatibleClusterStateVersionException.java | 35 + .../cluster/block/ClusterBlocks.java | 75 ++- .../cluster/metadata/AliasMetaData.java | 85 ++- .../cluster/metadata/IndexMetaData.java | 234 ++++--- .../metadata/IndexTemplateMetaData.java | 105 +-- .../cluster/metadata/MappingMetaData.java | 48 +- .../cluster/metadata/MetaData.java | 266 +++++--- .../metadata/MetaDataCreateIndexService.java | 2 +- .../metadata/RepositoriesMetaData.java | 236 +++---- .../cluster/metadata/RepositoryMetaData.java | 21 + .../cluster/metadata/RestoreMetaData.java | 220 +++--- .../cluster/metadata/SnapshotMetaData.java | 223 ++++--- .../cluster/node/DiscoveryNodes.java | 73 +- .../cluster/routing/IndexRoutingTable.java | 72 +- .../routing/IndexShardRoutingTable.java | 22 + .../cluster/routing/RoutingTable.java | 90 ++- .../service/InternalClusterService.java | 8 +- .../ClusterDynamicSettingsModule.java | 1 + .../common/io/stream/StreamableReader.java | 30 + .../common/io/stream/Writeable.java | 30 + .../elasticsearch/discovery/Discovery.java | 3 +- .../discovery/DiscoveryService.java | 5 +- .../discovery/DiscoverySettings.java | 13 + .../discovery/local/LocalDiscovery.java | 46 +- .../discovery/zen/ZenDiscovery.java | 9 +- .../publish/PublishClusterStateAction.java | 194 ++++-- .../org/elasticsearch/gateway/Gateway.java | 2 +- .../gateway/LocalAllocateDangledIndices.java | 2 +- .../TransportNodesListGatewayMetaState.java | 2 +- .../get/RestGetRepositoriesAction.java | 2 +- .../indices/get/RestGetIndicesAction.java | 2 +- .../warmer/get/RestGetWarmerAction.java | 2 +- .../search/warmer/IndexWarmersMetaData.java | 318 +++++---- .../ClusterStateDiffPublishingTests.java | 625 ++++++++++++++++++ .../cluster/ClusterStateDiffTests.java | 534 +++++++++++++++ .../ClusterSerializationTests.java | 2 +- .../cluster/serialization/DiffableTests.java | 127 ++++ .../common/xcontent/XContentTestUtils.java | 100 +++ .../discovery/ZenUnicastDiscoveryTests.java | 1 + .../discovery/zen/ZenDiscoveryTests.java | 10 +- .../timestamp/TimestampMappingTests.java | 12 +- .../store/IndicesStoreIntegrationTests.java | 7 + .../template/SimpleIndexTemplateTests.java | 1 + .../DedicatedClusterSnapshotRestoreTests.java | 218 +++--- .../test/ElasticsearchIntegrationTest.java | 38 +- .../test/ElasticsearchTestCase.java | 14 + 60 files changed, 3831 insertions(+), 1134 deletions(-) create mode 100644 src/main/java/org/elasticsearch/cluster/AbstractDiffable.java create mode 100644 src/main/java/org/elasticsearch/cluster/Diff.java create mode 100644 src/main/java/org/elasticsearch/cluster/Diffable.java create mode 100644 src/main/java/org/elasticsearch/cluster/DiffableUtils.java create mode 100644 src/main/java/org/elasticsearch/cluster/IncompatibleClusterStateVersionException.java create mode 100644 src/main/java/org/elasticsearch/common/io/stream/StreamableReader.java create mode 100644 src/main/java/org/elasticsearch/common/io/stream/Writeable.java create mode 100644 src/test/java/org/elasticsearch/cluster/ClusterStateDiffPublishingTests.java create mode 100644 src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java create mode 100644 src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java create mode 100644 src/test/java/org/elasticsearch/common/xcontent/XContentTestUtils.java diff --git a/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java b/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java index 79b31f620d5..28f9cb1db90 100644 --- a/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java @@ -68,7 +68,7 @@ public class ClusterRerouteResponse extends AcknowledgedResponse { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - ClusterState.Builder.writeTo(state, out); + state.writeTo(out); writeAcknowledged(out); RoutingExplanations.writeTo(explanations, out); } diff --git a/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateResponse.java b/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateResponse.java index 861a84a9e71..e9aa9b723fa 100644 --- a/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateResponse.java @@ -62,6 +62,6 @@ public class ClusterStateResponse extends ActionResponse { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); clusterName.writeTo(out); - ClusterState.Builder.writeTo(clusterState, out); + clusterState.writeTo(out); } } diff --git a/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java b/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java index 7b114c92d43..5c8905fd97b 100644 --- a/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java +++ b/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java @@ -19,7 +19,6 @@ package org.elasticsearch.action.admin.cluster.state; -import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; @@ -29,7 +28,6 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; -import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData.Custom; @@ -39,11 +37,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import java.util.List; - -import static com.google.common.collect.Lists.newArrayList; -import static org.elasticsearch.cluster.metadata.MetaData.lookupFactorySafe; - /** * */ @@ -84,6 +77,7 @@ public class TransportClusterStateAction extends TransportMasterNodeReadOperatio logger.trace("Serving cluster state request using version {}", currentState.version()); ClusterState.Builder builder = ClusterState.builder(currentState.getClusterName()); builder.version(currentState.version()); + builder.uuid(currentState.uuid()); if (request.nodes()) { builder.nodes(currentState.nodes()); } @@ -122,10 +116,9 @@ public class TransportClusterStateAction extends TransportMasterNodeReadOperatio } // Filter our metadata that shouldn't be returned by API - for(ObjectCursor type : currentState.metaData().customs().keys()) { - Custom.Factory factory = lookupFactorySafe(type.value); - if(!factory.context().contains(MetaData.XContentContext.API)) { - mdBuilder.removeCustom(type.value); + for(ObjectObjectCursor custom : currentState.metaData().customs()) { + if(!custom.value.context().contains(MetaData.XContentContext.API)) { + mdBuilder.removeCustom(custom.key); } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesResponse.java index 765a9395afc..106e864a367 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesResponse.java @@ -74,7 +74,7 @@ public class GetAliasesResponse extends ActionResponse { out.writeString(entry.key); out.writeVInt(entry.value.size()); for (AliasMetaData aliasMetaData : entry.value) { - AliasMetaData.Builder.writeTo(aliasMetaData, out); + aliasMetaData.writeTo(out); } } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java index d79c2128611..60a265de785 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java @@ -396,11 +396,11 @@ public class CreateIndexRequest extends AcknowledgedRequest aliases((Map) entry.getValue()); } else { // maybe custom? - IndexMetaData.Custom.Factory factory = IndexMetaData.lookupFactory(name); - if (factory != null) { + IndexMetaData.Custom proto = IndexMetaData.lookupPrototype(name); + if (proto != null) { found = true; try { - customs.put(name, factory.fromMap((Map) entry.getValue())); + customs.put(name, proto.fromMap((Map) entry.getValue())); } catch (IOException e) { throw new ElasticsearchParseException("failed to parse custom metadata for [" + name + "]"); } @@ -448,7 +448,7 @@ public class CreateIndexRequest extends AcknowledgedRequest int customSize = in.readVInt(); for (int i = 0; i < customSize; i++) { String type = in.readString(); - IndexMetaData.Custom customIndexMetaData = IndexMetaData.lookupFactorySafe(type).readFrom(in); + IndexMetaData.Custom customIndexMetaData = IndexMetaData.lookupPrototypeSafe(type).readFrom(in); customs.put(type, customIndexMetaData); } int aliasesSize = in.readVInt(); @@ -472,7 +472,7 @@ public class CreateIndexRequest extends AcknowledgedRequest out.writeVInt(customs.size()); for (Map.Entry entry : customs.entrySet()) { out.writeString(entry.getKey()); - IndexMetaData.lookupFactorySafe(entry.getKey()).writeTo(entry.getValue(), out); + entry.getValue().writeTo(out); } out.writeVInt(aliases.size()); for (Alias alias : aliases) { diff --git a/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java index 35e6cfa4804..7080a694a11 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java @@ -134,7 +134,7 @@ public class GetIndexResponse extends ActionResponse { int valueSize = in.readVInt(); ImmutableOpenMap.Builder mappingEntryBuilder = ImmutableOpenMap.builder(); for (int j = 0; j < valueSize; j++) { - mappingEntryBuilder.put(in.readString(), MappingMetaData.readFrom(in)); + mappingEntryBuilder.put(in.readString(), MappingMetaData.PROTO.readFrom(in)); } mappingsMapBuilder.put(key, mappingEntryBuilder.build()); } @@ -181,7 +181,7 @@ public class GetIndexResponse extends ActionResponse { out.writeVInt(indexEntry.value.size()); for (ObjectObjectCursor mappingEntry : indexEntry.value) { out.writeString(mappingEntry.key); - MappingMetaData.writeTo(mappingEntry.value, out); + mappingEntry.value.writeTo(out); } } out.writeVInt(aliases.size()); @@ -189,7 +189,7 @@ public class GetIndexResponse extends ActionResponse { out.writeString(indexEntry.key); out.writeVInt(indexEntry.value.size()); for (AliasMetaData aliasEntry : indexEntry.value) { - AliasMetaData.Builder.writeTo(aliasEntry, out); + aliasEntry.writeTo(out); } } out.writeVInt(settings.size()); diff --git a/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java index b27577f8da3..30e9e24c493 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java @@ -59,7 +59,7 @@ public class GetMappingsResponse extends ActionResponse { int valueSize = in.readVInt(); ImmutableOpenMap.Builder typeMapBuilder = ImmutableOpenMap.builder(); for (int j = 0; j < valueSize; j++) { - typeMapBuilder.put(in.readString(), MappingMetaData.readFrom(in)); + typeMapBuilder.put(in.readString(), MappingMetaData.PROTO.readFrom(in)); } indexMapBuilder.put(key, typeMapBuilder.build()); } @@ -75,7 +75,7 @@ public class GetMappingsResponse extends ActionResponse { out.writeVInt(indexEntry.value.size()); for (ObjectObjectCursor typeEntry : indexEntry.value) { out.writeString(typeEntry.key); - MappingMetaData.writeTo(typeEntry.value, out); + typeEntry.value.writeTo(out); } } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java index 56de19872f2..2ce6d8d2c1a 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java @@ -60,7 +60,7 @@ public class GetIndexTemplatesResponse extends ActionResponse { super.writeTo(out); out.writeVInt(indexTemplates.size()); for (IndexTemplateMetaData indexTemplate : indexTemplates) { - IndexTemplateMetaData.Builder.writeTo(indexTemplate, out); + indexTemplate.writeTo(out); } } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java index 41dd9ec2b45..1b752855c20 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java @@ -292,10 +292,10 @@ public class PutIndexTemplateRequest extends MasterNodeOperationRequest) entry.getValue()); } else { // maybe custom? - IndexMetaData.Custom.Factory factory = IndexMetaData.lookupFactory(name); - if (factory != null) { + IndexMetaData.Custom proto = IndexMetaData.lookupPrototype(name); + if (proto != null) { try { - customs.put(name, factory.fromMap((Map) entry.getValue())); + customs.put(name, proto.fromMap((Map) entry.getValue())); } catch (IOException e) { throw new ElasticsearchParseException("failed to parse custom metadata for [" + name + "]"); } @@ -440,7 +440,7 @@ public class PutIndexTemplateRequest extends MasterNodeOperationRequest entry : customs.entrySet()) { out.writeString(entry.getKey()); - IndexMetaData.lookupFactorySafe(entry.getKey()).writeTo(entry.getValue(), out); + entry.getValue().writeTo(out); } out.writeVInt(aliases.size()); for (Alias alias : aliases) { diff --git a/src/main/java/org/elasticsearch/cluster/AbstractDiffable.java b/src/main/java/org/elasticsearch/cluster/AbstractDiffable.java new file mode 100644 index 00000000000..4e6da2bd569 --- /dev/null +++ b/src/main/java/org/elasticsearch/cluster/AbstractDiffable.java @@ -0,0 +1,108 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.io.stream.StreamableReader; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Abstract diffable object with simple diffs implementation that sends the entire object if object has changed or + * nothing is object remained the same. + */ +public abstract class AbstractDiffable> implements Diffable { + + @Override + public Diff diff(T previousState) { + if (this.get().equals(previousState)) { + return new CompleteDiff<>(); + } else { + return new CompleteDiff<>(get()); + } + } + + @Override + public Diff readDiffFrom(StreamInput in) throws IOException { + return new CompleteDiff<>(this, in); + } + + public static > Diff readDiffFrom(StreamableReader reader, StreamInput in) throws IOException { + return new CompleteDiff(reader, in); + } + + private static class CompleteDiff> implements Diff { + + @Nullable + private final T part; + + /** + * Creates simple diff with changes + */ + public CompleteDiff(T part) { + this.part = part; + } + + /** + * Creates simple diff without changes + */ + public CompleteDiff() { + this.part = null; + } + + /** + * Read simple diff from the stream + */ + public CompleteDiff(StreamableReader reader, StreamInput in) throws IOException { + if (in.readBoolean()) { + this.part = reader.readFrom(in); + } else { + this.part = null; + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + if (part != null) { + out.writeBoolean(true); + part.writeTo(out); + } else { + out.writeBoolean(false); + } + } + + @Override + public T apply(T part) { + if (this.part != null) { + return this.part; + } else { + return part; + } + } + } + + @SuppressWarnings("unchecked") + public T get() { + return (T) this; + } +} + diff --git a/src/main/java/org/elasticsearch/cluster/ClusterState.java b/src/main/java/org/elasticsearch/cluster/ClusterState.java index b90bc0bb2ac..4f63d9e00e3 100644 --- a/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -22,6 +22,7 @@ package org.elasticsearch.cluster; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.collect.ImmutableSet; +import org.elasticsearch.cluster.DiffableUtils.KeyedReader; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -55,7 +56,9 @@ import java.util.Map; /** * */ -public class ClusterState implements ToXContent { +public class ClusterState implements ToXContent, Diffable { + + public static final ClusterState PROTO = builder(ClusterName.DEFAULT).build(); public static enum ClusterStateStatus { UNKNOWN((byte) 0), @@ -74,47 +77,43 @@ public class ClusterState implements ToXContent { } } - public interface Custom { + public interface Custom extends Diffable, ToXContent { - interface Factory { - - String type(); - - T readFrom(StreamInput in) throws IOException; - - void writeTo(T customState, StreamOutput out) throws IOException; - - void toXContent(T customState, XContentBuilder builder, ToXContent.Params params); - } + String type(); } - private final static Map customFactories = new HashMap<>(); + private final static Map customPrototypes = new HashMap<>(); /** * Register a custom index meta data factory. Make sure to call it from a static block. */ - public static void registerFactory(String type, Custom.Factory factory) { - customFactories.put(type, factory); + public static void registerPrototype(String type, Custom proto) { + customPrototypes.put(type, proto); } @Nullable - public static Custom.Factory lookupFactory(String type) { - return customFactories.get(type); + public static T lookupPrototype(String type) { + //noinspection unchecked + return (T) customPrototypes.get(type); } - public static Custom.Factory lookupFactorySafe(String type) { - Custom.Factory factory = customFactories.get(type); - if (factory == null) { - throw new IllegalArgumentException("No custom state factory registered for type [" + type + "]"); + public static T lookupPrototypeSafe(String type) { + @SuppressWarnings("unchecked") + T proto = (T)customPrototypes.get(type); + if (proto == null) { + throw new IllegalArgumentException("No custom state prototype registered for type [" + type + "]"); } - return factory; + return proto; } + public static final String UNKNOWN_UUID = "_na_"; public static final long UNKNOWN_VERSION = -1; private final long version; + private final String uuid; + private final RoutingTable routingTable; private final DiscoveryNodes nodes; @@ -127,17 +126,20 @@ public class ClusterState implements ToXContent { private final ClusterName clusterName; + private final boolean wasReadFromDiff; + // built on demand private volatile RoutingNodes routingNodes; private volatile ClusterStateStatus status; - public ClusterState(long version, ClusterState state) { - this(state.clusterName, version, state.metaData(), state.routingTable(), state.nodes(), state.blocks(), state.customs()); + public ClusterState(long version, String uuid, ClusterState state) { + this(state.clusterName, version, uuid, state.metaData(), state.routingTable(), state.nodes(), state.blocks(), state.customs(), false); } - public ClusterState(ClusterName clusterName, long version, MetaData metaData, RoutingTable routingTable, DiscoveryNodes nodes, ClusterBlocks blocks, ImmutableOpenMap customs) { + public ClusterState(ClusterName clusterName, long version, String uuid, MetaData metaData, RoutingTable routingTable, DiscoveryNodes nodes, ClusterBlocks blocks, ImmutableOpenMap customs, boolean wasReadFromDiff) { this.version = version; + this.uuid = uuid; this.clusterName = clusterName; this.metaData = metaData; this.routingTable = routingTable; @@ -145,6 +147,7 @@ public class ClusterState implements ToXContent { this.blocks = blocks; this.customs = customs; this.status = ClusterStateStatus.UNKNOWN; + this.wasReadFromDiff = wasReadFromDiff; } public ClusterStateStatus status() { @@ -164,6 +167,14 @@ public class ClusterState implements ToXContent { return version(); } + /** + * This uuid is automatically generated for for each version of cluster state. It is used to make sure that + * we are applying diffs to the right previous state. + */ + public String uuid() { + return this.uuid; + } + public DiscoveryNodes nodes() { return this.nodes; } @@ -216,6 +227,11 @@ public class ClusterState implements ToXContent { return this.clusterName; } + // Used for testing and logging to determine how this cluster state was send over the wire + boolean wasReadFromDiff() { + return wasReadFromDiff; + } + /** * Returns a built (on demand) routing nodes view of the routing table. NOTE, the routing nodes * are mutable, use them just for read operations @@ -231,6 +247,8 @@ public class ClusterState implements ToXContent { public String prettyPrint() { StringBuilder sb = new StringBuilder(); sb.append("version: ").append(version).append("\n"); + sb.append("uuid: ").append(uuid).append("\n"); + sb.append("from_diff: ").append(wasReadFromDiff).append("\n"); sb.append("meta data version: ").append(metaData.version()).append("\n"); sb.append(nodes().prettyPrint()); sb.append(routingTable().prettyPrint()); @@ -302,14 +320,13 @@ public class ClusterState implements ToXContent { } } - - @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { EnumSet metrics = Metric.parseString(params.param("metric", "_all"), true); if (metrics.contains(Metric.VERSION)) { builder.field("version", version); + builder.field("uuid", uuid); } if (metrics.contains(Metric.MASTER_NODE)) { @@ -434,7 +451,7 @@ public class ClusterState implements ToXContent { for (ObjectObjectCursor cursor : metaData.customs()) { builder.startObject(cursor.key); - MetaData.lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); + cursor.value.toXContent(builder, params); builder.endObject(); } @@ -473,7 +490,7 @@ public class ClusterState implements ToXContent { builder.startObject("nodes"); for (RoutingNode routingNode : readOnlyRoutingNodes()) { - builder.startArray(routingNode.nodeId(), XContentBuilder.FieldCaseConversion.NONE); + builder.startArray(routingNode.nodeId() == null ? "null" : routingNode.nodeId(), XContentBuilder.FieldCaseConversion.NONE); for (ShardRouting shardRouting : routingNode) { shardRouting.toXContent(builder, params); } @@ -486,7 +503,7 @@ public class ClusterState implements ToXContent { if (metrics.contains(Metric.CUSTOMS)) { for (ObjectObjectCursor cursor : customs) { builder.startObject(cursor.key); - lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); + cursor.value.toXContent(builder, params); builder.endObject(); } } @@ -506,21 +523,25 @@ public class ClusterState implements ToXContent { private final ClusterName clusterName; private long version = 0; + private String uuid = UNKNOWN_UUID; private MetaData metaData = MetaData.EMPTY_META_DATA; private RoutingTable routingTable = RoutingTable.EMPTY_ROUTING_TABLE; private DiscoveryNodes nodes = DiscoveryNodes.EMPTY_NODES; private ClusterBlocks blocks = ClusterBlocks.EMPTY_CLUSTER_BLOCK; private final ImmutableOpenMap.Builder customs; + private boolean fromDiff; public Builder(ClusterState state) { this.clusterName = state.clusterName; this.version = state.version(); + this.uuid = state.uuid(); this.nodes = state.nodes(); this.routingTable = state.routingTable(); this.metaData = state.metaData(); this.blocks = state.blocks(); this.customs = ImmutableOpenMap.builder(state.customs()); + this.fromDiff = false; } public Builder(ClusterName clusterName) { @@ -574,6 +595,17 @@ public class ClusterState implements ToXContent { return this; } + public Builder incrementVersion() { + this.version = version + 1; + this.uuid = UNKNOWN_UUID; + return this; + } + + public Builder uuid(String uuid) { + this.uuid = uuid; + return this; + } + public Custom getCustom(String type) { return customs.get(type); } @@ -588,13 +620,26 @@ public class ClusterState implements ToXContent { return this; } + public Builder customs(ImmutableOpenMap customs) { + this.customs.putAll(customs); + return this; + } + + public Builder fromDiff(boolean fromDiff) { + this.fromDiff = fromDiff; + return this; + } + public ClusterState build() { - return new ClusterState(clusterName, version, metaData, routingTable, nodes, blocks, customs.build()); + if (UNKNOWN_UUID.equals(uuid)) { + uuid = Strings.randomBase64UUID(); + } + return new ClusterState(clusterName, version, uuid, metaData, routingTable, nodes, blocks, customs.build(), fromDiff); } public static byte[] toBytes(ClusterState state) throws IOException { BytesStreamOutput os = new BytesStreamOutput(); - writeTo(state, os); + state.writeTo(os); return os.bytes().toBytes(); } @@ -606,39 +651,152 @@ public class ClusterState implements ToXContent { return readFrom(new BytesStreamInput(data), localNode); } - public static void writeTo(ClusterState state, StreamOutput out) throws IOException { - state.clusterName.writeTo(out); - out.writeLong(state.version()); - MetaData.Builder.writeTo(state.metaData(), out); - RoutingTable.Builder.writeTo(state.routingTable(), out); - DiscoveryNodes.Builder.writeTo(state.nodes(), out); - ClusterBlocks.Builder.writeClusterBlocks(state.blocks(), out); - out.writeVInt(state.customs().size()); - for (ObjectObjectCursor cursor : state.customs()) { - out.writeString(cursor.key); - lookupFactorySafe(cursor.key).writeTo(cursor.value, out); - } - } - /** * @param in input stream * @param localNode used to set the local node in the cluster state. can be null. */ public static ClusterState readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException { - ClusterName clusterName = ClusterName.readClusterName(in); + return PROTO.readFrom(in, localNode); + } + + } + + @Override + public Diff diff(ClusterState previousState) { + return new ClusterStateDiff(previousState, this); + } + + @Override + public Diff readDiffFrom(StreamInput in) throws IOException { + return new ClusterStateDiff(in, this); + } + + public ClusterState readFrom(StreamInput in, DiscoveryNode localNode) throws IOException { + ClusterName clusterName = ClusterName.readClusterName(in); + Builder builder = new Builder(clusterName); + builder.version = in.readLong(); + builder.uuid = in.readString(); + builder.metaData = MetaData.Builder.readFrom(in); + builder.routingTable = RoutingTable.Builder.readFrom(in); + builder.nodes = DiscoveryNodes.Builder.readFrom(in, localNode); + builder.blocks = ClusterBlocks.Builder.readClusterBlocks(in); + int customSize = in.readVInt(); + for (int i = 0; i < customSize; i++) { + String type = in.readString(); + Custom customIndexMetaData = lookupPrototypeSafe(type).readFrom(in); + builder.putCustom(type, customIndexMetaData); + } + return builder.build(); + } + + @Override + public ClusterState readFrom(StreamInput in) throws IOException { + return readFrom(in, nodes.localNode()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + clusterName.writeTo(out); + out.writeLong(version); + out.writeString(uuid); + metaData.writeTo(out); + routingTable.writeTo(out); + nodes.writeTo(out); + blocks.writeTo(out); + out.writeVInt(customs.size()); + for (ObjectObjectCursor cursor : customs) { + out.writeString(cursor.key); + cursor.value.writeTo(out); + } + } + + private static class ClusterStateDiff implements Diff { + + private final long toVersion; + + private final String fromUuid; + + private final String toUuid; + + private final ClusterName clusterName; + + private final Diff routingTable; + + private final Diff nodes; + + private final Diff metaData; + + private final Diff blocks; + + private final Diff> customs; + + public ClusterStateDiff(ClusterState before, ClusterState after) { + fromUuid = before.uuid; + toUuid = after.uuid; + toVersion = after.version; + clusterName = after.clusterName; + routingTable = after.routingTable.diff(before.routingTable); + nodes = after.nodes.diff(before.nodes); + metaData = after.metaData.diff(before.metaData); + blocks = after.blocks.diff(before.blocks); + customs = DiffableUtils.diff(before.customs, after.customs); + } + + public ClusterStateDiff(StreamInput in, ClusterState proto) throws IOException { + clusterName = ClusterName.readClusterName(in); + fromUuid = in.readString(); + toUuid = in.readString(); + toVersion = in.readLong(); + routingTable = proto.routingTable.readDiffFrom(in); + nodes = proto.nodes.readDiffFrom(in); + metaData = proto.metaData.readDiffFrom(in); + blocks = proto.blocks.readDiffFrom(in); + customs = DiffableUtils.readImmutableOpenMapDiff(in, new KeyedReader() { + @Override + public Custom readFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readFrom(in); + } + + @Override + public Diff readDiffFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readDiffFrom(in); + } + }); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + clusterName.writeTo(out); + out.writeString(fromUuid); + out.writeString(toUuid); + out.writeLong(toVersion); + routingTable.writeTo(out); + nodes.writeTo(out); + metaData.writeTo(out); + blocks.writeTo(out); + customs.writeTo(out); + } + + @Override + public ClusterState apply(ClusterState state) { Builder builder = new Builder(clusterName); - builder.version = in.readLong(); - builder.metaData = MetaData.Builder.readFrom(in); - builder.routingTable = RoutingTable.Builder.readFrom(in); - builder.nodes = DiscoveryNodes.Builder.readFrom(in, localNode); - builder.blocks = ClusterBlocks.Builder.readClusterBlocks(in); - int customSize = in.readVInt(); - for (int i = 0; i < customSize; i++) { - String type = in.readString(); - Custom customIndexMetaData = lookupFactorySafe(type).readFrom(in); - builder.putCustom(type, customIndexMetaData); + if (toUuid.equals(state.uuid)) { + // no need to read the rest - cluster state didn't change + return state; } + if (fromUuid.equals(state.uuid) == false) { + throw new IncompatibleClusterStateVersionException(state.version, state.uuid, toVersion, fromUuid); + } + builder.uuid(toUuid); + builder.version(toVersion); + builder.routingTable(routingTable.apply(state.routingTable)); + builder.nodes(nodes.apply(state.nodes)); + builder.metaData(metaData.apply(state.metaData)); + builder.blocks(blocks.apply(state.blocks)); + builder.customs(customs.apply(state.customs)); + builder.fromDiff(true); return builder.build(); } } + } diff --git a/src/main/java/org/elasticsearch/cluster/Diff.java b/src/main/java/org/elasticsearch/cluster/Diff.java new file mode 100644 index 00000000000..2e571f43bca --- /dev/null +++ b/src/main/java/org/elasticsearch/cluster/Diff.java @@ -0,0 +1,42 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Represents difference between states of cluster state parts + */ +public interface Diff { + + /** + * Applies difference to the specified part and retunrs the resulted part + */ + T apply(T part); + + /** + * Writes the differences into the output stream + * @param out + * @throws IOException + */ + void writeTo(StreamOutput out) throws IOException; +} diff --git a/src/main/java/org/elasticsearch/cluster/Diffable.java b/src/main/java/org/elasticsearch/cluster/Diffable.java new file mode 100644 index 00000000000..7ce60047a2b --- /dev/null +++ b/src/main/java/org/elasticsearch/cluster/Diffable.java @@ -0,0 +1,42 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.io.stream.StreamInput; + +import java.io.IOException; + +/** + * Cluster state part, changes in which can be serialized + */ +public interface Diffable extends Writeable { + + /** + * Returns serializable object representing differences between this and previousState + */ + Diff diff(T previousState); + + /** + * Reads the {@link org.elasticsearch.cluster.Diff} from StreamInput + */ + Diff readDiffFrom(StreamInput in) throws IOException; + +} diff --git a/src/main/java/org/elasticsearch/cluster/DiffableUtils.java b/src/main/java/org/elasticsearch/cluster/DiffableUtils.java new file mode 100644 index 00000000000..4e912a34f97 --- /dev/null +++ b/src/main/java/org/elasticsearch/cluster/DiffableUtils.java @@ -0,0 +1,283 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import com.carrotsearch.hppc.cursors.ObjectCursor; +import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static com.google.common.collect.Lists.newArrayList; +import static com.google.common.collect.Maps.newHashMap; + +public final class DiffableUtils { + private DiffableUtils() { + } + + /** + * Calculates diff between two ImmutableOpenMaps of Diffable objects + */ + public static > Diff> diff(ImmutableOpenMap before, ImmutableOpenMap after) { + assert after != null && before != null; + return new ImmutableOpenMapDiff<>(before, after); + } + + /** + * Calculates diff between two ImmutableMaps of Diffable objects + */ + public static > Diff> diff(ImmutableMap before, ImmutableMap after) { + assert after != null && before != null; + return new ImmutableMapDiff<>(before, after); + } + + /** + * Loads an object that represents difference between two ImmutableOpenMaps + */ + public static > Diff> readImmutableOpenMapDiff(StreamInput in, KeyedReader keyedReader) throws IOException { + return new ImmutableOpenMapDiff<>(in, keyedReader); + } + + /** + * Loads an object that represents difference between two ImmutableMaps + */ + public static > Diff> readImmutableMapDiff(StreamInput in, KeyedReader keyedReader) throws IOException { + return new ImmutableMapDiff<>(in, keyedReader); + } + + /** + * Loads an object that represents difference between two ImmutableOpenMaps + */ + public static > Diff> readImmutableOpenMapDiff(StreamInput in, T proto) throws IOException { + return new ImmutableOpenMapDiff<>(in, new PrototypeReader<>(proto)); + } + + /** + * Loads an object that represents difference between two ImmutableMaps + */ + public static > Diff> readImmutableMapDiff(StreamInput in, T proto) throws IOException { + return new ImmutableMapDiff<>(in, new PrototypeReader<>(proto)); + } + + /** + * A reader that can deserialize an object. The reader can select the deserialization type based on the key. It's + * used in custom metadata deserialization. + */ + public interface KeyedReader { + + /** + * reads an object of the type T from the stream input + */ + T readFrom(StreamInput in, String key) throws IOException; + + /** + * reads an object that respresents differences between two objects with the type T from the stream input + */ + Diff readDiffFrom(StreamInput in, String key) throws IOException; + } + + /** + * Implementation of the KeyedReader that is using a prototype object for reading operations + * + * Note: this implementation is ignoring the key. + */ + public static class PrototypeReader> implements KeyedReader { + private T proto; + + public PrototypeReader(T proto) { + this.proto = proto; + } + + @Override + public T readFrom(StreamInput in, String key) throws IOException { + return proto.readFrom(in); + } + + @Override + public Diff readDiffFrom(StreamInput in, String key) throws IOException { + return proto.readDiffFrom(in); + } + } + + /** + * Represents differences between two ImmutableMaps of diffable objects + * + * @param the diffable object + */ + private static class ImmutableMapDiff> extends MapDiff> { + + protected ImmutableMapDiff(StreamInput in, KeyedReader reader) throws IOException { + super(in, reader); + } + + public ImmutableMapDiff(ImmutableMap before, ImmutableMap after) { + assert after != null && before != null; + for (String key : before.keySet()) { + if (!after.containsKey(key)) { + deletes.add(key); + } + } + for (ImmutableMap.Entry partIter : after.entrySet()) { + T beforePart = before.get(partIter.getKey()); + if (beforePart == null) { + adds.put(partIter.getKey(), partIter.getValue()); + } else if (partIter.getValue().equals(beforePart) == false) { + diffs.put(partIter.getKey(), partIter.getValue().diff(beforePart)); + } + } + } + + @Override + public ImmutableMap apply(ImmutableMap map) { + HashMap builder = newHashMap(); + builder.putAll(map); + + for (String part : deletes) { + builder.remove(part); + } + + for (Map.Entry> diff : diffs.entrySet()) { + builder.put(diff.getKey(), diff.getValue().apply(builder.get(diff.getKey()))); + } + + for (Map.Entry additon : adds.entrySet()) { + builder.put(additon.getKey(), additon.getValue()); + } + return ImmutableMap.copyOf(builder); + } + } + + /** + * Represents differences between two ImmutableOpenMap of diffable objects + * + * @param the diffable object + */ + private static class ImmutableOpenMapDiff> extends MapDiff> { + + protected ImmutableOpenMapDiff(StreamInput in, KeyedReader reader) throws IOException { + super(in, reader); + } + + public ImmutableOpenMapDiff(ImmutableOpenMap before, ImmutableOpenMap after) { + assert after != null && before != null; + for (ObjectCursor key : before.keys()) { + if (!after.containsKey(key.value)) { + deletes.add(key.value); + } + } + for (ObjectObjectCursor partIter : after) { + T beforePart = before.get(partIter.key); + if (beforePart == null) { + adds.put(partIter.key, partIter.value); + } else if (partIter.value.equals(beforePart) == false) { + diffs.put(partIter.key, partIter.value.diff(beforePart)); + } + } + } + + @Override + public ImmutableOpenMap apply(ImmutableOpenMap map) { + ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); + builder.putAll(map); + + for (String part : deletes) { + builder.remove(part); + } + + for (Map.Entry> diff : diffs.entrySet()) { + builder.put(diff.getKey(), diff.getValue().apply(builder.get(diff.getKey()))); + } + + for (Map.Entry additon : adds.entrySet()) { + builder.put(additon.getKey(), additon.getValue()); + } + return builder.build(); + } + } + + /** + * Represents differences between two maps of diffable objects + * + * This class is used as base class for different map implementations + * + * @param the diffable object + */ + private static abstract class MapDiff, M> implements Diff { + + protected final List deletes; + protected final Map> diffs; + protected final Map adds; + + protected MapDiff() { + deletes = newArrayList(); + diffs = newHashMap(); + adds = newHashMap(); + } + + protected MapDiff(StreamInput in, KeyedReader reader) throws IOException { + deletes = newArrayList(); + diffs = newHashMap(); + adds = newHashMap(); + int deletesCount = in.readVInt(); + for (int i = 0; i < deletesCount; i++) { + deletes.add(in.readString()); + } + + int diffsCount = in.readVInt(); + for (int i = 0; i < diffsCount; i++) { + String key = in.readString(); + Diff diff = reader.readDiffFrom(in, key); + diffs.put(key, diff); + } + + int addsCount = in.readVInt(); + for (int i = 0; i < addsCount; i++) { + String key = in.readString(); + T part = reader.readFrom(in, key); + adds.put(key, part); + } + } + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(deletes.size()); + for (String delete : deletes) { + out.writeString(delete); + } + + out.writeVInt(diffs.size()); + for (Map.Entry> entry : diffs.entrySet()) { + out.writeString(entry.getKey()); + entry.getValue().writeTo(out); + } + + out.writeVInt(adds.size()); + for (Map.Entry entry : adds.entrySet()) { + out.writeString(entry.getKey()); + entry.getValue().writeTo(out); + } + } + } +} diff --git a/src/main/java/org/elasticsearch/cluster/IncompatibleClusterStateVersionException.java b/src/main/java/org/elasticsearch/cluster/IncompatibleClusterStateVersionException.java new file mode 100644 index 00000000000..92f5897bf2e --- /dev/null +++ b/src/main/java/org/elasticsearch/cluster/IncompatibleClusterStateVersionException.java @@ -0,0 +1,35 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import org.elasticsearch.ElasticsearchException; + +/** + * Thrown by {@link Diffable#readDiffAndApply(org.elasticsearch.common.io.stream.StreamInput)} method + */ +public class IncompatibleClusterStateVersionException extends ElasticsearchException { + public IncompatibleClusterStateVersionException(String msg) { + super(msg); + } + + public IncompatibleClusterStateVersionException(long expectedVersion, String expectedUuid, long receivedVersion, String receivedUuid) { + super("Expected diff for version " + expectedVersion + " with uuid " + expectedUuid + " got version " + receivedVersion + " and uuid " + receivedUuid); + } +} diff --git a/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java b/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java index bb7d332de4f..95c0ba7127e 100644 --- a/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java +++ b/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.common.io.stream.StreamInput; @@ -36,10 +37,12 @@ import java.util.Set; /** * Represents current cluster level blocks to block dirty operations done against the cluster. */ -public class ClusterBlocks { +public class ClusterBlocks extends AbstractDiffable { public static final ClusterBlocks EMPTY_CLUSTER_BLOCK = new ClusterBlocks(ImmutableSet.of(), ImmutableMap.>of()); + public static final ClusterBlocks PROTO = EMPTY_CLUSTER_BLOCK; + private final ImmutableSet global; private final ImmutableMap> indicesBlocks; @@ -203,6 +206,43 @@ public class ClusterBlocks { return new ClusterBlockException(builder.build()); } + @Override + public void writeTo(StreamOutput out) throws IOException { + writeBlockSet(global, out); + out.writeVInt(indicesBlocks.size()); + for (Map.Entry> entry : indicesBlocks.entrySet()) { + out.writeString(entry.getKey()); + writeBlockSet(entry.getValue(), out); + } + } + + private static void writeBlockSet(ImmutableSet blocks, StreamOutput out) throws IOException { + out.writeVInt(blocks.size()); + for (ClusterBlock block : blocks) { + block.writeTo(out); + } + } + + @Override + public ClusterBlocks readFrom(StreamInput in) throws IOException { + ImmutableSet global = readBlockSet(in); + ImmutableMap.Builder> indicesBuilder = ImmutableMap.builder(); + int size = in.readVInt(); + for (int j = 0; j < size; j++) { + indicesBuilder.put(in.readString().intern(), readBlockSet(in)); + } + return new ClusterBlocks(global, indicesBuilder.build()); + } + + private static ImmutableSet readBlockSet(StreamInput in) throws IOException { + ImmutableSet.Builder builder = ImmutableSet.builder(); + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + builder.add(ClusterBlock.readClusterBlock(in)); + } + return builder.build(); + } + static class ImmutableLevelHolder { static final ImmutableLevelHolder EMPTY = new ImmutableLevelHolder(ImmutableSet.of(), ImmutableMap.>of()); @@ -313,38 +353,7 @@ public class ClusterBlocks { } public static ClusterBlocks readClusterBlocks(StreamInput in) throws IOException { - ImmutableSet global = readBlockSet(in); - ImmutableMap.Builder> indicesBuilder = ImmutableMap.builder(); - int size = in.readVInt(); - for (int j = 0; j < size; j++) { - indicesBuilder.put(in.readString().intern(), readBlockSet(in)); - } - return new ClusterBlocks(global, indicesBuilder.build()); - } - - public static void writeClusterBlocks(ClusterBlocks blocks, StreamOutput out) throws IOException { - writeBlockSet(blocks.global(), out); - out.writeVInt(blocks.indices().size()); - for (Map.Entry> entry : blocks.indices().entrySet()) { - out.writeString(entry.getKey()); - writeBlockSet(entry.getValue(), out); - } - } - - private static void writeBlockSet(ImmutableSet blocks, StreamOutput out) throws IOException { - out.writeVInt(blocks.size()); - for (ClusterBlock block : blocks) { - block.writeTo(out); - } - } - - private static ImmutableSet readBlockSet(StreamInput in) throws IOException { - ImmutableSet.Builder builder = ImmutableSet.builder(); - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - builder.add(ClusterBlock.readClusterBlock(in)); - } - return builder.build(); + return PROTO.readFrom(in); } } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java index 008935ec026..0f7e55c8087 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java @@ -21,6 +21,7 @@ package org.elasticsearch.cluster.metadata; import com.google.common.collect.ImmutableSet; import org.elasticsearch.ElasticsearchGenerationException; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.compress.CompressedString; import org.elasticsearch.common.io.stream.StreamInput; @@ -38,7 +39,9 @@ import java.util.Set; /** * */ -public class AliasMetaData { +public class AliasMetaData extends AbstractDiffable { + + public static final AliasMetaData PROTO = new AliasMetaData("", null, null, null); private final String alias; @@ -146,6 +149,48 @@ public class AliasMetaData { return result; } + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(alias()); + if (filter() != null) { + out.writeBoolean(true); + filter.writeTo(out); + } else { + out.writeBoolean(false); + } + if (indexRouting() != null) { + out.writeBoolean(true); + out.writeString(indexRouting()); + } else { + out.writeBoolean(false); + } + if (searchRouting() != null) { + out.writeBoolean(true); + out.writeString(searchRouting()); + } else { + out.writeBoolean(false); + } + + } + + @Override + public AliasMetaData readFrom(StreamInput in) throws IOException { + String alias = in.readString(); + CompressedString filter = null; + if (in.readBoolean()) { + filter = CompressedString.readCompressedString(in); + } + String indexRouting = null; + if (in.readBoolean()) { + indexRouting = in.readString(); + } + String searchRouting = null; + if (in.readBoolean()) { + searchRouting = in.readString(); + } + return new AliasMetaData(alias, filter, indexRouting, searchRouting); + } + public static class Builder { private final String alias; @@ -294,44 +339,12 @@ public class AliasMetaData { return builder.build(); } - public static void writeTo(AliasMetaData aliasMetaData, StreamOutput out) throws IOException { - out.writeString(aliasMetaData.alias()); - if (aliasMetaData.filter() != null) { - out.writeBoolean(true); - aliasMetaData.filter.writeTo(out); - } else { - out.writeBoolean(false); - } - if (aliasMetaData.indexRouting() != null) { - out.writeBoolean(true); - out.writeString(aliasMetaData.indexRouting()); - } else { - out.writeBoolean(false); - } - if (aliasMetaData.searchRouting() != null) { - out.writeBoolean(true); - out.writeString(aliasMetaData.searchRouting()); - } else { - out.writeBoolean(false); - } - + public void writeTo(AliasMetaData aliasMetaData, StreamOutput out) throws IOException { + aliasMetaData.writeTo(out); } public static AliasMetaData readFrom(StreamInput in) throws IOException { - String alias = in.readString(); - CompressedString filter = null; - if (in.readBoolean()) { - filter = CompressedString.readCompressedString(in); - } - String indexRouting = null; - if (in.readBoolean()) { - indexRouting = in.readString(); - } - String searchRouting = null; - if (in.readBoolean()) { - searchRouting = in.readString(); - } - return new AliasMetaData(alias, filter, indexRouting, searchRouting); + return PROTO.readFrom(in); } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index d6bcacf1615..fe76d0f3f2b 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -24,6 +24,9 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import org.elasticsearch.Version; +import org.elasticsearch.cluster.Diff; +import org.elasticsearch.cluster.Diffable; +import org.elasticsearch.cluster.DiffableUtils; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.node.DiscoveryNodeFilters; @@ -59,60 +62,54 @@ import static org.elasticsearch.common.settings.ImmutableSettings.*; /** * */ -public class IndexMetaData { +public class IndexMetaData implements Diffable { + public static final IndexMetaData PROTO = IndexMetaData.builder("") + .settings(ImmutableSettings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); - public interface Custom { + public interface Custom extends Diffable, ToXContent { String type(); - interface Factory { + Custom fromMap(Map map) throws IOException; - String type(); + Custom fromXContent(XContentParser parser) throws IOException; - T readFrom(StreamInput in) throws IOException; - - void writeTo(T customIndexMetaData, StreamOutput out) throws IOException; - - T fromMap(Map map) throws IOException; - - T fromXContent(XContentParser parser) throws IOException; - - void toXContent(T customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException; - - /** - * Merges from first to second, with first being more important, i.e., if something exists in first and second, - * first will prevail. - */ - T merge(T first, T second); - } + /** + * Merges from this to another, with this being more important, i.e., if something exists in this and another, + * this will prevail. + */ + Custom mergeWith(Custom another); } - public static Map customFactories = new HashMap<>(); + public static Map customPrototypes = new HashMap<>(); static { // register non plugin custom metadata - registerFactory(IndexWarmersMetaData.TYPE, IndexWarmersMetaData.FACTORY); + registerPrototype(IndexWarmersMetaData.TYPE, IndexWarmersMetaData.PROTO); } /** * Register a custom index meta data factory. Make sure to call it from a static block. */ - public static void registerFactory(String type, Custom.Factory factory) { - customFactories.put(type, factory); + public static void registerPrototype(String type, Custom proto) { + customPrototypes.put(type, proto); } @Nullable - public static Custom.Factory lookupFactory(String type) { - return customFactories.get(type); + public static T lookupPrototype(String type) { + //noinspection unchecked + return (T) customPrototypes.get(type); } - public static Custom.Factory lookupFactorySafe(String type) { - Custom.Factory factory = customFactories.get(type); - if (factory == null) { - throw new IllegalArgumentException("No custom index metadata factoy registered for type [" + type + "]"); + public static T lookupPrototypeSafe(String type) { + //noinspection unchecked + T proto = (T) customPrototypes.get(type); + if (proto == null) { + throw new IllegalArgumentException("No custom metadata prototype registered for type [" + type + "]"); } - return factory; + return proto; } public static final ClusterBlock INDEX_READ_ONLY_BLOCK = new ClusterBlock(5, "index read-only (api)", false, false, RestStatus.FORBIDDEN, EnumSet.of(ClusterBlockLevel.WRITE, ClusterBlockLevel.METADATA_WRITE)); @@ -451,7 +448,9 @@ public class IndexMetaData { if (state != that.state) { return false; } - + if (!customs.equals(that.customs)) { + return false; + } return true; } @@ -465,6 +464,126 @@ public class IndexMetaData { return result; } + @Override + public Diff diff(IndexMetaData previousState) { + return new IndexMetaDataDiff(previousState, this); + } + + @Override + public Diff readDiffFrom(StreamInput in) throws IOException { + return new IndexMetaDataDiff(in); + } + + private static class IndexMetaDataDiff implements Diff { + + private final String index; + private final long version; + private final State state; + private final Settings settings; + private final Diff> mappings; + private final Diff> aliases; + private Diff> customs; + + public IndexMetaDataDiff(IndexMetaData before, IndexMetaData after) { + index = after.index; + version = after.version; + state = after.state; + settings = after.settings; + mappings = DiffableUtils.diff(before.mappings, after.mappings); + aliases = DiffableUtils.diff(before.aliases, after.aliases); + customs = DiffableUtils.diff(before.customs, after.customs); + } + + public IndexMetaDataDiff(StreamInput in) throws IOException { + index = in.readString(); + version = in.readLong(); + state = State.fromId(in.readByte()); + settings = ImmutableSettings.readSettingsFromStream(in); + mappings = DiffableUtils.readImmutableOpenMapDiff(in, MappingMetaData.PROTO); + aliases = DiffableUtils.readImmutableOpenMapDiff(in, AliasMetaData.PROTO); + customs = DiffableUtils.readImmutableOpenMapDiff(in, new DiffableUtils.KeyedReader() { + @Override + public Custom readFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readFrom(in); + } + + @Override + public Diff readDiffFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readDiffFrom(in); + } + }); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(index); + out.writeLong(version); + out.writeByte(state.id); + ImmutableSettings.writeSettingsToStream(settings, out); + mappings.writeTo(out); + aliases.writeTo(out); + customs.writeTo(out); + } + + @Override + public IndexMetaData apply(IndexMetaData part) { + Builder builder = builder(index); + builder.version(version); + builder.state(state); + builder.settings(settings); + builder.mappings.putAll(mappings.apply(part.mappings)); + builder.aliases.putAll(aliases.apply(part.aliases)); + builder.customs.putAll(customs.apply(part.customs)); + return builder.build(); + } + } + + @Override + public IndexMetaData readFrom(StreamInput in) throws IOException { + Builder builder = new Builder(in.readString()); + builder.version(in.readLong()); + builder.state(State.fromId(in.readByte())); + builder.settings(readSettingsFromStream(in)); + int mappingsSize = in.readVInt(); + for (int i = 0; i < mappingsSize; i++) { + MappingMetaData mappingMd = MappingMetaData.PROTO.readFrom(in); + builder.putMapping(mappingMd); + } + int aliasesSize = in.readVInt(); + for (int i = 0; i < aliasesSize; i++) { + AliasMetaData aliasMd = AliasMetaData.Builder.readFrom(in); + builder.putAlias(aliasMd); + } + int customSize = in.readVInt(); + for (int i = 0; i < customSize; i++) { + String type = in.readString(); + Custom customIndexMetaData = lookupPrototypeSafe(type).readFrom(in); + builder.putCustom(type, customIndexMetaData); + } + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(index); + out.writeLong(version); + out.writeByte(state.id()); + writeSettingsToStream(settings, out); + out.writeVInt(mappings.size()); + for (ObjectCursor cursor : mappings.values()) { + cursor.value.writeTo(out); + } + out.writeVInt(aliases.size()); + for (ObjectCursor cursor : aliases.values()) { + cursor.value.writeTo(out); + } + out.writeVInt(customs.size()); + for (ObjectObjectCursor cursor : customs) { + out.writeString(cursor.key); + cursor.value.writeTo(out); + } + } + public static Builder builder(String index) { return new Builder(index); } @@ -660,7 +779,7 @@ public class IndexMetaData { for (ObjectObjectCursor cursor : indexMetaData.customs()) { builder.startObject(cursor.key, XContentBuilder.FieldCaseConversion.NONE); - lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); + cursor.value.toXContent(builder, params); builder.endObject(); } @@ -707,12 +826,13 @@ public class IndexMetaData { } } else { // check if its a custom index metadata - Custom.Factory factory = lookupFactory(currentFieldName); - if (factory == null) { + Custom proto = lookupPrototype(currentFieldName); + if (proto == null) { //TODO warn parser.skipChildren(); } else { - builder.putCustom(factory.type(), factory.fromXContent(parser)); + Custom custom = proto.fromXContent(parser); + builder.putCustom(custom.type(), custom); } } } else if (token == XContentParser.Token.START_ARRAY) { @@ -741,47 +861,7 @@ public class IndexMetaData { } public static IndexMetaData readFrom(StreamInput in) throws IOException { - Builder builder = new Builder(in.readString()); - builder.version(in.readLong()); - builder.state(State.fromId(in.readByte())); - builder.settings(readSettingsFromStream(in)); - int mappingsSize = in.readVInt(); - for (int i = 0; i < mappingsSize; i++) { - MappingMetaData mappingMd = MappingMetaData.readFrom(in); - builder.putMapping(mappingMd); - } - int aliasesSize = in.readVInt(); - for (int i = 0; i < aliasesSize; i++) { - AliasMetaData aliasMd = AliasMetaData.Builder.readFrom(in); - builder.putAlias(aliasMd); - } - int customSize = in.readVInt(); - for (int i = 0; i < customSize; i++) { - String type = in.readString(); - Custom customIndexMetaData = lookupFactorySafe(type).readFrom(in); - builder.putCustom(type, customIndexMetaData); - } - return builder.build(); - } - - public static void writeTo(IndexMetaData indexMetaData, StreamOutput out) throws IOException { - out.writeString(indexMetaData.index()); - out.writeLong(indexMetaData.version()); - out.writeByte(indexMetaData.state().id()); - writeSettingsToStream(indexMetaData.settings(), out); - out.writeVInt(indexMetaData.mappings().size()); - for (ObjectCursor cursor : indexMetaData.mappings().values()) { - MappingMetaData.writeTo(cursor.value, out); - } - out.writeVInt(indexMetaData.aliases().size()); - for (ObjectCursor cursor : indexMetaData.aliases().values()) { - AliasMetaData.Builder.writeTo(cursor.value, out); - } - out.writeVInt(indexMetaData.customs().size()); - for (ObjectObjectCursor cursor : indexMetaData.customs()) { - out.writeString(cursor.key); - lookupFactorySafe(cursor.key).writeTo(cursor.value, out); - } + return PROTO.readFrom(in); } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java index 582e008550d..54150ee6a1e 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java @@ -21,7 +21,7 @@ package org.elasticsearch.cluster.metadata; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.collect.Sets; -import org.elasticsearch.Version; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.compress.CompressedString; @@ -42,7 +42,9 @@ import java.util.Set; /** * */ -public class IndexTemplateMetaData { +public class IndexTemplateMetaData extends AbstractDiffable { + + public static final IndexTemplateMetaData PROTO = IndexTemplateMetaData.builder("").build(); private final String name; @@ -161,11 +163,57 @@ public class IndexTemplateMetaData { return result; } + @Override + public IndexTemplateMetaData readFrom(StreamInput in) throws IOException { + Builder builder = new Builder(in.readString()); + builder.order(in.readInt()); + builder.template(in.readString()); + builder.settings(ImmutableSettings.readSettingsFromStream(in)); + int mappingsSize = in.readVInt(); + for (int i = 0; i < mappingsSize; i++) { + builder.putMapping(in.readString(), CompressedString.readCompressedString(in)); + } + int aliasesSize = in.readVInt(); + for (int i = 0; i < aliasesSize; i++) { + AliasMetaData aliasMd = AliasMetaData.Builder.readFrom(in); + builder.putAlias(aliasMd); + } + int customSize = in.readVInt(); + for (int i = 0; i < customSize; i++) { + String type = in.readString(); + IndexMetaData.Custom customIndexMetaData = IndexMetaData.lookupPrototypeSafe(type).readFrom(in); + builder.putCustom(type, customIndexMetaData); + } + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(name); + out.writeInt(order); + out.writeString(template); + ImmutableSettings.writeSettingsToStream(settings, out); + out.writeVInt(mappings.size()); + for (ObjectObjectCursor cursor : mappings) { + out.writeString(cursor.key); + cursor.value.writeTo(out); + } + out.writeVInt(aliases.size()); + for (ObjectCursor cursor : aliases.values()) { + cursor.value.writeTo(out); + } + out.writeVInt(customs.size()); + for (ObjectObjectCursor cursor : customs) { + out.writeString(cursor.key); + cursor.value.writeTo(out); + } + } + public static class Builder { private static final Set VALID_FIELDS = Sets.newHashSet("template", "order", "mappings", "settings"); static { - VALID_FIELDS.addAll(IndexMetaData.customFactories.keySet()); + VALID_FIELDS.addAll(IndexMetaData.customPrototypes.keySet()); } private String name; @@ -305,7 +353,7 @@ public class IndexTemplateMetaData { for (ObjectObjectCursor cursor : indexTemplateMetaData.customs()) { builder.startObject(cursor.key, XContentBuilder.FieldCaseConversion.NONE); - IndexMetaData.lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); + cursor.value.toXContent(builder, params); builder.endObject(); } @@ -347,12 +395,13 @@ public class IndexTemplateMetaData { } } else { // check if its a custom index metadata - IndexMetaData.Custom.Factory factory = IndexMetaData.lookupFactory(currentFieldName); - if (factory == null) { + IndexMetaData.Custom proto = IndexMetaData.lookupPrototype(currentFieldName); + if (proto == null) { //TODO warn parser.skipChildren(); } else { - builder.putCustom(factory.type(), factory.fromXContent(parser)); + IndexMetaData.Custom custom = proto.fromXContent(parser); + builder.putCustom(custom.type(), custom); } } } else if (token == XContentParser.Token.START_ARRAY) { @@ -401,47 +450,7 @@ public class IndexTemplateMetaData { } public static IndexTemplateMetaData readFrom(StreamInput in) throws IOException { - Builder builder = new Builder(in.readString()); - builder.order(in.readInt()); - builder.template(in.readString()); - builder.settings(ImmutableSettings.readSettingsFromStream(in)); - int mappingsSize = in.readVInt(); - for (int i = 0; i < mappingsSize; i++) { - builder.putMapping(in.readString(), CompressedString.readCompressedString(in)); - } - int aliasesSize = in.readVInt(); - for (int i = 0; i < aliasesSize; i++) { - AliasMetaData aliasMd = AliasMetaData.Builder.readFrom(in); - builder.putAlias(aliasMd); - } - int customSize = in.readVInt(); - for (int i = 0; i < customSize; i++) { - String type = in.readString(); - IndexMetaData.Custom customIndexMetaData = IndexMetaData.lookupFactorySafe(type).readFrom(in); - builder.putCustom(type, customIndexMetaData); - } - return builder.build(); - } - - public static void writeTo(IndexTemplateMetaData indexTemplateMetaData, StreamOutput out) throws IOException { - out.writeString(indexTemplateMetaData.name()); - out.writeInt(indexTemplateMetaData.order()); - out.writeString(indexTemplateMetaData.template()); - ImmutableSettings.writeSettingsToStream(indexTemplateMetaData.settings(), out); - out.writeVInt(indexTemplateMetaData.mappings().size()); - for (ObjectObjectCursor cursor : indexTemplateMetaData.mappings()) { - out.writeString(cursor.key); - cursor.value.writeTo(out); - } - out.writeVInt(indexTemplateMetaData.aliases().size()); - for (ObjectCursor cursor : indexTemplateMetaData.aliases().values()) { - AliasMetaData.Builder.writeTo(cursor.value, out); - } - out.writeVInt(indexTemplateMetaData.customs().size()); - for (ObjectObjectCursor cursor : indexTemplateMetaData.customs()) { - out.writeString(cursor.key); - IndexMetaData.lookupFactorySafe(cursor.key).writeTo(cursor.value, out); - } + return PROTO.readFrom(in); } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java index f80c6072bfc..7225a43d5ef 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java @@ -19,8 +19,10 @@ package org.elasticsearch.cluster.metadata; +import com.google.common.collect.Maps; import org.elasticsearch.Version; import org.elasticsearch.action.TimestampParsingException; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.compress.CompressedString; @@ -38,14 +40,18 @@ import org.elasticsearch.index.mapper.internal.TimestampFieldMapper; import java.io.IOException; import java.util.Arrays; +import java.util.HashMap; import java.util.Map; +import static com.google.common.collect.Maps.newHashMap; import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeBooleanValue; /** * Mapping configuration for a type. */ -public class MappingMetaData { +public class MappingMetaData extends AbstractDiffable { + + public static final MappingMetaData PROTO = new MappingMetaData(); public static class Id { @@ -317,6 +323,15 @@ public class MappingMetaData { initMappers(withoutType); } + private MappingMetaData() { + this.type = ""; + try { + this.source = new CompressedString(""); + } catch (IOException ex) { + throw new IllegalStateException("Cannot create MappingMetaData prototype", ex); + } + } + private void initMappers(Map withoutType) { if (withoutType.containsKey("_id")) { String path = null; @@ -532,34 +547,35 @@ public class MappingMetaData { } } - public static void writeTo(MappingMetaData mappingMd, StreamOutput out) throws IOException { - out.writeString(mappingMd.type()); - mappingMd.source().writeTo(out); + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(type()); + source().writeTo(out); // id - if (mappingMd.id().hasPath()) { + if (id().hasPath()) { out.writeBoolean(true); - out.writeString(mappingMd.id().path()); + out.writeString(id().path()); } else { out.writeBoolean(false); } // routing - out.writeBoolean(mappingMd.routing().required()); - if (mappingMd.routing().hasPath()) { + out.writeBoolean(routing().required()); + if (routing().hasPath()) { out.writeBoolean(true); - out.writeString(mappingMd.routing().path()); + out.writeString(routing().path()); } else { out.writeBoolean(false); } // timestamp - out.writeBoolean(mappingMd.timestamp().enabled()); - out.writeOptionalString(mappingMd.timestamp().path()); - out.writeString(mappingMd.timestamp().format()); - out.writeOptionalString(mappingMd.timestamp().defaultTimestamp()); + out.writeBoolean(timestamp().enabled()); + out.writeOptionalString(timestamp().path()); + out.writeString(timestamp().format()); + out.writeOptionalString(timestamp().defaultTimestamp()); // TODO Remove the test in elasticsearch 2.0.0 if (out.getVersion().onOrAfter(Version.V_1_5_0)) { - out.writeOptionalBoolean(mappingMd.timestamp().ignoreMissing()); + out.writeOptionalBoolean(timestamp().ignoreMissing()); } - out.writeBoolean(mappingMd.hasParentField()); + out.writeBoolean(hasParentField()); } @Override @@ -588,7 +604,7 @@ public class MappingMetaData { return result; } - public static MappingMetaData readFrom(StreamInput in) throws IOException { + public MappingMetaData readFrom(StreamInput in) throws IOException { String type = in.readString(); CompressedString source = CompressedString.readCompressedString(in); // id diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index ea25a6d5256..97a1367d8e8 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -25,7 +25,9 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.base.Predicate; import com.google.common.collect.*; +import org.elasticsearch.cluster.*; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.cluster.DiffableUtils.KeyedReader; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.common.Nullable; @@ -55,7 +57,9 @@ import static org.elasticsearch.common.settings.ImmutableSettings.*; /** * */ -public class MetaData implements Iterable { +public class MetaData implements Iterable, Diffable { + + public static final MetaData PROTO = builder().build(); public static final String ALL = "_all"; @@ -67,60 +71,51 @@ public class MetaData implements Iterable { GATEWAY, /* Custom metadata should be stored as part of a snapshot */ - SNAPSHOT; + SNAPSHOT } public static EnumSet API_ONLY = EnumSet.of(XContentContext.API); public static EnumSet API_AND_GATEWAY = EnumSet.of(XContentContext.API, XContentContext.GATEWAY); public static EnumSet API_AND_SNAPSHOT = EnumSet.of(XContentContext.API, XContentContext.SNAPSHOT); - public interface Custom { + public interface Custom extends Diffable, ToXContent { - abstract class Factory { + String type(); - public abstract String type(); + Custom fromXContent(XContentParser parser) throws IOException; - public abstract T readFrom(StreamInput in) throws IOException; - - public abstract void writeTo(T customIndexMetaData, StreamOutput out) throws IOException; - - public abstract T fromXContent(XContentParser parser) throws IOException; - - public abstract void toXContent(T customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException; - - public EnumSet context() { - return API_ONLY; - } - } + EnumSet context(); } - public static Map customFactories = new HashMap<>(); + public static Map customPrototypes = new HashMap<>(); static { // register non plugin custom metadata - registerFactory(RepositoriesMetaData.TYPE, RepositoriesMetaData.FACTORY); - registerFactory(SnapshotMetaData.TYPE, SnapshotMetaData.FACTORY); - registerFactory(RestoreMetaData.TYPE, RestoreMetaData.FACTORY); + registerPrototype(RepositoriesMetaData.TYPE, RepositoriesMetaData.PROTO); + registerPrototype(SnapshotMetaData.TYPE, SnapshotMetaData.PROTO); + registerPrototype(RestoreMetaData.TYPE, RestoreMetaData.PROTO); } /** * Register a custom index meta data factory. Make sure to call it from a static block. */ - public static void registerFactory(String type, Custom.Factory factory) { - customFactories.put(type, factory); + public static void registerPrototype(String type, Custom proto) { + customPrototypes.put(type, proto); } @Nullable - public static Custom.Factory lookupFactory(String type) { - return customFactories.get(type); + public static T lookupPrototype(String type) { + //noinspection unchecked + return (T) customPrototypes.get(type); } - public static Custom.Factory lookupFactorySafe(String type) { - Custom.Factory factory = customFactories.get(type); - if (factory == null) { - throw new IllegalArgumentException("No custom index metadata factory registered for type [" + type + "]"); + public static T lookupPrototypeSafe(String type) { + //noinspection unchecked + T proto = (T) customPrototypes.get(type); + if (proto == null) { + throw new IllegalArgumentException("No custom metadata prototype registered for type [" + type + "]"); } - return factory; + return proto; } @@ -644,14 +639,22 @@ public class MetaData implements Iterable { /** * Translates the provided indices or aliases, eventually containing wildcard expressions, into actual indices. * - * @param indicesOptions how the aliases or indices need to be resolved to concrete indices + * @param indicesOptions how the aliases or indices need to be resolved to concrete indices * @param aliasesOrIndices the aliases or indices to be resolved to concrete indices * @return the obtained concrete indices +<<<<<<< HEAD * @throws IndexMissingException if one of the aliases or indices is missing and the provided indices options * don't allow such a case, or if the final result of the indices resolution is no indices and the indices options * don't allow such a case. * @throws IllegalArgumentException if one of the aliases resolve to multiple indices and the provided * indices options don't allow such a case. +======= + * @throws IndexMissingException if one of the aliases or indices is missing and the provided indices options + * don't allow such a case, or if the final result of the indices resolution is no indices and the indices options + * don't allow such a case. + * @throws ElasticsearchIllegalArgumentException if one of the aliases resolve to multiple indices and the provided + * indices options don't allow such a case. +>>>>>>> Add support for cluster state diffs */ public String[] concreteIndices(IndicesOptions indicesOptions, String... aliasesOrIndices) throws IndexMissingException, IllegalArgumentException { if (indicesOptions.expandWildcardsOpen() || indicesOptions.expandWildcardsClosed()) { @@ -1139,14 +1142,14 @@ public class MetaData implements Iterable { // Check if any persistent metadata needs to be saved int customCount1 = 0; for (ObjectObjectCursor cursor : metaData1.customs) { - if (customFactories.get(cursor.key).context().contains(XContentContext.GATEWAY)) { + if (customPrototypes.get(cursor.key).context().contains(XContentContext.GATEWAY)) { if (!cursor.value.equals(metaData2.custom(cursor.key))) return false; customCount1++; } } int customCount2 = 0; for (ObjectObjectCursor cursor : metaData2.customs) { - if (customFactories.get(cursor.key).context().contains(XContentContext.GATEWAY)) { + if (customPrototypes.get(cursor.key).context().contains(XContentContext.GATEWAY)) { customCount2++; } } @@ -1154,6 +1157,129 @@ public class MetaData implements Iterable { return true; } + @Override + public Diff diff(MetaData previousState) { + return new MetaDataDiff(previousState, this); + } + + @Override + public Diff readDiffFrom(StreamInput in) throws IOException { + return new MetaDataDiff(in); + } + + private static class MetaDataDiff implements Diff { + + private long version; + + private String uuid; + + private Settings transientSettings; + private Settings persistentSettings; + private Diff> indices; + private Diff> templates; + private Diff> customs; + + + public MetaDataDiff(MetaData before, MetaData after) { + uuid = after.uuid; + version = after.version; + transientSettings = after.transientSettings; + persistentSettings = after.persistentSettings; + indices = DiffableUtils.diff(before.indices, after.indices); + templates = DiffableUtils.diff(before.templates, after.templates); + customs = DiffableUtils.diff(before.customs, after.customs); + } + + public MetaDataDiff(StreamInput in) throws IOException { + uuid = in.readString(); + version = in.readLong(); + transientSettings = ImmutableSettings.readSettingsFromStream(in); + persistentSettings = ImmutableSettings.readSettingsFromStream(in); + indices = DiffableUtils.readImmutableOpenMapDiff(in, IndexMetaData.PROTO); + templates = DiffableUtils.readImmutableOpenMapDiff(in, IndexTemplateMetaData.PROTO); + customs = DiffableUtils.readImmutableOpenMapDiff(in, new KeyedReader() { + @Override + public Custom readFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readFrom(in); + } + + @Override + public Diff readDiffFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readDiffFrom(in); + } + }); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(uuid); + out.writeLong(version); + ImmutableSettings.writeSettingsToStream(transientSettings, out); + ImmutableSettings.writeSettingsToStream(persistentSettings, out); + indices.writeTo(out); + templates.writeTo(out); + customs.writeTo(out); + } + + @Override + public MetaData apply(MetaData part) { + Builder builder = builder(); + builder.uuid(uuid); + builder.version(version); + builder.transientSettings(transientSettings); + builder.persistentSettings(persistentSettings); + builder.indices(indices.apply(part.indices)); + builder.templates(templates.apply(part.templates)); + builder.customs(customs.apply(part.customs)); + return builder.build(); + } + } + + @Override + public MetaData readFrom(StreamInput in) throws IOException { + Builder builder = new Builder(); + builder.version = in.readLong(); + builder.uuid = in.readString(); + builder.transientSettings(readSettingsFromStream(in)); + builder.persistentSettings(readSettingsFromStream(in)); + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + builder.put(IndexMetaData.Builder.readFrom(in), false); + } + size = in.readVInt(); + for (int i = 0; i < size; i++) { + builder.put(IndexTemplateMetaData.Builder.readFrom(in)); + } + int customSize = in.readVInt(); + for (int i = 0; i < customSize; i++) { + String type = in.readString(); + Custom customIndexMetaData = lookupPrototypeSafe(type).readFrom(in); + builder.putCustom(type, customIndexMetaData); + } + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(version); + out.writeString(uuid); + writeSettingsToStream(transientSettings, out); + writeSettingsToStream(persistentSettings, out); + out.writeVInt(indices.size()); + for (IndexMetaData indexMetaData : this) { + indexMetaData.writeTo(out); + } + out.writeVInt(templates.size()); + for (ObjectCursor cursor : templates.values()) { + cursor.value.writeTo(out); + } + out.writeVInt(customs.size()); + for (ObjectObjectCursor cursor : customs) { + out.writeString(cursor.key); + cursor.value.writeTo(out); + } + } + public static Builder builder() { return new Builder(); } @@ -1225,6 +1351,11 @@ public class MetaData implements Iterable { return this; } + public Builder indices(ImmutableOpenMap indices) { + this.indices.putAll(indices); + return this; + } + public Builder put(IndexTemplateMetaData.Builder template) { return put(template.build()); } @@ -1239,6 +1370,11 @@ public class MetaData implements Iterable { return this; } + public Builder templates(ImmutableOpenMap templates) { + this.templates.putAll(templates); + return this; + } + public Custom getCustom(String type) { return customs.get(type); } @@ -1253,6 +1389,11 @@ public class MetaData implements Iterable { return this; } + public Builder customs(ImmutableOpenMap customs) { + this.customs.putAll(customs); + return this; + } + public Builder updateSettings(Settings settings, String... indices) { if (indices == null || indices.length == 0) { indices = this.indices.keys().toArray(String.class); @@ -1305,6 +1446,11 @@ public class MetaData implements Iterable { return this; } + public Builder uuid(String uuid) { + this.uuid = uuid; + return this; + } + public Builder generateUuidIfNeeded() { if (uuid.equals("_na_")) { uuid = Strings.randomBase64UUID(); @@ -1363,10 +1509,10 @@ public class MetaData implements Iterable { } for (ObjectObjectCursor cursor : metaData.customs()) { - Custom.Factory factory = lookupFactorySafe(cursor.key); - if (factory.context().contains(context)) { + Custom proto = lookupPrototypeSafe(cursor.key); + if (proto.context().contains(context)) { builder.startObject(cursor.key); - factory.toXContent(cursor.value, builder, params); + cursor.value.toXContent(builder, params); builder.endObject(); } } @@ -1410,12 +1556,13 @@ public class MetaData implements Iterable { } } else { // check if its a custom index metadata - Custom.Factory factory = lookupFactory(currentFieldName); - if (factory == null) { + Custom proto = lookupPrototype(currentFieldName); + if (proto == null) { //TODO warn parser.skipChildren(); } else { - builder.putCustom(factory.type(), factory.fromXContent(parser)); + Custom custom = proto.fromXContent(parser); + builder.putCustom(custom.type(), custom); } } } else if (token.isValue()) { @@ -1430,46 +1577,7 @@ public class MetaData implements Iterable { } public static MetaData readFrom(StreamInput in) throws IOException { - Builder builder = new Builder(); - builder.version = in.readLong(); - builder.uuid = in.readString(); - builder.transientSettings(readSettingsFromStream(in)); - builder.persistentSettings(readSettingsFromStream(in)); - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - builder.put(IndexMetaData.Builder.readFrom(in), false); - } - size = in.readVInt(); - for (int i = 0; i < size; i++) { - builder.put(IndexTemplateMetaData.Builder.readFrom(in)); - } - int customSize = in.readVInt(); - for (int i = 0; i < customSize; i++) { - String type = in.readString(); - Custom customIndexMetaData = lookupFactorySafe(type).readFrom(in); - builder.putCustom(type, customIndexMetaData); - } - return builder.build(); - } - - public static void writeTo(MetaData metaData, StreamOutput out) throws IOException { - out.writeLong(metaData.version); - out.writeString(metaData.uuid); - writeSettingsToStream(metaData.transientSettings(), out); - writeSettingsToStream(metaData.persistentSettings(), out); - out.writeVInt(metaData.indices.size()); - for (IndexMetaData indexMetaData : metaData) { - IndexMetaData.Builder.writeTo(indexMetaData, out); - } - out.writeVInt(metaData.templates.size()); - for (ObjectCursor cursor : metaData.templates.values()) { - IndexTemplateMetaData.Builder.writeTo(cursor.value, out); - } - out.writeVInt(metaData.customs().size()); - for (ObjectObjectCursor cursor : metaData.customs()) { - out.writeString(cursor.key); - lookupFactorySafe(cursor.key).writeTo(cursor.value, out); - } + return PROTO.readFrom(in); } } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index 9fcb5182180..732561f66f1 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -272,7 +272,7 @@ public class MetaDataCreateIndexService extends AbstractComponent { if (existing == null) { customs.put(type, custom); } else { - IndexMetaData.Custom merged = IndexMetaData.lookupFactorySafe(type).merge(existing, custom); + IndexMetaData.Custom merged = existing.mergeWith(custom); customs.put(type, merged); } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java index 81b11fc14b1..51cd5db086b 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java @@ -21,6 +21,8 @@ package org.elasticsearch.cluster.metadata; import com.google.common.collect.ImmutableList; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.cluster.AbstractDiffable; +import org.elasticsearch.cluster.metadata.MetaData.Custom; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.ImmutableSettings; @@ -39,11 +41,11 @@ import java.util.Map; /** * Contains metadata about registered snapshot repositories */ -public class RepositoriesMetaData implements MetaData.Custom { +public class RepositoriesMetaData extends AbstractDiffable implements MetaData.Custom { public static final String TYPE = "repositories"; - public static final Factory FACTORY = new Factory(); + public static final RepositoriesMetaData PROTO = new RepositoriesMetaData(); private final ImmutableList repositories; @@ -80,122 +82,132 @@ public class RepositoriesMetaData implements MetaData.Custom { return null; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + RepositoriesMetaData that = (RepositoriesMetaData) o; + + return repositories.equals(that.repositories); + + } + + @Override + public int hashCode() { + return repositories.hashCode(); + } + /** - * Repository metadata factory + * {@inheritDoc} */ - public static class Factory extends MetaData.Custom.Factory { + @Override + public String type() { + return TYPE; + } - /** - * {@inheritDoc} - */ - @Override - public String type() { - return TYPE; + /** + * {@inheritDoc} + */ + @Override + public Custom readFrom(StreamInput in) throws IOException { + RepositoryMetaData[] repository = new RepositoryMetaData[in.readVInt()]; + for (int i = 0; i < repository.length; i++) { + repository[i] = RepositoryMetaData.readFrom(in); } + return new RepositoriesMetaData(repository); + } - /** - * {@inheritDoc} - */ - @Override - public RepositoriesMetaData readFrom(StreamInput in) throws IOException { - RepositoryMetaData[] repository = new RepositoryMetaData[in.readVInt()]; - for (int i = 0; i < repository.length; i++) { - repository[i] = RepositoryMetaData.readFrom(in); - } - return new RepositoriesMetaData(repository); - } - - /** - * {@inheritDoc} - */ - @Override - public void writeTo(RepositoriesMetaData repositories, StreamOutput out) throws IOException { - out.writeVInt(repositories.repositories().size()); - for (RepositoryMetaData repository : repositories.repositories()) { - repository.writeTo(out); - } - } - - /** - * {@inheritDoc} - */ - @Override - public RepositoriesMetaData fromXContent(XContentParser parser) throws IOException { - XContentParser.Token token; - List repository = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - String name = parser.currentName(); - if (parser.nextToken() != XContentParser.Token.START_OBJECT) { - throw new ElasticsearchParseException("failed to parse repository [" + name + "], expected object"); - } - String type = null; - Settings settings = ImmutableSettings.EMPTY; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - String currentFieldName = parser.currentName(); - if ("type".equals(currentFieldName)) { - if (parser.nextToken() != XContentParser.Token.VALUE_STRING) { - throw new ElasticsearchParseException("failed to parse repository [" + name + "], unknown type"); - } - type = parser.text(); - } else if ("settings".equals(currentFieldName)) { - if (parser.nextToken() != XContentParser.Token.START_OBJECT) { - throw new ElasticsearchParseException("failed to parse repository [" + name + "], incompatible params"); - } - settings = ImmutableSettings.settingsBuilder().put(SettingsLoader.Helper.loadNestedFromMap(parser.mapOrdered())).build(); - } else { - throw new ElasticsearchParseException("failed to parse repository [" + name + "], unknown field [" + currentFieldName + "]"); - } - } else { - throw new ElasticsearchParseException("failed to parse repository [" + name + "]"); - } - } - if (type == null) { - throw new ElasticsearchParseException("failed to parse repository [" + name + "], missing repository type"); - } - repository.add(new RepositoryMetaData(name, type, settings)); - } else { - throw new ElasticsearchParseException("failed to parse repositories"); - } - } - return new RepositoriesMetaData(repository.toArray(new RepositoryMetaData[repository.size()])); - } - - /** - * {@inheritDoc} - */ - @Override - public void toXContent(RepositoriesMetaData customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException { - for (RepositoryMetaData repository : customIndexMetaData.repositories()) { - toXContent(repository, builder, params); - } - } - - @Override - public EnumSet context() { - return MetaData.API_AND_GATEWAY; - } - - /** - * Serializes information about a single repository - * - * @param repository repository metadata - * @param builder XContent builder - * @param params serialization parameters - * @throws IOException - */ - public void toXContent(RepositoryMetaData repository, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startObject(repository.name(), XContentBuilder.FieldCaseConversion.NONE); - builder.field("type", repository.type()); - builder.startObject("settings"); - for (Map.Entry settingEntry : repository.settings().getAsMap().entrySet()) { - builder.field(settingEntry.getKey(), settingEntry.getValue()); - } - builder.endObject(); - - builder.endObject(); + /** + * {@inheritDoc} + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(repositories.size()); + for (RepositoryMetaData repository : repositories) { + repository.writeTo(out); } } + /** + * {@inheritDoc} + */ + @Override + public RepositoriesMetaData fromXContent(XContentParser parser) throws IOException { + XContentParser.Token token; + List repository = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + String name = parser.currentName(); + if (parser.nextToken() != XContentParser.Token.START_OBJECT) { + throw new ElasticsearchParseException("failed to parse repository [" + name + "], expected object"); + } + String type = null; + Settings settings = ImmutableSettings.EMPTY; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + String currentFieldName = parser.currentName(); + if ("type".equals(currentFieldName)) { + if (parser.nextToken() != XContentParser.Token.VALUE_STRING) { + throw new ElasticsearchParseException("failed to parse repository [" + name + "], unknown type"); + } + type = parser.text(); + } else if ("settings".equals(currentFieldName)) { + if (parser.nextToken() != XContentParser.Token.START_OBJECT) { + throw new ElasticsearchParseException("failed to parse repository [" + name + "], incompatible params"); + } + settings = ImmutableSettings.settingsBuilder().put(SettingsLoader.Helper.loadNestedFromMap(parser.mapOrdered())).build(); + } else { + throw new ElasticsearchParseException("failed to parse repository [" + name + "], unknown field [" + currentFieldName + "]"); + } + } else { + throw new ElasticsearchParseException("failed to parse repository [" + name + "]"); + } + } + if (type == null) { + throw new ElasticsearchParseException("failed to parse repository [" + name + "], missing repository type"); + } + repository.add(new RepositoryMetaData(name, type, settings)); + } else { + throw new ElasticsearchParseException("failed to parse repositories"); + } + } + return new RepositoriesMetaData(repository.toArray(new RepositoryMetaData[repository.size()])); + } + + /** + * {@inheritDoc} + */ + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + for (RepositoryMetaData repository : repositories) { + toXContent(repository, builder, params); + } + return builder; + } + + @Override + public EnumSet context() { + return MetaData.API_AND_GATEWAY; + } + + /** + * Serializes information about a single repository + * + * @param repository repository metadata + * @param builder XContent builder + * @param params serialization parameters + * @throws IOException + */ + public static void toXContent(RepositoryMetaData repository, XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(repository.name(), XContentBuilder.FieldCaseConversion.NONE); + builder.field("type", repository.type()); + builder.startObject("settings"); + for (Map.Entry settingEntry : repository.settings().getAsMap().entrySet()) { + builder.field(settingEntry.getKey(), settingEntry.getValue()); + } + builder.endObject(); + + builder.endObject(); + } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java index ea50b30ba88..a283f1f43c1 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java @@ -99,4 +99,25 @@ public class RepositoryMetaData { out.writeString(type); ImmutableSettings.writeSettingsToStream(settings, out); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + RepositoryMetaData that = (RepositoryMetaData) o; + + if (!name.equals(that.name)) return false; + if (!type.equals(that.type)) return false; + return settings.equals(that.settings); + + } + + @Override + public int hashCode() { + int result = name.hashCode(); + result = 31 * result + type.hashCode(); + result = 31 * result + settings.hashCode(); + return result; + } } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/cluster/metadata/RestoreMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/RestoreMetaData.java index 642136d7b7e..51fd5e0514a 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/RestoreMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/RestoreMetaData.java @@ -21,6 +21,7 @@ package org.elasticsearch.cluster.metadata; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContent; @@ -29,16 +30,17 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.shard.ShardId; import java.io.IOException; +import java.util.EnumSet; import java.util.Map; /** * Meta data about restore processes that are currently executing */ -public class RestoreMetaData implements MetaData.Custom { +public class RestoreMetaData extends AbstractDiffable implements MetaData.Custom { public static final String TYPE = "restore"; - public static final Factory FACTORY = new Factory(); + public static final RestoreMetaData PROTO = new RestoreMetaData(); private final ImmutableList entries; @@ -394,124 +396,122 @@ public class RestoreMetaData implements MetaData.Custom { } /** - * Restore metadata factory + * {@inheritDoc} */ - public static class Factory extends MetaData.Custom.Factory { + @Override + public String type() { + return TYPE; + } - /** - * {@inheritDoc} - */ - @Override - public String type() { - return TYPE; - } - - /** - * {@inheritDoc} - */ - @Override - public RestoreMetaData readFrom(StreamInput in) throws IOException { - Entry[] entries = new Entry[in.readVInt()]; - for (int i = 0; i < entries.length; i++) { - SnapshotId snapshotId = SnapshotId.readSnapshotId(in); - State state = State.fromValue(in.readByte()); - int indices = in.readVInt(); - ImmutableList.Builder indexBuilder = ImmutableList.builder(); - for (int j = 0; j < indices; j++) { - indexBuilder.add(in.readString()); - } - ImmutableMap.Builder builder = ImmutableMap.builder(); - int shards = in.readVInt(); - for (int j = 0; j < shards; j++) { - ShardId shardId = ShardId.readShardId(in); - ShardRestoreStatus shardState = ShardRestoreStatus.readShardRestoreStatus(in); - builder.put(shardId, shardState); - } - entries[i] = new Entry(snapshotId, state, indexBuilder.build(), builder.build()); + /** + * {@inheritDoc} + */ + @Override + public RestoreMetaData readFrom(StreamInput in) throws IOException { + Entry[] entries = new Entry[in.readVInt()]; + for (int i = 0; i < entries.length; i++) { + SnapshotId snapshotId = SnapshotId.readSnapshotId(in); + State state = State.fromValue(in.readByte()); + int indices = in.readVInt(); + ImmutableList.Builder indexBuilder = ImmutableList.builder(); + for (int j = 0; j < indices; j++) { + indexBuilder.add(in.readString()); } - return new RestoreMetaData(entries); - } - - /** - * {@inheritDoc} - */ - @Override - public void writeTo(RestoreMetaData repositories, StreamOutput out) throws IOException { - out.writeVInt(repositories.entries().size()); - for (Entry entry : repositories.entries()) { - entry.snapshotId().writeTo(out); - out.writeByte(entry.state().value()); - out.writeVInt(entry.indices().size()); - for (String index : entry.indices()) { - out.writeString(index); - } - out.writeVInt(entry.shards().size()); - for (Map.Entry shardEntry : entry.shards().entrySet()) { - shardEntry.getKey().writeTo(out); - shardEntry.getValue().writeTo(out); - } + ImmutableMap.Builder builder = ImmutableMap.builder(); + int shards = in.readVInt(); + for (int j = 0; j < shards; j++) { + ShardId shardId = ShardId.readShardId(in); + ShardRestoreStatus shardState = ShardRestoreStatus.readShardRestoreStatus(in); + builder.put(shardId, shardState); } + entries[i] = new Entry(snapshotId, state, indexBuilder.build(), builder.build()); } + return new RestoreMetaData(entries); + } - /** - * {@inheritDoc} - */ - @Override - public RestoreMetaData fromXContent(XContentParser parser) throws IOException { - throw new UnsupportedOperationException(); - } - - /** - * {@inheritDoc} - */ - @Override - public void toXContent(RestoreMetaData customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startArray("snapshots"); - for (Entry entry : customIndexMetaData.entries()) { - toXContent(entry, builder, params); + /** + * {@inheritDoc} + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(entries.size()); + for (Entry entry : entries) { + entry.snapshotId().writeTo(out); + out.writeByte(entry.state().value()); + out.writeVInt(entry.indices().size()); + for (String index : entry.indices()) { + out.writeString(index); } - builder.endArray(); - } - - /** - * Serializes single restore operation - * - * @param entry restore operation metadata - * @param builder XContent builder - * @param params serialization parameters - * @throws IOException - */ - public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startObject(); - builder.field("snapshot", entry.snapshotId().getSnapshot()); - builder.field("repository", entry.snapshotId().getRepository()); - builder.field("state", entry.state()); - builder.startArray("indices"); - { - for (String index : entry.indices()) { - builder.value(index); - } + out.writeVInt(entry.shards().size()); + for (Map.Entry shardEntry : entry.shards().entrySet()) { + shardEntry.getKey().writeTo(out); + shardEntry.getValue().writeTo(out); } - builder.endArray(); - builder.startArray("shards"); - { - for (Map.Entry shardEntry : entry.shards.entrySet()) { - ShardId shardId = shardEntry.getKey(); - ShardRestoreStatus status = shardEntry.getValue(); - builder.startObject(); - { - builder.field("index", shardId.getIndex()); - builder.field("shard", shardId.getId()); - builder.field("state", status.state()); - } - builder.endObject(); - } - } - - builder.endArray(); - builder.endObject(); } } + /** + * {@inheritDoc} + */ + @Override + public RestoreMetaData fromXContent(XContentParser parser) throws IOException { + throw new UnsupportedOperationException(); + } + @Override + public EnumSet context() { + return MetaData.API_ONLY; + } + + /** + * {@inheritDoc} + */ + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startArray("snapshots"); + for (Entry entry : entries) { + toXContent(entry, builder, params); + } + builder.endArray(); + return builder; + } + + /** + * Serializes single restore operation + * + * @param entry restore operation metadata + * @param builder XContent builder + * @param params serialization parameters + * @throws IOException + */ + public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field("snapshot", entry.snapshotId().getSnapshot()); + builder.field("repository", entry.snapshotId().getRepository()); + builder.field("state", entry.state()); + builder.startArray("indices"); + { + for (String index : entry.indices()) { + builder.value(index); + } + } + builder.endArray(); + builder.startArray("shards"); + { + for (Map.Entry shardEntry : entry.shards.entrySet()) { + ShardId shardId = shardEntry.getKey(); + ShardRestoreStatus status = shardEntry.getValue(); + builder.startObject(); + { + builder.field("index", shardId.getIndex()); + builder.field("shard", shardId.getId()); + builder.field("state", status.state()); + } + builder.endObject(); + } + } + + builder.endArray(); + builder.endObject(); + } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/SnapshotMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/SnapshotMetaData.java index b1bcc92b8bd..b23c58710a0 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/SnapshotMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/SnapshotMetaData.java @@ -21,6 +21,8 @@ package org.elasticsearch.cluster.metadata; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.elasticsearch.cluster.AbstractDiffable; +import org.elasticsearch.cluster.metadata.MetaData.Custom; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContent; @@ -30,6 +32,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.shard.ShardId; import java.io.IOException; +import java.util.EnumSet; import java.util.Map; import static com.google.common.collect.Maps.newHashMap; @@ -37,10 +40,10 @@ import static com.google.common.collect.Maps.newHashMap; /** * Meta data about snapshots that are currently executing */ -public class SnapshotMetaData implements MetaData.Custom { +public class SnapshotMetaData extends AbstractDiffable implements MetaData.Custom { public static final String TYPE = "snapshots"; - public static final Factory FACTORY = new Factory(); + public static final SnapshotMetaData PROTO = new SnapshotMetaData(); @Override public boolean equals(Object o) { @@ -329,123 +332,123 @@ public class SnapshotMetaData implements MetaData.Custom { return null; } + @Override + public String type() { + return TYPE; + } - public static class Factory extends MetaData.Custom.Factory { - - @Override - public String type() { - return TYPE; //To change body of implemented methods use File | Settings | File Templates. - } - - @Override - public SnapshotMetaData readFrom(StreamInput in) throws IOException { - Entry[] entries = new Entry[in.readVInt()]; - for (int i = 0; i < entries.length; i++) { - SnapshotId snapshotId = SnapshotId.readSnapshotId(in); - boolean includeGlobalState = in.readBoolean(); - State state = State.fromValue(in.readByte()); - int indices = in.readVInt(); - ImmutableList.Builder indexBuilder = ImmutableList.builder(); - for (int j = 0; j < indices; j++) { - indexBuilder.add(in.readString()); - } - long startTime = in.readLong(); - ImmutableMap.Builder builder = ImmutableMap.builder(); - int shards = in.readVInt(); - for (int j = 0; j < shards; j++) { - ShardId shardId = ShardId.readShardId(in); - String nodeId = in.readOptionalString(); - State shardState = State.fromValue(in.readByte()); - builder.put(shardId, new ShardSnapshotStatus(nodeId, shardState)); - } - entries[i] = new Entry(snapshotId, includeGlobalState, state, indexBuilder.build(), startTime, builder.build()); + @Override + public SnapshotMetaData readFrom(StreamInput in) throws IOException { + Entry[] entries = new Entry[in.readVInt()]; + for (int i = 0; i < entries.length; i++) { + SnapshotId snapshotId = SnapshotId.readSnapshotId(in); + boolean includeGlobalState = in.readBoolean(); + State state = State.fromValue(in.readByte()); + int indices = in.readVInt(); + ImmutableList.Builder indexBuilder = ImmutableList.builder(); + for (int j = 0; j < indices; j++) { + indexBuilder.add(in.readString()); } - return new SnapshotMetaData(entries); - } - - @Override - public void writeTo(SnapshotMetaData repositories, StreamOutput out) throws IOException { - out.writeVInt(repositories.entries().size()); - for (Entry entry : repositories.entries()) { - entry.snapshotId().writeTo(out); - out.writeBoolean(entry.includeGlobalState()); - out.writeByte(entry.state().value()); - out.writeVInt(entry.indices().size()); - for (String index : entry.indices()) { - out.writeString(index); - } - out.writeLong(entry.startTime()); - out.writeVInt(entry.shards().size()); - for (Map.Entry shardEntry : entry.shards().entrySet()) { - shardEntry.getKey().writeTo(out); - out.writeOptionalString(shardEntry.getValue().nodeId()); - out.writeByte(shardEntry.getValue().state().value()); - } + long startTime = in.readLong(); + ImmutableMap.Builder builder = ImmutableMap.builder(); + int shards = in.readVInt(); + for (int j = 0; j < shards; j++) { + ShardId shardId = ShardId.readShardId(in); + String nodeId = in.readOptionalString(); + State shardState = State.fromValue(in.readByte()); + builder.put(shardId, new ShardSnapshotStatus(nodeId, shardState)); } + entries[i] = new Entry(snapshotId, includeGlobalState, state, indexBuilder.build(), startTime, builder.build()); } + return new SnapshotMetaData(entries); + } - @Override - public SnapshotMetaData fromXContent(XContentParser parser) throws IOException { - throw new UnsupportedOperationException(); - } - - static final class Fields { - static final XContentBuilderString REPOSITORY = new XContentBuilderString("repository"); - static final XContentBuilderString SNAPSHOTS = new XContentBuilderString("snapshots"); - static final XContentBuilderString SNAPSHOT = new XContentBuilderString("snapshot"); - static final XContentBuilderString INCLUDE_GLOBAL_STATE = new XContentBuilderString("include_global_state"); - static final XContentBuilderString STATE = new XContentBuilderString("state"); - static final XContentBuilderString INDICES = new XContentBuilderString("indices"); - static final XContentBuilderString START_TIME_MILLIS = new XContentBuilderString("start_time_millis"); - static final XContentBuilderString START_TIME = new XContentBuilderString("start_time"); - static final XContentBuilderString SHARDS = new XContentBuilderString("shards"); - static final XContentBuilderString INDEX = new XContentBuilderString("index"); - static final XContentBuilderString SHARD = new XContentBuilderString("shard"); - static final XContentBuilderString NODE = new XContentBuilderString("node"); - } - - @Override - public void toXContent(SnapshotMetaData customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startArray(Fields.SNAPSHOTS); - for (Entry entry : customIndexMetaData.entries()) { - toXContent(entry, builder, params); + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(entries.size()); + for (Entry entry : entries) { + entry.snapshotId().writeTo(out); + out.writeBoolean(entry.includeGlobalState()); + out.writeByte(entry.state().value()); + out.writeVInt(entry.indices().size()); + for (String index : entry.indices()) { + out.writeString(index); } - builder.endArray(); - } - - public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startObject(); - builder.field(Fields.REPOSITORY, entry.snapshotId().getRepository()); - builder.field(Fields.SNAPSHOT, entry.snapshotId().getSnapshot()); - builder.field(Fields.INCLUDE_GLOBAL_STATE, entry.includeGlobalState()); - builder.field(Fields.STATE, entry.state()); - builder.startArray(Fields.INDICES); - { - for (String index : entry.indices()) { - builder.value(index); - } + out.writeLong(entry.startTime()); + out.writeVInt(entry.shards().size()); + for (Map.Entry shardEntry : entry.shards().entrySet()) { + shardEntry.getKey().writeTo(out); + out.writeOptionalString(shardEntry.getValue().nodeId()); + out.writeByte(shardEntry.getValue().state().value()); } - builder.endArray(); - builder.timeValueField(Fields.START_TIME_MILLIS, Fields.START_TIME, entry.startTime()); - builder.startArray(Fields.SHARDS); - { - for (Map.Entry shardEntry : entry.shards.entrySet()) { - ShardId shardId = shardEntry.getKey(); - ShardSnapshotStatus status = shardEntry.getValue(); - builder.startObject(); - { - builder.field(Fields.INDEX, shardId.getIndex()); - builder.field(Fields.SHARD, shardId.getId()); - builder.field(Fields.STATE, status.state()); - builder.field(Fields.NODE, status.nodeId()); - } - builder.endObject(); - } - } - builder.endArray(); - builder.endObject(); } } + @Override + public SnapshotMetaData fromXContent(XContentParser parser) throws IOException { + throw new UnsupportedOperationException(); + } + @Override + public EnumSet context() { + return MetaData.API_ONLY; + } + + static final class Fields { + static final XContentBuilderString REPOSITORY = new XContentBuilderString("repository"); + static final XContentBuilderString SNAPSHOTS = new XContentBuilderString("snapshots"); + static final XContentBuilderString SNAPSHOT = new XContentBuilderString("snapshot"); + static final XContentBuilderString INCLUDE_GLOBAL_STATE = new XContentBuilderString("include_global_state"); + static final XContentBuilderString STATE = new XContentBuilderString("state"); + static final XContentBuilderString INDICES = new XContentBuilderString("indices"); + static final XContentBuilderString START_TIME_MILLIS = new XContentBuilderString("start_time_millis"); + static final XContentBuilderString START_TIME = new XContentBuilderString("start_time"); + static final XContentBuilderString SHARDS = new XContentBuilderString("shards"); + static final XContentBuilderString INDEX = new XContentBuilderString("index"); + static final XContentBuilderString SHARD = new XContentBuilderString("shard"); + static final XContentBuilderString NODE = new XContentBuilderString("node"); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startArray(Fields.SNAPSHOTS); + for (Entry entry : entries) { + toXContent(entry, builder, params); + } + builder.endArray(); + return builder; + } + + public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field(Fields.REPOSITORY, entry.snapshotId().getRepository()); + builder.field(Fields.SNAPSHOT, entry.snapshotId().getSnapshot()); + builder.field(Fields.INCLUDE_GLOBAL_STATE, entry.includeGlobalState()); + builder.field(Fields.STATE, entry.state()); + builder.startArray(Fields.INDICES); + { + for (String index : entry.indices()) { + builder.value(index); + } + } + builder.endArray(); + builder.timeValueField(Fields.START_TIME_MILLIS, Fields.START_TIME, entry.startTime()); + builder.startArray(Fields.SHARDS); + { + for (Map.Entry shardEntry : entry.shards.entrySet()) { + ShardId shardId = shardEntry.getKey(); + ShardSnapshotStatus status = shardEntry.getValue(); + builder.startObject(); + { + builder.field(Fields.INDEX, shardId.getIndex()); + builder.field(Fields.SHARD, shardId.getId()); + builder.field(Fields.STATE, status.state()); + builder.field(Fields.NODE, status.nodeId()); + } + builder.endObject(); + } + } + builder.endArray(); + builder.endObject(); + } } diff --git a/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index 2831af8183d..8692e5fb006 100644 --- a/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -25,6 +25,7 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.collect.ImmutableList; import com.google.common.collect.UnmodifiableIterator; import org.elasticsearch.Version; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.ImmutableOpenMap; @@ -44,9 +45,10 @@ import static com.google.common.collect.Lists.newArrayList; * This class holds all {@link DiscoveryNode} in the cluster and provides convenience methods to * access, modify merge / diff discovery nodes. */ -public class DiscoveryNodes implements Iterable { +public class DiscoveryNodes extends AbstractDiffable implements Iterable { public static final DiscoveryNodes EMPTY_NODES = builder().build(); + public static final DiscoveryNodes PROTO = EMPTY_NODES; private final ImmutableOpenMap nodes; private final ImmutableOpenMap dataNodes; @@ -567,6 +569,44 @@ public class DiscoveryNodes implements Iterable { } } + public void writeTo(StreamOutput out) throws IOException { + if (masterNodeId == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeString(masterNodeId); + } + out.writeVInt(nodes.size()); + for (DiscoveryNode node : this) { + node.writeTo(out); + } + } + + public DiscoveryNodes readFrom(StreamInput in, DiscoveryNode localNode) throws IOException { + Builder builder = new Builder(); + if (in.readBoolean()) { + builder.masterNodeId(in.readString()); + } + if (localNode != null) { + builder.localNodeId(localNode.id()); + } + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + DiscoveryNode node = DiscoveryNode.readNode(in); + if (localNode != null && node.id().equals(localNode.id())) { + // reuse the same instance of our address and local node id for faster equality + node = localNode; + } + builder.put(node); + } + return builder.build(); + } + + @Override + public DiscoveryNodes readFrom(StreamInput in) throws IOException { + return readFrom(in, localNode()); + } + public static Builder builder() { return new Builder(); } @@ -631,37 +671,8 @@ public class DiscoveryNodes implements Iterable { return new DiscoveryNodes(nodes.build(), dataNodesBuilder.build(), masterNodesBuilder.build(), masterNodeId, localNodeId, minNodeVersion, minNonClientNodeVersion); } - public static void writeTo(DiscoveryNodes nodes, StreamOutput out) throws IOException { - if (nodes.masterNodeId() == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - out.writeString(nodes.masterNodeId); - } - out.writeVInt(nodes.size()); - for (DiscoveryNode node : nodes) { - node.writeTo(out); - } - } - public static DiscoveryNodes readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException { - Builder builder = new Builder(); - if (in.readBoolean()) { - builder.masterNodeId(in.readString()); - } - if (localNode != null) { - builder.localNodeId(localNode.id()); - } - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - DiscoveryNode node = DiscoveryNode.readNode(in); - if (localNode != null && node.id().equals(localNode.id())) { - // reuse the same instance of our address and local node id for faster equality - node = localNode; - } - builder.put(node); - } - return builder.build(); + return PROTO.readFrom(in, localNode); } } } diff --git a/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java index de4ed5434e1..6aaa260c4b5 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java @@ -25,6 +25,7 @@ import com.carrotsearch.hppc.cursors.IntObjectCursor; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import com.google.common.collect.UnmodifiableIterator; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.collect.ImmutableOpenIntMap; @@ -55,7 +56,9 @@ import static com.google.common.collect.Lists.newArrayList; * represented as {@link ShardRouting}. *

*/ -public class IndexRoutingTable implements Iterable { +public class IndexRoutingTable extends AbstractDiffable implements Iterable { + + public static final IndexRoutingTable PROTO = builder("").build(); private final String index; private final ShardShuffler shuffler; @@ -314,9 +317,51 @@ public class IndexRoutingTable implements Iterable { return new GroupShardsIterator(set); } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + IndexRoutingTable that = (IndexRoutingTable) o; + + if (!index.equals(that.index)) return false; + if (!shards.equals(that.shards)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = index.hashCode(); + result = 31 * result + shards.hashCode(); + return result; + } + public void validate() throws RoutingValidationException { } + @Override + public IndexRoutingTable readFrom(StreamInput in) throws IOException { + String index = in.readString(); + Builder builder = new Builder(index); + + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + builder.addIndexShard(IndexShardRoutingTable.Builder.readFromThin(in, index)); + } + + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(index); + out.writeVInt(shards.size()); + for (IndexShardRoutingTable indexShard : this) { + IndexShardRoutingTable.Builder.writeToThin(indexShard, out); + } + } + public static Builder builder(String index) { return new Builder(index); } @@ -338,30 +383,7 @@ public class IndexRoutingTable implements Iterable { * @throws IOException if something happens during read */ public static IndexRoutingTable readFrom(StreamInput in) throws IOException { - String index = in.readString(); - Builder builder = new Builder(index); - - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - builder.addIndexShard(IndexShardRoutingTable.Builder.readFromThin(in, index)); - } - - return builder.build(); - } - - /** - * Writes an {@link IndexRoutingTable} to a {@link StreamOutput}. - * - * @param index {@link IndexRoutingTable} to write - * @param out {@link StreamOutput} to write to - * @throws IOException if something happens during write - */ - public static void writeTo(IndexRoutingTable index, StreamOutput out) throws IOException { - out.writeString(index.index()); - out.writeVInt(index.shards.size()); - for (IndexShardRoutingTable indexShard : index) { - IndexShardRoutingTable.Builder.writeToThin(indexShard, out); - } + return PROTO.readFrom(in); } /** diff --git a/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java index 00e50b76129..2371b96f5b0 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java @@ -347,6 +347,28 @@ public class IndexShardRoutingTable implements Iterable { return new PlainShardIterator(shardId, ordered); } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + IndexShardRoutingTable that = (IndexShardRoutingTable) o; + + if (primaryAllocatedPostApi != that.primaryAllocatedPostApi) return false; + if (!shardId.equals(that.shardId)) return false; + if (!shards.equals(that.shards)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = shardId.hashCode(); + result = 31 * result + shards.hashCode(); + result = 31 * result + (primaryAllocatedPostApi ? 1 : 0); + return result; + } + /** * Returns true iff all shards in the routing table are started otherwise false */ diff --git a/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index 9f1b5db6c6b..25a8bac2f88 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -21,7 +21,7 @@ package org.elasticsearch.cluster.routing; import com.carrotsearch.hppc.IntSet; import com.google.common.collect.*; -import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.*; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.io.stream.StreamInput; @@ -44,7 +44,9 @@ import static com.google.common.collect.Maps.newHashMap; * * @see IndexRoutingTable */ -public class RoutingTable implements Iterable { +public class RoutingTable implements Iterable, Diffable { + + public static RoutingTable PROTO = builder().build(); public static final RoutingTable EMPTY_ROUTING_TABLE = builder().build(); @@ -254,6 +256,66 @@ public class RoutingTable implements Iterable { return new GroupShardsIterator(set); } + @Override + public Diff diff(RoutingTable previousState) { + return new RoutingTableDiff(previousState, this); + } + + @Override + public Diff readDiffFrom(StreamInput in) throws IOException { + return new RoutingTableDiff(in); + } + + @Override + public RoutingTable readFrom(StreamInput in) throws IOException { + Builder builder = new Builder(); + builder.version = in.readLong(); + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + IndexRoutingTable index = IndexRoutingTable.Builder.readFrom(in); + builder.add(index); + } + + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(version); + out.writeVInt(indicesRouting.size()); + for (IndexRoutingTable index : indicesRouting.values()) { + index.writeTo(out); + } + } + + private static class RoutingTableDiff implements Diff { + + private final long version; + + private final Diff> indicesRouting; + + public RoutingTableDiff(RoutingTable before, RoutingTable after) { + version = after.version; + indicesRouting = DiffableUtils.diff(before.indicesRouting, after.indicesRouting); + } + + public RoutingTableDiff(StreamInput in) throws IOException { + version = in.readLong(); + indicesRouting = DiffableUtils.readImmutableMapDiff(in, IndexRoutingTable.PROTO); + } + + @Override + public RoutingTable apply(RoutingTable part) { + return new RoutingTable(version, indicesRouting.apply(part.indicesRouting)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(version); + indicesRouting.writeTo(out); + } + } + public static Builder builder() { return new Builder(); } @@ -403,6 +465,11 @@ public class RoutingTable implements Iterable { return this; } + public Builder indicesRouting(ImmutableMap indicesRouting) { + this.indicesRouting.putAll(indicesRouting); + return this; + } + public Builder remove(String index) { indicesRouting.remove(index); return this; @@ -422,23 +489,7 @@ public class RoutingTable implements Iterable { } public static RoutingTable readFrom(StreamInput in) throws IOException { - Builder builder = new Builder(); - builder.version = in.readLong(); - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - IndexRoutingTable index = IndexRoutingTable.Builder.readFrom(in); - builder.add(index); - } - - return builder.build(); - } - - public static void writeTo(RoutingTable table, StreamOutput out) throws IOException { - out.writeLong(table.version); - out.writeVInt(table.indicesRouting.size()); - for (IndexRoutingTable index : table.indicesRouting.values()) { - IndexRoutingTable.Builder.writeTo(index, out); - } + return PROTO.readFrom(in); } } @@ -450,5 +501,4 @@ public class RoutingTable implements Iterable { return sb.toString(); } - } diff --git a/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java b/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java index 17350ba6c04..b1823e5d74e 100644 --- a/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java +++ b/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java @@ -401,7 +401,7 @@ public class InternalClusterService extends AbstractLifecycleComponent { + /** + * Reads a copy of an object with the same type form the stream input + * + * The caller object remains unchanged. + */ + T readFrom(StreamInput in) throws IOException; +} diff --git a/src/main/java/org/elasticsearch/common/io/stream/Writeable.java b/src/main/java/org/elasticsearch/common/io/stream/Writeable.java new file mode 100644 index 00000000000..9025315dc43 --- /dev/null +++ b/src/main/java/org/elasticsearch/common/io/stream/Writeable.java @@ -0,0 +1,30 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.io.stream; + +import java.io.IOException; + +public interface Writeable extends StreamableReader { + + /** + * Writes the current object into the output stream out + */ + void writeTo(StreamOutput out) throws IOException; +} diff --git a/src/main/java/org/elasticsearch/discovery/Discovery.java b/src/main/java/org/elasticsearch/discovery/Discovery.java index dfd51e6348f..36b8e5da6f5 100644 --- a/src/main/java/org/elasticsearch/discovery/Discovery.java +++ b/src/main/java/org/elasticsearch/discovery/Discovery.java @@ -19,6 +19,7 @@ package org.elasticsearch.discovery; +import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.allocation.AllocationService; @@ -59,7 +60,7 @@ public interface Discovery extends LifecycleComponent { * The {@link AckListener} allows to keep track of the ack received from nodes, and verify whether * they updated their own cluster state or not. */ - void publish(ClusterState clusterState, AckListener ackListener); + void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener); public static interface AckListener { void onNodeAck(DiscoveryNode node, @Nullable Throwable t); diff --git a/src/main/java/org/elasticsearch/discovery/DiscoveryService.java b/src/main/java/org/elasticsearch/discovery/DiscoveryService.java index 1f7207abd5b..a95c313447b 100644 --- a/src/main/java/org/elasticsearch/discovery/DiscoveryService.java +++ b/src/main/java/org/elasticsearch/discovery/DiscoveryService.java @@ -21,6 +21,7 @@ package org.elasticsearch.discovery; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchTimeoutException; +import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -132,9 +133,9 @@ public class DiscoveryService extends AbstractLifecycleComponent implem private static final ConcurrentMap clusterGroups = ConcurrentCollections.newConcurrentMap(); + private volatile ClusterState lastProcessedClusterState; + @Inject public LocalDiscovery(Settings settings, ClusterName clusterName, TransportService transportService, ClusterService clusterService, DiscoveryNodeService discoveryNodeService, Version version, DiscoverySettings discoverySettings) { @@ -273,7 +279,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem } @Override - public void publish(ClusterState clusterState, final Discovery.AckListener ackListener) { + public void publish(ClusterChangedEvent clusterChangedEvent, final Discovery.AckListener ackListener) { if (!master) { throw new IllegalStateException("Shouldn't publish state when not master"); } @@ -286,7 +292,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem } nodesToPublishTo.add(localDiscovery.localNode); } - publish(members, clusterState, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); + publish(members, clusterChangedEvent, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); } } @@ -299,17 +305,47 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem return members.toArray(new LocalDiscovery[members.size()]); } - private void publish(LocalDiscovery[] members, ClusterState clusterState, final BlockingClusterStatePublishResponseHandler publishResponseHandler) { + private void publish(LocalDiscovery[] members, ClusterChangedEvent clusterChangedEvent, final BlockingClusterStatePublishResponseHandler publishResponseHandler) { try { // we do the marshaling intentionally, to check it works well... - final byte[] clusterStateBytes = Builder.toBytes(clusterState); + byte[] clusterStateBytes = null; + byte[] clusterStateDiffBytes = null; + ClusterState clusterState = clusterChangedEvent.state(); for (final LocalDiscovery discovery : members) { if (discovery.master) { continue; } - final ClusterState nodeSpecificClusterState = ClusterState.Builder.fromBytes(clusterStateBytes, discovery.localNode); + ClusterState newNodeSpecificClusterState = null; + synchronized (this) { + // we do the marshaling intentionally, to check it works well... + // check if we publsihed cluster state at least once and node was in the cluster when we published cluster state the last time + if (discovery.lastProcessedClusterState != null && clusterChangedEvent.previousState().nodes().nodeExists(discovery.localNode.id())) { + // both conditions are true - which means we can try sending cluster state as diffs + if (clusterStateDiffBytes == null) { + Diff diff = clusterState.diff(clusterChangedEvent.previousState()); + BytesStreamOutput os = new BytesStreamOutput(); + diff.writeTo(os); + clusterStateDiffBytes = os.bytes().toBytes(); + } + try { + newNodeSpecificClusterState = discovery.lastProcessedClusterState.readDiffFrom(new BytesStreamInput(clusterStateDiffBytes)).apply(discovery.lastProcessedClusterState); + logger.debug("sending diff cluster state version with size {} to [{}]", clusterStateDiffBytes.length, discovery.localNode.getName()); + } catch (IncompatibleClusterStateVersionException ex) { + logger.warn("incompatible cluster state version - resending complete cluster state", ex); + } + } + if (newNodeSpecificClusterState == null) { + if (clusterStateBytes == null) { + clusterStateBytes = Builder.toBytes(clusterState); + } + newNodeSpecificClusterState = ClusterState.Builder.fromBytes(clusterStateBytes, discovery.localNode); + } + discovery.lastProcessedClusterState = newNodeSpecificClusterState; + } + final ClusterState nodeSpecificClusterState = newNodeSpecificClusterState; + nodeSpecificClusterState.status(ClusterState.ClusterStateStatus.RECEIVED); // ignore cluster state messages that do not include "me", not in the game yet... if (nodeSpecificClusterState.nodes().localNode() != null) { diff --git a/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 0defcb7edd5..5bec60abf04 100644 --- a/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -22,7 +22,6 @@ package org.elasticsearch.discovery.zen; import com.google.common.base.Objects; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.*; import org.elasticsearch.cluster.block.ClusterBlocks; @@ -329,12 +328,12 @@ public class ZenDiscovery extends AbstractLifecycleComponent implemen @Override - public void publish(ClusterState clusterState, AckListener ackListener) { - if (!clusterState.getNodes().localNodeMaster()) { + public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) { + if (!clusterChangedEvent.state().getNodes().localNodeMaster()) { throw new IllegalStateException("Shouldn't publish state when not master"); } - nodesFD.updateNodesAndPing(clusterState); - publishClusterState.publish(clusterState, ackListener); + nodesFD.updateNodesAndPing(clusterChangedEvent.state()); + publishClusterState.publish(clusterChangedEvent, ackListener); } /** diff --git a/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java b/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java index fd1ba85c25c..c4ad8895e79 100644 --- a/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java +++ b/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java @@ -21,8 +21,12 @@ package org.elasticsearch.discovery.zen.publish; import com.google.common.collect.Maps; import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.Diff; +import org.elasticsearch.cluster.IncompatibleClusterStateVersionException; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.compress.Compressor; @@ -40,10 +44,13 @@ import org.elasticsearch.discovery.zen.DiscoveryNodesProvider; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.*; +import java.io.IOException; import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; /** * @@ -83,73 +90,43 @@ public class PublishClusterStateAction extends AbstractComponent { transportService.removeHandler(ACTION_NAME); } - public void publish(ClusterState clusterState, final Discovery.AckListener ackListener) { - Set nodesToPublishTo = new HashSet<>(clusterState.nodes().size()); + public void publish(ClusterChangedEvent clusterChangedEvent, final Discovery.AckListener ackListener) { + Set nodesToPublishTo = new HashSet<>(clusterChangedEvent.state().nodes().size()); DiscoveryNode localNode = nodesProvider.nodes().localNode(); - for (final DiscoveryNode node : clusterState.nodes()) { + for (final DiscoveryNode node : clusterChangedEvent.state().nodes()) { if (node.equals(localNode)) { continue; } nodesToPublishTo.add(node); } - publish(clusterState, nodesToPublishTo, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); + publish(clusterChangedEvent, nodesToPublishTo, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); } - private void publish(final ClusterState clusterState, final Set nodesToPublishTo, + private void publish(final ClusterChangedEvent clusterChangedEvent, final Set nodesToPublishTo, final BlockingClusterStatePublishResponseHandler publishResponseHandler) { Map serializedStates = Maps.newHashMap(); + Map serializedDiffs = Maps.newHashMap(); + final ClusterState clusterState = clusterChangedEvent.state(); + final ClusterState previousState = clusterChangedEvent.previousState(); final AtomicBoolean timedOutWaitingForNodes = new AtomicBoolean(false); final TimeValue publishTimeout = discoverySettings.getPublishTimeout(); + final boolean sendFullVersion = !discoverySettings.getPublishDiff() || previousState == null; + Diff diff = null; for (final DiscoveryNode node : nodesToPublishTo) { // try and serialize the cluster state once (or per version), so we don't serialize it // per node when we send it over the wire, compress it while we are at it... - BytesReference bytes = serializedStates.get(node.version()); - if (bytes == null) { - try { - BytesStreamOutput bStream = new BytesStreamOutput(); - StreamOutput stream = CompressorFactory.defaultCompressor().streamOutput(bStream); - stream.setVersion(node.version()); - ClusterState.Builder.writeTo(clusterState, stream); - stream.close(); - bytes = bStream.bytes(); - serializedStates.put(node.version(), bytes); - } catch (Throwable e) { - logger.warn("failed to serialize cluster_state before publishing it to node {}", e, node); - publishResponseHandler.onFailure(node, e); - continue; + // we don't send full version if node didn't exist in the previous version of cluster state + if (sendFullVersion || !previousState.nodes().nodeExists(node.id())) { + sendFullClusterState(clusterState, serializedStates, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler); + } else { + if (diff == null) { + diff = clusterState.diff(previousState); } - } - try { - TransportRequestOptions options = TransportRequestOptions.options().withType(TransportRequestOptions.Type.STATE).withCompress(false); - // no need to put a timeout on the options here, because we want the response to eventually be received - // and not log an error if it arrives after the timeout - transportService.sendRequest(node, ACTION_NAME, - new BytesTransportRequest(bytes, node.version()), - options, // no need to compress, we already compressed the bytes - - new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { - - @Override - public void handleResponse(TransportResponse.Empty response) { - if (timedOutWaitingForNodes.get()) { - logger.debug("node {} responded for cluster state [{}] (took longer than [{}])", node, clusterState.version(), publishTimeout); - } - publishResponseHandler.onResponse(node); - } - - @Override - public void handleException(TransportException exp) { - logger.debug("failed to send cluster state to {}", exp, node); - publishResponseHandler.onFailure(node, exp); - } - }); - } catch (Throwable t) { - logger.debug("error sending cluster state to {}", t, node); - publishResponseHandler.onFailure(node, t); + sendClusterStateDiff(clusterState, diff, serializedDiffs, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler); } } @@ -171,7 +148,107 @@ public class PublishClusterStateAction extends AbstractComponent { } } + private void sendFullClusterState(ClusterState clusterState, @Nullable Map serializedStates, + DiscoveryNode node, AtomicBoolean timedOutWaitingForNodes, TimeValue publishTimeout, + BlockingClusterStatePublishResponseHandler publishResponseHandler) { + BytesReference bytes = null; + if (serializedStates != null) { + bytes = serializedStates.get(node.version()); + } + if (bytes == null) { + try { + bytes = serializeFullClusterState(clusterState, node.version()); + if (serializedStates != null) { + serializedStates.put(node.version(), bytes); + } + } catch (Throwable e) { + logger.warn("failed to serialize cluster_state before publishing it to node {}", e, node); + publishResponseHandler.onFailure(node, e); + return; + } + } + publishClusterStateToNode(clusterState, bytes, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler, false); + } + + private void sendClusterStateDiff(ClusterState clusterState, Diff diff, Map serializedDiffs, DiscoveryNode node, + AtomicBoolean timedOutWaitingForNodes, TimeValue publishTimeout, + BlockingClusterStatePublishResponseHandler publishResponseHandler) { + BytesReference bytes = serializedDiffs.get(node.version()); + if (bytes == null) { + try { + bytes = serializeDiffClusterState(diff, node.version()); + serializedDiffs.put(node.version(), bytes); + } catch (Throwable e) { + logger.warn("failed to serialize diff of cluster_state before publishing it to node {}", e, node); + publishResponseHandler.onFailure(node, e); + return; + } + } + publishClusterStateToNode(clusterState, bytes, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler, true); + } + + private void publishClusterStateToNode(final ClusterState clusterState, BytesReference bytes, + final DiscoveryNode node, final AtomicBoolean timedOutWaitingForNodes, + final TimeValue publishTimeout, + final BlockingClusterStatePublishResponseHandler publishResponseHandler, + final boolean sendDiffs) { + try { + TransportRequestOptions options = TransportRequestOptions.options().withType(TransportRequestOptions.Type.STATE).withCompress(false); + // no need to put a timeout on the options here, because we want the response to eventually be received + // and not log an error if it arrives after the timeout + transportService.sendRequest(node, ACTION_NAME, + new BytesTransportRequest(bytes, node.version()), + options, // no need to compress, we already compressed the bytes + + new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { + + @Override + public void handleResponse(TransportResponse.Empty response) { + if (timedOutWaitingForNodes.get()) { + logger.debug("node {} responded for cluster state [{}] (took longer than [{}])", node, clusterState.version(), publishTimeout); + } + publishResponseHandler.onResponse(node); + } + + @Override + public void handleException(TransportException exp) { + if (sendDiffs && exp.unwrapCause() instanceof IncompatibleClusterStateVersionException) { + logger.debug("resending full cluster state to node {} reason {}", node, exp.getDetailedMessage()); + sendFullClusterState(clusterState, null, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler); + } else { + logger.debug("failed to send cluster state to {}", exp, node); + publishResponseHandler.onFailure(node, exp); + } + } + }); + } catch (Throwable t) { + logger.warn("error sending cluster state to {}", t, node); + publishResponseHandler.onFailure(node, t); + } + } + + public static BytesReference serializeFullClusterState(ClusterState clusterState, Version nodeVersion) throws IOException { + BytesStreamOutput bStream = new BytesStreamOutput(); + StreamOutput stream = CompressorFactory.defaultCompressor().streamOutput(bStream); + stream.setVersion(nodeVersion); + stream.writeBoolean(true); + clusterState.writeTo(stream); + stream.close(); + return bStream.bytes(); + } + + public static BytesReference serializeDiffClusterState(Diff diff, Version nodeVersion) throws IOException { + BytesStreamOutput bStream = new BytesStreamOutput(); + StreamOutput stream = CompressorFactory.defaultCompressor().streamOutput(bStream); + stream.setVersion(nodeVersion); + stream.writeBoolean(false); + diff.writeTo(stream); + stream.close(); + return bStream.bytes(); + } + private class PublishClusterStateRequestHandler implements TransportRequestHandler { + private ClusterState lastSeenClusterState; @Override public void messageReceived(BytesTransportRequest request, final TransportChannel channel) throws Exception { @@ -183,11 +260,24 @@ public class PublishClusterStateAction extends AbstractComponent { in = request.bytes().streamInput(); } in.setVersion(request.version()); - ClusterState clusterState = ClusterState.Builder.readFrom(in, nodesProvider.nodes().localNode()); - clusterState.status(ClusterState.ClusterStateStatus.RECEIVED); - logger.debug("received cluster state version {}", clusterState.version()); + synchronized (this) { + // If true we received full cluster state - otherwise diffs + if (in.readBoolean()) { + lastSeenClusterState = ClusterState.Builder.readFrom(in, nodesProvider.nodes().localNode()); + logger.debug("received full cluster state version {} with size {}", lastSeenClusterState.version(), request.bytes().length()); + } else if (lastSeenClusterState != null) { + Diff diff = lastSeenClusterState.readDiffFrom(in); + lastSeenClusterState = diff.apply(lastSeenClusterState); + logger.debug("received diff cluster state version {} with uuid {}, diff size {}", lastSeenClusterState.version(), lastSeenClusterState.uuid(), request.bytes().length()); + } else { + logger.debug("received diff for but don't have any local cluster state - requesting full state"); + throw new IncompatibleClusterStateVersionException("have no local cluster state"); + } + lastSeenClusterState.status(ClusterState.ClusterStateStatus.RECEIVED); + } + try { - listener.onNewClusterState(clusterState, new NewClusterStateListener.NewStateProcessed() { + listener.onNewClusterState(lastSeenClusterState, new NewClusterStateListener.NewStateProcessed() { @Override public void onNewClusterStateProcessed() { try { @@ -207,7 +297,7 @@ public class PublishClusterStateAction extends AbstractComponent { } }); } catch (Exception e) { - logger.warn("unexpected error while processing cluster state version [{}]", e, clusterState.version()); + logger.warn("unexpected error while processing cluster state version [{}]", e, lastSeenClusterState.version()); try { channel.sendResponse(e); } catch (Throwable e1) { diff --git a/src/main/java/org/elasticsearch/gateway/Gateway.java b/src/main/java/org/elasticsearch/gateway/Gateway.java index cd15bccdc4a..139b5763489 100644 --- a/src/main/java/org/elasticsearch/gateway/Gateway.java +++ b/src/main/java/org/elasticsearch/gateway/Gateway.java @@ -31,7 +31,7 @@ import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.NodeEnvironment; -import org.elasticsearch.indices.IndicesService; + import java.nio.file.Path; diff --git a/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java b/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java index 43dec7edb51..5538ef6d043 100644 --- a/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java +++ b/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java @@ -198,7 +198,7 @@ public class LocalAllocateDangledIndices extends AbstractComponent { fromNode.writeTo(out); out.writeVInt(indices.length); for (IndexMetaData indexMetaData : indices) { - IndexMetaData.Builder.writeTo(indexMetaData, out); + indexMetaData.writeTo(out); } } } diff --git a/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java b/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java index 36372009f87..900a2e7ffc7 100644 --- a/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java +++ b/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java @@ -221,7 +221,7 @@ public class TransportNodesListGatewayMetaState extends TransportNodesOperationA out.writeBoolean(false); } else { out.writeBoolean(true); - MetaData.Builder.writeTo(metaData, out); + metaData.writeTo(out); } } } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/cluster/repositories/get/RestGetRepositoriesAction.java b/src/main/java/org/elasticsearch/rest/action/admin/cluster/repositories/get/RestGetRepositoriesAction.java index be4e1b4e3f3..85b46925b5f 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/cluster/repositories/get/RestGetRepositoriesAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/cluster/repositories/get/RestGetRepositoriesAction.java @@ -58,7 +58,7 @@ public class RestGetRepositoriesAction extends BaseRestHandler { public RestResponse buildResponse(GetRepositoriesResponse response, XContentBuilder builder) throws Exception { builder.startObject(); for (RepositoryMetaData repositoryMetaData : response.repositories()) { - RepositoriesMetaData.FACTORY.toXContent(repositoryMetaData, builder, request); + RepositoriesMetaData.toXContent(repositoryMetaData, builder, request); } builder.endObject(); diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/get/RestGetIndicesAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/get/RestGetIndicesAction.java index 7e4e56710b7..dd1dca34bbc 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/get/RestGetIndicesAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/get/RestGetIndicesAction.java @@ -146,7 +146,7 @@ public class RestGetIndicesAction extends BaseRestHandler { builder.startObject(Fields.WARMERS); if (warmers != null) { for (IndexWarmersMetaData.Entry warmer : warmers) { - IndexWarmersMetaData.FACTORY.toXContent(warmer, builder, params); + IndexWarmersMetaData.toXContent(warmer, builder, params); } } builder.endObject(); diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/warmer/get/RestGetWarmerAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/warmer/get/RestGetWarmerAction.java index 7023eecedd4..be83ccbe4b5 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/warmer/get/RestGetWarmerAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/warmer/get/RestGetWarmerAction.java @@ -72,7 +72,7 @@ public class RestGetWarmerAction extends BaseRestHandler { builder.startObject(entry.key, XContentBuilder.FieldCaseConversion.NONE); builder.startObject(IndexWarmersMetaData.TYPE, XContentBuilder.FieldCaseConversion.NONE); for (IndexWarmersMetaData.Entry warmerEntry : entry.value) { - IndexWarmersMetaData.FACTORY.toXContent(warmerEntry, builder, request); + IndexWarmersMetaData.toXContent(warmerEntry, builder, request); } builder.endObject(); builder.endObject(); diff --git a/src/main/java/org/elasticsearch/search/warmer/IndexWarmersMetaData.java b/src/main/java/org/elasticsearch/search/warmer/IndexWarmersMetaData.java index de56f823eac..ef1ef44ffb9 100644 --- a/src/main/java/org/elasticsearch/search/warmer/IndexWarmersMetaData.java +++ b/src/main/java/org/elasticsearch/search/warmer/IndexWarmersMetaData.java @@ -22,7 +22,9 @@ package org.elasticsearch.search.warmer; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.elasticsearch.Version; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; @@ -33,16 +35,33 @@ import org.elasticsearch.common.xcontent.*; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; /** */ -public class IndexWarmersMetaData implements IndexMetaData.Custom { +public class IndexWarmersMetaData extends AbstractDiffable implements IndexMetaData.Custom { public static final String TYPE = "warmers"; - public static final Factory FACTORY = new Factory(); + public static final IndexWarmersMetaData PROTO = new IndexWarmersMetaData(); + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + IndexWarmersMetaData that = (IndexWarmersMetaData) o; + + return entries.equals(that.entries); + + } + + @Override + public int hashCode() { + return entries.hashCode(); + } public static class Entry { private final String name; @@ -74,6 +93,29 @@ public class IndexWarmersMetaData implements IndexMetaData.Custom { public Boolean queryCache() { return this.queryCache; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Entry entry = (Entry) o; + + if (!name.equals(entry.name)) return false; + if (!Arrays.equals(types, entry.types)) return false; + if (!source.equals(entry.source)) return false; + return !(queryCache != null ? !queryCache.equals(entry.queryCache) : entry.queryCache != null); + + } + + @Override + public int hashCode() { + int result = name.hashCode(); + result = 31 * result + Arrays.hashCode(types); + result = 31 * result + source.hashCode(); + result = 31 * result + (queryCache != null ? queryCache.hashCode() : 0); + return result; + } } private final ImmutableList entries; @@ -92,149 +134,143 @@ public class IndexWarmersMetaData implements IndexMetaData.Custom { return TYPE; } - public static class Factory implements IndexMetaData.Custom.Factory { - - @Override - public String type() { - return TYPE; + @Override + public IndexWarmersMetaData readFrom(StreamInput in) throws IOException { + Entry[] entries = new Entry[in.readVInt()]; + for (int i = 0; i < entries.length; i++) { + String name = in.readString(); + String[] types = in.readStringArray(); + BytesReference source = null; + if (in.readBoolean()) { + source = in.readBytesReference(); + } + Boolean queryCache; + queryCache = in.readOptionalBoolean(); + entries[i] = new Entry(name, types, queryCache, source); } + return new IndexWarmersMetaData(entries); + } - @Override - public IndexWarmersMetaData readFrom(StreamInput in) throws IOException { - Entry[] entries = new Entry[in.readVInt()]; - for (int i = 0; i < entries.length; i++) { - String name = in.readString(); - String[] types = in.readStringArray(); - BytesReference source = null; - if (in.readBoolean()) { - source = in.readBytesReference(); - } - Boolean queryCache = null; - queryCache = in.readOptionalBoolean(); - entries[i] = new Entry(name, types, queryCache, source); - } - return new IndexWarmersMetaData(entries); - } - - @Override - public void writeTo(IndexWarmersMetaData warmers, StreamOutput out) throws IOException { - out.writeVInt(warmers.entries().size()); - for (Entry entry : warmers.entries()) { - out.writeString(entry.name()); - out.writeStringArray(entry.types()); - if (entry.source() == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - out.writeBytesReference(entry.source()); - } - out.writeOptionalBoolean(entry.queryCache()); - } - } - - @Override - public IndexWarmersMetaData fromMap(Map map) throws IOException { - // if it starts with the type, remove it - if (map.size() == 1 && map.containsKey(TYPE)) { - map = (Map) map.values().iterator().next(); - } - XContentBuilder builder = XContentFactory.smileBuilder().map(map); - try (XContentParser parser = XContentFactory.xContent(XContentType.SMILE).createParser(builder.bytes())) { - // move to START_OBJECT - parser.nextToken(); - return fromXContent(parser); - } - } - - @Override - public IndexWarmersMetaData fromXContent(XContentParser parser) throws IOException { - // we get here after we are at warmers token - String currentFieldName = null; - XContentParser.Token token; - List entries = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (token == XContentParser.Token.START_OBJECT) { - String name = currentFieldName; - List types = new ArrayList<>(2); - BytesReference source = null; - Boolean queryCache = null; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (token == XContentParser.Token.START_ARRAY) { - if ("types".equals(currentFieldName)) { - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - types.add(parser.text()); - } - } - } else if (token == XContentParser.Token.START_OBJECT) { - if ("source".equals(currentFieldName)) { - XContentBuilder builder = XContentFactory.jsonBuilder().map(parser.mapOrdered()); - source = builder.bytes(); - } - } else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) { - if ("source".equals(currentFieldName)) { - source = new BytesArray(parser.binaryValue()); - } - } else if (token.isValue()) { - if ("queryCache".equals(currentFieldName) || "query_cache".equals(currentFieldName)) { - queryCache = parser.booleanValue(); - } - } - } - entries.add(new Entry(name, types.size() == 0 ? Strings.EMPTY_ARRAY : types.toArray(new String[types.size()]), queryCache, source)); - } - } - return new IndexWarmersMetaData(entries.toArray(new Entry[entries.size()])); - } - - @Override - public void toXContent(IndexWarmersMetaData warmers, XContentBuilder builder, ToXContent.Params params) throws IOException { - //No need, IndexMetaData already writes it - //builder.startObject(TYPE, XContentBuilder.FieldCaseConversion.NONE); - for (Entry entry : warmers.entries()) { - toXContent(entry, builder, params); - } - //No need, IndexMetaData already writes it - //builder.endObject(); - } - - public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { - boolean binary = params.paramAsBoolean("binary", false); - builder.startObject(entry.name(), XContentBuilder.FieldCaseConversion.NONE); - builder.field("types", entry.types()); - if (entry.queryCache() != null) { - builder.field("queryCache", entry.queryCache()); - } - builder.field("source"); - if (binary) { - builder.value(entry.source()); + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(entries().size()); + for (Entry entry : entries()) { + out.writeString(entry.name()); + out.writeStringArray(entry.types()); + if (entry.source() == null) { + out.writeBoolean(false); } else { - Map mapping = XContentFactory.xContent(entry.source()).createParser(entry.source()).mapOrderedAndClose(); - builder.map(mapping); + out.writeBoolean(true); + out.writeBytesReference(entry.source()); } - builder.endObject(); - } - - @Override - public IndexWarmersMetaData merge(IndexWarmersMetaData first, IndexWarmersMetaData second) { - List entries = Lists.newArrayList(); - entries.addAll(first.entries()); - for (Entry secondEntry : second.entries()) { - boolean found = false; - for (Entry firstEntry : first.entries()) { - if (firstEntry.name().equals(secondEntry.name())) { - found = true; - break; - } - } - if (!found) { - entries.add(secondEntry); - } - } - return new IndexWarmersMetaData(entries.toArray(new Entry[entries.size()])); + out.writeOptionalBoolean(entry.queryCache()); } } + + @Override + public IndexWarmersMetaData fromMap(Map map) throws IOException { + // if it starts with the type, remove it + if (map.size() == 1 && map.containsKey(TYPE)) { + map = (Map) map.values().iterator().next(); + } + XContentBuilder builder = XContentFactory.smileBuilder().map(map); + try (XContentParser parser = XContentFactory.xContent(XContentType.SMILE).createParser(builder.bytes())) { + // move to START_OBJECT + parser.nextToken(); + return fromXContent(parser); + } + } + + @Override + public IndexWarmersMetaData fromXContent(XContentParser parser) throws IOException { + // we get here after we are at warmers token + String currentFieldName = null; + XContentParser.Token token; + List entries = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.START_OBJECT) { + String name = currentFieldName; + List types = new ArrayList<>(2); + BytesReference source = null; + Boolean queryCache = null; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.START_ARRAY) { + if ("types".equals(currentFieldName)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + types.add(parser.text()); + } + } + } else if (token == XContentParser.Token.START_OBJECT) { + if ("source".equals(currentFieldName)) { + XContentBuilder builder = XContentFactory.jsonBuilder().map(parser.mapOrdered()); + source = builder.bytes(); + } + } else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) { + if ("source".equals(currentFieldName)) { + source = new BytesArray(parser.binaryValue()); + } + } else if (token.isValue()) { + if ("queryCache".equals(currentFieldName) || "query_cache".equals(currentFieldName)) { + queryCache = parser.booleanValue(); + } + } + } + entries.add(new Entry(name, types.size() == 0 ? Strings.EMPTY_ARRAY : types.toArray(new String[types.size()]), queryCache, source)); + } + } + return new IndexWarmersMetaData(entries.toArray(new Entry[entries.size()])); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + //No need, IndexMetaData already writes it + //builder.startObject(TYPE, XContentBuilder.FieldCaseConversion.NONE); + for (Entry entry : entries()) { + toXContent(entry, builder, params); + } + //No need, IndexMetaData already writes it + //builder.endObject(); + return builder; + } + + public static void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { + boolean binary = params.paramAsBoolean("binary", false); + builder.startObject(entry.name(), XContentBuilder.FieldCaseConversion.NONE); + builder.field("types", entry.types()); + if (entry.queryCache() != null) { + builder.field("queryCache", entry.queryCache()); + } + builder.field("source"); + if (binary) { + builder.value(entry.source()); + } else { + Map mapping = XContentFactory.xContent(entry.source()).createParser(entry.source()).mapOrderedAndClose(); + builder.map(mapping); + } + builder.endObject(); + } + + @Override + public IndexMetaData.Custom mergeWith(IndexMetaData.Custom other) { + IndexWarmersMetaData second = (IndexWarmersMetaData) other; + List entries = Lists.newArrayList(); + entries.addAll(entries()); + for (Entry secondEntry : second.entries()) { + boolean found = false; + for (Entry firstEntry : entries()) { + if (firstEntry.name().equals(secondEntry.name())) { + found = true; + break; + } + } + if (!found) { + entries.add(secondEntry); + } + } + return new IndexWarmersMetaData(entries.toArray(new Entry[entries.size()])); + } } diff --git a/src/test/java/org/elasticsearch/cluster/ClusterStateDiffPublishingTests.java b/src/test/java/org/elasticsearch/cluster/ClusterStateDiffPublishingTests.java new file mode 100644 index 00000000000..33008fd63d2 --- /dev/null +++ b/src/test/java/org/elasticsearch/cluster/ClusterStateDiffPublishingTests.java @@ -0,0 +1,625 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.discovery.Discovery; +import org.elasticsearch.discovery.DiscoverySettings; +import org.elasticsearch.discovery.zen.DiscoveryNodesProvider; +import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction; +import org.elasticsearch.node.service.NodeService; +import org.elasticsearch.node.settings.NodeSettingsService; +import org.elasticsearch.test.ElasticsearchTestCase; +import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportConnectionListener; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.transport.local.LocalTransport; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.google.common.collect.Maps.newHashMap; +import static org.hamcrest.Matchers.*; + +public class ClusterStateDiffPublishingTests extends ElasticsearchTestCase { + + protected ThreadPool threadPool; + protected Map nodes = newHashMap(); + + public static class MockNode { + public final DiscoveryNode discoveryNode; + public final MockTransportService service; + public final PublishClusterStateAction action; + public final MockDiscoveryNodesProvider nodesProvider; + + public MockNode(DiscoveryNode discoveryNode, MockTransportService service, PublishClusterStateAction action, MockDiscoveryNodesProvider nodesProvider) { + this.discoveryNode = discoveryNode; + this.service = service; + this.action = action; + this.nodesProvider = nodesProvider; + } + + public void connectTo(DiscoveryNode node) { + service.connectToNode(node); + nodesProvider.addNode(node); + } + } + + public MockNode createMockNode(final String name, Settings settings, Version version) throws Exception { + return createMockNode(name, settings, version, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + logger.debug("Node [{}] onNewClusterState version [{}], uuid [{}]", name, clusterState.version(), clusterState.uuid()); + newStateProcessed.onNewClusterStateProcessed(); + } + }); + } + + public MockNode createMockNode(String name, Settings settings, Version version, PublishClusterStateAction.NewClusterStateListener listener) throws Exception { + MockTransportService service = buildTransportService( + ImmutableSettings.builder().put(settings).put("name", name, TransportService.SETTING_TRACE_LOG_INCLUDE, "", TransportService.SETTING_TRACE_LOG_EXCLUDE, "NOTHING").build(), + version + ); + DiscoveryNode discoveryNode = new DiscoveryNode(name, name, service.boundAddress().publishAddress(), ImmutableMap.of(), version); + MockDiscoveryNodesProvider nodesProvider = new MockDiscoveryNodesProvider(discoveryNode); + PublishClusterStateAction action = buildPublishClusterStateAction(settings, service, nodesProvider, listener); + MockNode node = new MockNode(discoveryNode, service, action, nodesProvider); + nodesProvider.addNode(discoveryNode); + final CountDownLatch latch = new CountDownLatch(nodes.size() * 2 + 1); + TransportConnectionListener waitForConnection = new TransportConnectionListener() { + @Override + public void onNodeConnected(DiscoveryNode node) { + latch.countDown(); + } + + @Override + public void onNodeDisconnected(DiscoveryNode node) { + fail("disconnect should not be called " + node); + } + }; + node.service.addConnectionListener(waitForConnection); + for (MockNode curNode : nodes.values()) { + curNode.service.addConnectionListener(waitForConnection); + curNode.connectTo(node.discoveryNode); + node.connectTo(curNode.discoveryNode); + } + node.connectTo(node.discoveryNode); + assertThat("failed to wait for all nodes to connect", latch.await(5, TimeUnit.SECONDS), equalTo(true)); + for (MockNode curNode : nodes.values()) { + curNode.service.removeConnectionListener(waitForConnection); + } + node.service.removeConnectionListener(waitForConnection); + if (nodes.put(name, node) != null) { + fail("Node with the name " + name + " already exist"); + } + return node; + } + + public MockTransportService service(String name) { + MockNode node = nodes.get(name); + if (node != null) { + return node.service; + } + return null; + } + + public PublishClusterStateAction action(String name) { + MockNode node = nodes.get(name); + if (node != null) { + return node.action; + } + return null; + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + threadPool = new ThreadPool(getClass().getName()); + } + + @Override + @After + public void tearDown() throws Exception { + super.tearDown(); + for (MockNode curNode : nodes.values()) { + curNode.action.close(); + curNode.service.close(); + } + terminate(threadPool); + } + + protected MockTransportService buildTransportService(Settings settings, Version version) { + MockTransportService transportService = new MockTransportService(settings, new LocalTransport(settings, threadPool, version), threadPool); + transportService.start(); + return transportService; + } + + protected PublishClusterStateAction buildPublishClusterStateAction(Settings settings, MockTransportService transportService, MockDiscoveryNodesProvider nodesProvider, + PublishClusterStateAction.NewClusterStateListener listener) { + DiscoverySettings discoverySettings = new DiscoverySettings(settings, new NodeSettingsService(settings)); + return new PublishClusterStateAction(settings, transportService, nodesProvider, listener, discoverySettings); + } + + + static class MockDiscoveryNodesProvider implements DiscoveryNodesProvider { + + private DiscoveryNodes discoveryNodes = DiscoveryNodes.EMPTY_NODES; + + public MockDiscoveryNodesProvider(DiscoveryNode localNode) { + discoveryNodes = DiscoveryNodes.builder().put(localNode).localNodeId(localNode.id()).build(); + } + + public void addNode(DiscoveryNode node) { + discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(node).build(); + } + + @Override + public DiscoveryNodes nodes() { + return discoveryNodes; + } + + @Override + public NodeService nodeService() { + assert false; + throw new UnsupportedOperationException("Shouldn't be here"); + } + } + + + @Test + @TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG") + public void testSimpleClusterStatePublishing() throws Exception { + MockNewClusterStateListener mockListenerA = new MockNewClusterStateListener(); + MockNode nodeA = createMockNode("nodeA", ImmutableSettings.EMPTY, Version.CURRENT, mockListenerA); + + MockNewClusterStateListener mockListenerB = new MockNewClusterStateListener(); + MockNode nodeB = createMockNode("nodeB", ImmutableSettings.EMPTY, Version.CURRENT, mockListenerB); + + // Initial cluster state + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build(); + ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + + // cluster state update - add nodeB + discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeB.discoveryNode).build(); + ClusterState previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - add block + previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + assertThat(clusterState.blocks().global().size(), equalTo(1)); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - remove block + previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + assertThat(clusterState.blocks().global().size(), equalTo(0)); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // Adding new node - this node should get full cluster state while nodeB should still be getting diffs + + MockNewClusterStateListener mockListenerC = new MockNewClusterStateListener(); + MockNode nodeC = createMockNode("nodeC", ImmutableSettings.EMPTY, Version.CURRENT, mockListenerC); + + // cluster state update 3 - register node C + previousClusterState = clusterState; + discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeC.discoveryNode).build(); + clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + assertThat(clusterState.blocks().global().size(), equalTo(0)); + } + }); + mockListenerC.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + // First state + assertFalse(clusterState.wasReadFromDiff()); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update 4 - update settings + previousClusterState = clusterState; + MetaData metaData = MetaData.builder(clusterState.metaData()).transientSettings(ImmutableSettings.settingsBuilder().put("foo", "bar").build()).build(); + clusterState = ClusterState.builder(clusterState).metaData(metaData).incrementVersion().build(); + NewClusterStateExpectation expectation = new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + assertThat(clusterState.blocks().global().size(), equalTo(0)); + } + }; + mockListenerB.add(expectation); + mockListenerC.add(expectation); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - skipping one version change - should request full cluster state + previousClusterState = ClusterState.builder(clusterState).incrementVersion().build(); + clusterState = ClusterState.builder(clusterState).incrementVersion().build(); + expectation = new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }; + mockListenerB.add(expectation); + mockListenerC.add(expectation); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - skipping one version change - should request full cluster state + previousClusterState = ClusterState.builder(clusterState).incrementVersion().build(); + clusterState = ClusterState.builder(clusterState).incrementVersion().build(); + expectation = new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }; + mockListenerB.add(expectation); + mockListenerC.add(expectation); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // node B becomes the master and sends a version of the cluster state that goes back + discoveryNodes = DiscoveryNodes.builder(discoveryNodes) + .put(nodeA.discoveryNode) + .put(nodeB.discoveryNode) + .put(nodeC.discoveryNode) + .build(); + previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); + expectation = new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }; + mockListenerA.add(expectation); + mockListenerC.add(expectation); + publishStateDiffAndWait(nodeB.action, clusterState, previousClusterState); + } + + @Test + @TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG") + public void testUnexpectedDiffPublishing() throws Exception { + + MockNode nodeA = createMockNode("nodeA", ImmutableSettings.EMPTY, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + fail("Shouldn't send cluster state to myself"); + } + }); + + MockNewClusterStateListener mockListenerB = new MockNewClusterStateListener(); + MockNode nodeB = createMockNode("nodeB", ImmutableSettings.EMPTY, Version.CURRENT, mockListenerB); + + // Initial cluster state with both states - the second node still shouldn't get diff even though it's present in the previous cluster state + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).put(nodeB.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build(); + ClusterState previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - add block + previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + } + + @Test + @TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG") + public void testDisablingDiffPublishing() throws Exception { + Settings noDiffPublishingSettings = ImmutableSettings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE, false).build(); + + MockNode nodeA = createMockNode("nodeA", noDiffPublishingSettings, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + fail("Shouldn't send cluster state to myself"); + } + }); + + MockNode nodeB = createMockNode("nodeB", noDiffPublishingSettings, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + logger.debug("Got cluster state update, version [{}], guid [{}], from diff [{}]", clusterState.version(), clusterState.uuid(), clusterState.wasReadFromDiff()); + assertFalse(clusterState.wasReadFromDiff()); + newStateProcessed.onNewClusterStateProcessed(); + } + }); + + // Initial cluster state + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build(); + ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + + // cluster state update - add nodeB + discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeB.discoveryNode).build(); + ClusterState previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - add block + previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + } + + + @Test + @TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG") + public void testSimultaneousClusterStatePublishing() throws Exception { + int numberOfNodes = randomIntBetween(2, 10); + int numberOfIterations = randomIntBetween(50, 200); + Settings settings = ImmutableSettings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT, "100ms").put(DiscoverySettings.PUBLISH_DIFF_ENABLE, true).build(); + MockNode[] nodes = new MockNode[numberOfNodes]; + DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder(); + for (int i = 0; i < nodes.length; i++) { + final String name = "node" + i; + nodes[i] = createMockNode(name, settings, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public synchronized void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + assertProperMetaDataForVersion(clusterState.metaData(), clusterState.version()); + if (randomInt(10) < 2) { + // Cause timeouts from time to time + try { + Thread.sleep(randomInt(110)); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + } + newStateProcessed.onNewClusterStateProcessed(); + } + }); + discoveryNodesBuilder.put(nodes[i].discoveryNode); + } + + AssertingAckListener[] listeners = new AssertingAckListener[numberOfIterations]; + DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build(); + MetaData metaData = MetaData.EMPTY_META_DATA; + ClusterState clusterState = ClusterState.builder(new ClusterName("test")).metaData(metaData).build(); + ClusterState previousState; + for (int i = 0; i < numberOfIterations; i++) { + previousState = clusterState; + metaData = buildMetaDataForVersion(metaData, i + 1); + clusterState = ClusterState.builder(clusterState).incrementVersion().metaData(metaData).nodes(discoveryNodes).build(); + listeners[i] = publishStateDiff(nodes[0].action, clusterState, previousState); + } + + for (int i = 0; i < numberOfIterations; i++) { + listeners[i].await(1, TimeUnit.SECONDS); + } + } + + @Test + @TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG") + public void testSerializationFailureDuringDiffPublishing() throws Exception { + + MockNode nodeA = createMockNode("nodeA", ImmutableSettings.EMPTY, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + fail("Shouldn't send cluster state to myself"); + } + }); + + MockNewClusterStateListener mockListenerB = new MockNewClusterStateListener(); + MockNode nodeB = createMockNode("nodeB", ImmutableSettings.EMPTY, Version.CURRENT, mockListenerB); + + // Initial cluster state with both states - the second node still shouldn't get diff even though it's present in the previous cluster state + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).put(nodeB.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build(); + ClusterState previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - add block + previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + } + }); + + ClusterState unserializableClusterState = new ClusterState(clusterState.version(), clusterState.uuid(), clusterState) { + @Override + public Diff diff(ClusterState previousState) { + return new Diff() { + @Override + public ClusterState apply(ClusterState part) { + fail("this diff shouldn't be applied"); + return part; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + throw new IOException("Simulated failure of diff serialization"); + } + }; + } + }; + List> errors = publishStateDiff(nodeA.action, unserializableClusterState, previousClusterState).awaitErrors(1, TimeUnit.SECONDS); + assertThat(errors.size(), equalTo(1)); + assertThat(errors.get(0).v2().getMessage(), containsString("Simulated failure of diff serialization")); + } + + private MetaData buildMetaDataForVersion(MetaData metaData, long version) { + ImmutableOpenMap.Builder indices = ImmutableOpenMap.builder(metaData.indices()); + indices.put("test" + version, IndexMetaData.builder("test" + version).settings(ImmutableSettings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards((int) version).numberOfReplicas(0).build()); + return MetaData.builder(metaData) + .transientSettings(ImmutableSettings.builder().put("test", version).build()) + .indices(indices.build()) + .build(); + } + + private void assertProperMetaDataForVersion(MetaData metaData, long version) { + for (long i = 1; i <= version; i++) { + assertThat(metaData.index("test" + i), notNullValue()); + assertThat(metaData.index("test" + i).numberOfShards(), equalTo((int) i)); + } + assertThat(metaData.index("test" + (version + 1)), nullValue()); + assertThat(metaData.transientSettings().get("test"), equalTo(Long.toString(version))); + } + + public void publishStateDiffAndWait(PublishClusterStateAction action, ClusterState state, ClusterState previousState) throws InterruptedException { + publishStateDiff(action, state, previousState).await(1, TimeUnit.SECONDS); + } + + public AssertingAckListener publishStateDiff(PublishClusterStateAction action, ClusterState state, ClusterState previousState) throws InterruptedException { + AssertingAckListener assertingAckListener = new AssertingAckListener(state.nodes().getSize() - 1); + ClusterChangedEvent changedEvent = new ClusterChangedEvent("test update", state, previousState); + action.publish(changedEvent, assertingAckListener); + return assertingAckListener; + } + + public static class AssertingAckListener implements Discovery.AckListener { + private final List> errors = new CopyOnWriteArrayList<>(); + private final AtomicBoolean timeoutOccured = new AtomicBoolean(); + private final CountDownLatch countDown; + + public AssertingAckListener(int nodeCount) { + countDown = new CountDownLatch(nodeCount); + } + + @Override + public void onNodeAck(DiscoveryNode node, @Nullable Throwable t) { + if (t != null) { + errors.add(new Tuple<>(node, t)); + } + countDown.countDown(); + } + + @Override + public void onTimeout() { + timeoutOccured.set(true); + // Fast forward the counter - no reason to wait here + long currentCount = countDown.getCount(); + for (long i = 0; i < currentCount; i++) { + countDown.countDown(); + } + } + + public void await(long timeout, TimeUnit unit) throws InterruptedException { + assertThat(awaitErrors(timeout, unit), emptyIterable()); + } + + public List> awaitErrors(long timeout, TimeUnit unit) throws InterruptedException { + countDown.await(timeout, unit); + assertFalse(timeoutOccured.get()); + return errors; + } + + } + + public interface NewClusterStateExpectation { + void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed); + } + + public static class MockNewClusterStateListener implements PublishClusterStateAction.NewClusterStateListener { + CopyOnWriteArrayList expectations = new CopyOnWriteArrayList(); + + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + final NewClusterStateExpectation expectation; + try { + expectation = expectations.remove(0); + } catch (ArrayIndexOutOfBoundsException ex) { + fail("Unexpected cluster state update " + clusterState.prettyPrint()); + return; + } + expectation.check(clusterState, newStateProcessed); + newStateProcessed.onNewClusterStateProcessed(); + } + + public void add(NewClusterStateExpectation expectation) { + expectations.add(expectation); + } + } + + public static class DelegatingClusterState extends ClusterState { + + public DelegatingClusterState(ClusterState clusterState) { + super(clusterState.version(), clusterState.uuid(), clusterState); + } + + + } + +} diff --git a/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java b/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java new file mode 100644 index 00000000000..84df1eaf209 --- /dev/null +++ b/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java @@ -0,0 +1,534 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import com.carrotsearch.hppc.cursors.ObjectCursor; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.block.ClusterBlock; +import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.metadata.*; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.*; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.io.stream.BytesStreamInput; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.LocalTransportAddress; +import org.elasticsearch.discovery.DiscoverySettings; +import org.elasticsearch.gateway.GatewayService; +import org.elasticsearch.index.query.FilterBuilders; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.warmer.IndexWarmersMetaData; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.junit.Test; + +import java.util.List; + +import static org.elasticsearch.cluster.metadata.AliasMetaData.newAliasMetaDataBuilder; +import static org.elasticsearch.common.xcontent.XContentTestUtils.convertToMap; +import static org.elasticsearch.common.xcontent.XContentTestUtils.mapsEqualIgnoringArrayOrder; +import static org.elasticsearch.test.VersionUtils.randomVersion; +import static org.hamcrest.Matchers.equalTo; + + +@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.SUITE, numDataNodes = 0, numClientNodes = 0) +public class ClusterStateDiffTests extends ElasticsearchIntegrationTest { + + @Test + public void testClusterStateDiffSerialization() throws Exception { + DiscoveryNode masterNode = new DiscoveryNode("master", new LocalTransportAddress("master"), Version.CURRENT); + DiscoveryNode otherNode = new DiscoveryNode("other", new LocalTransportAddress("other"), Version.CURRENT); + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(masterNode).put(otherNode).localNodeId(masterNode.id()).build(); + ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + ClusterState clusterStateFromDiffs = ClusterState.Builder.fromBytes(ClusterState.Builder.toBytes(clusterState), otherNode); + + int iterationCount = randomIntBetween(10, 300); + for (int iteration = 0; iteration < iterationCount; iteration++) { + ClusterState previousClusterState = clusterState; + ClusterState previousClusterStateFromDiffs = clusterStateFromDiffs; + int changesCount = randomIntBetween(1, 4); + ClusterState.Builder builder = null; + for (int i = 0; i < changesCount; i++) { + if (i > 0) { + clusterState = builder.build(); + } + switch (randomInt(4)) { + case 0: + builder = randomNodes(clusterState); + break; + case 1: + builder = randomRoutingTable(clusterState); + break; + case 2: + builder = randomBlocks(clusterState); + break; + case 3: + case 4: + builder = randomMetaDataChanges(clusterState); + break; + default: + throw new IllegalArgumentException("Shouldn't be here"); + } + } + clusterState = builder.incrementVersion().build(); + + if (randomIntBetween(0, 10) < 1) { + // Update cluster state via full serialization from time to time + clusterStateFromDiffs = ClusterState.Builder.fromBytes(ClusterState.Builder.toBytes(clusterState), previousClusterStateFromDiffs.nodes().localNode()); + } else { + // Update cluster states using diffs + Diff diffBeforeSerialization = clusterState.diff(previousClusterState); + BytesStreamOutput os = new BytesStreamOutput(); + diffBeforeSerialization.writeTo(os); + byte[] diffBytes = os.bytes().toBytes(); + Diff diff; + try (BytesStreamInput input = new BytesStreamInput(diffBytes)) { + diff = previousClusterStateFromDiffs.readDiffFrom(input); + clusterStateFromDiffs = diff.apply(previousClusterStateFromDiffs); + } + } + + + try { + // Check non-diffable elements + assertThat(clusterStateFromDiffs.version(), equalTo(clusterState.version())); + assertThat(clusterStateFromDiffs.uuid(), equalTo(clusterState.uuid())); + + // Check nodes + assertThat(clusterStateFromDiffs.nodes().nodes(), equalTo(clusterState.nodes().nodes())); + assertThat(clusterStateFromDiffs.nodes().localNodeId(), equalTo(previousClusterStateFromDiffs.nodes().localNodeId())); + assertThat(clusterStateFromDiffs.nodes().nodes(), equalTo(clusterState.nodes().nodes())); + for (ObjectCursor node : clusterStateFromDiffs.nodes().nodes().keys()) { + DiscoveryNode node1 = clusterState.nodes().get(node.value); + DiscoveryNode node2 = clusterStateFromDiffs.nodes().get(node.value); + assertThat(node1.version(), equalTo(node2.version())); + assertThat(node1.address(), equalTo(node2.address())); + assertThat(node1.attributes(), equalTo(node2.attributes())); + } + + // Check routing table + assertThat(clusterStateFromDiffs.routingTable().version(), equalTo(clusterState.routingTable().version())); + assertThat(clusterStateFromDiffs.routingTable().indicesRouting(), equalTo(clusterState.routingTable().indicesRouting())); + + // Check cluster blocks + assertThat(clusterStateFromDiffs.blocks().global(), equalTo(clusterStateFromDiffs.blocks().global())); + assertThat(clusterStateFromDiffs.blocks().indices(), equalTo(clusterStateFromDiffs.blocks().indices())); + assertThat(clusterStateFromDiffs.blocks().disableStatePersistence(), equalTo(clusterStateFromDiffs.blocks().disableStatePersistence())); + + // Check metadata + assertThat(clusterStateFromDiffs.metaData().version(), equalTo(clusterState.metaData().version())); + assertThat(clusterStateFromDiffs.metaData().uuid(), equalTo(clusterState.metaData().uuid())); + assertThat(clusterStateFromDiffs.metaData().transientSettings(), equalTo(clusterState.metaData().transientSettings())); + assertThat(clusterStateFromDiffs.metaData().persistentSettings(), equalTo(clusterState.metaData().persistentSettings())); + assertThat(clusterStateFromDiffs.metaData().indices(), equalTo(clusterState.metaData().indices())); + assertThat(clusterStateFromDiffs.metaData().templates(), equalTo(clusterState.metaData().templates())); + assertThat(clusterStateFromDiffs.metaData().customs(), equalTo(clusterState.metaData().customs())); + assertThat(clusterStateFromDiffs.metaData().aliases(), equalTo(clusterState.metaData().aliases())); + + // JSON Serialization test - make sure that both states produce similar JSON + assertThat(mapsEqualIgnoringArrayOrder(convertToMap(clusterStateFromDiffs), convertToMap(clusterState)), equalTo(true)); + + // Smoke test - we cannot compare bytes to bytes because some elements might get serialized in different order + // however, serialized size should remain the same + assertThat(ClusterState.Builder.toBytes(clusterStateFromDiffs).length, equalTo(ClusterState.Builder.toBytes(clusterState).length)); + } catch (AssertionError error) { + logger.error("Cluster state:\n{}\nCluster state from diffs:\n{}", clusterState.toString(), clusterStateFromDiffs.toString()); + throw error; + } + } + + logger.info("Final cluster state:[{}]", clusterState.toString()); + + } + + private ClusterState.Builder randomNodes(ClusterState clusterState) { + DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterState.nodes()); + List nodeIds = randomSubsetOf(randomInt(clusterState.nodes().nodes().size() - 1), clusterState.nodes().nodes().keys().toArray(String.class)); + for (String nodeId : nodeIds) { + if (nodeId.startsWith("node-")) { + if (randomBoolean()) { + nodes.remove(nodeId); + } else { + nodes.put(new DiscoveryNode(nodeId, new LocalTransportAddress(randomAsciiOfLength(10)), randomVersion(random()))); + } + } + } + int additionalNodeCount = randomIntBetween(1, 20); + for (int i = 0; i < additionalNodeCount; i++) { + nodes.put(new DiscoveryNode("node-" + randomAsciiOfLength(10), new LocalTransportAddress(randomAsciiOfLength(10)), randomVersion(random()))); + } + return ClusterState.builder(clusterState).nodes(nodes); + } + + private ClusterState.Builder randomRoutingTable(ClusterState clusterState) { + RoutingTable.Builder builder = RoutingTable.builder(clusterState.routingTable()); + int numberOfIndices = clusterState.routingTable().indicesRouting().size(); + if (numberOfIndices > 0) { + List randomIndices = randomSubsetOf(randomInt(numberOfIndices - 1), clusterState.routingTable().indicesRouting().keySet().toArray(new String[numberOfIndices])); + for (String index : randomIndices) { + if (randomBoolean()) { + builder.remove(index); + } else { + builder.add(randomIndexRoutingTable(index, clusterState.nodes().nodes().keys().toArray(String.class))); + } + } + } + int additionalIndexCount = randomIntBetween(1, 20); + for (int i = 0; i < additionalIndexCount; i++) { + builder.add(randomIndexRoutingTable("index-" + randomInt(), clusterState.nodes().nodes().keys().toArray(String.class))); + } + return ClusterState.builder(clusterState).routingTable(builder.build()); + } + + private IndexRoutingTable randomIndexRoutingTable(String index, String[] nodeIds) { + IndexRoutingTable.Builder builder = IndexRoutingTable.builder(index); + int shardCount = randomInt(10); + + for (int i = 0; i < shardCount; i++) { + IndexShardRoutingTable.Builder indexShard = new IndexShardRoutingTable.Builder(new ShardId(index, i), randomBoolean()); + int replicaCount = randomIntBetween(1, 10); + for (int j = 0; j < replicaCount; j++) { + indexShard.addShard( + new MutableShardRouting(index, i, randomFrom(nodeIds), j == 0, ShardRoutingState.fromValue((byte) randomIntBetween(1, 4)), 1)); + } + builder.addIndexShard(indexShard.build()); + } + return builder.build(); + } + + private ClusterState.Builder randomBlocks(ClusterState clusterState) { + ClusterBlocks.Builder builder = ClusterBlocks.builder().blocks(clusterState.blocks()); + int globalBlocksCount = clusterState.blocks().global().size(); + if (globalBlocksCount > 0) { + List blocks = randomSubsetOf(randomInt(globalBlocksCount - 1), clusterState.blocks().global().toArray(new ClusterBlock[globalBlocksCount])); + for (ClusterBlock block : blocks) { + builder.removeGlobalBlock(block); + } + } + int additionalGlobalBlocksCount = randomIntBetween(1, 3); + for (int i = 0; i < additionalGlobalBlocksCount; i++) { + builder.addGlobalBlock(randomGlobalBlock()); + } + return ClusterState.builder(clusterState).blocks(builder); + } + + private ClusterBlock randomGlobalBlock() { + switch (randomInt(2)) { + case 0: + return DiscoverySettings.NO_MASTER_BLOCK_ALL; + case 1: + return DiscoverySettings.NO_MASTER_BLOCK_WRITES; + default: + return GatewayService.STATE_NOT_RECOVERED_BLOCK; + } + } + + private ClusterState.Builder randomMetaDataChanges(ClusterState clusterState) { + MetaData metaData = clusterState.metaData(); + int changesCount = randomIntBetween(1, 10); + for (int i = 0; i < changesCount; i++) { + switch (randomInt(3)) { + case 0: + metaData = randomMetaDataSettings(metaData); + break; + case 1: + metaData = randomIndices(metaData); + break; + case 2: + metaData = randomTemplates(metaData); + break; + case 3: + metaData = randomMetaDataCustoms(metaData); + break; + default: + throw new IllegalArgumentException("Shouldn't be here"); + } + } + return ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).version(metaData.version() + 1).build()); + } + + private Settings randomSettings(Settings settings) { + ImmutableSettings.Builder builder = ImmutableSettings.builder(); + if (randomBoolean()) { + builder.put(settings); + } + int settingsCount = randomInt(10); + for (int i = 0; i < settingsCount; i++) { + builder.put(randomAsciiOfLength(10), randomAsciiOfLength(10)); + } + return builder.build(); + + } + + private MetaData randomMetaDataSettings(MetaData metaData) { + if (randomBoolean()) { + return MetaData.builder(metaData).persistentSettings(randomSettings(metaData.persistentSettings())).build(); + } else { + return MetaData.builder(metaData).transientSettings(randomSettings(metaData.transientSettings())).build(); + } + } + + private interface RandomPart { + /** + * Returns list of parts from metadata + */ + ImmutableOpenMap parts(MetaData metaData); + + /** + * Puts the part back into metadata + */ + MetaData.Builder put(MetaData.Builder builder, T part); + + /** + * Remove the part from metadata + */ + MetaData.Builder remove(MetaData.Builder builder, String name); + + /** + * Returns a random part with the specified name + */ + T randomCreate(String name); + + /** + * Makes random modifications to the part + */ + T randomChange(T part); + + } + + private MetaData randomParts(MetaData metaData, String prefix, RandomPart randomPart) { + MetaData.Builder builder = MetaData.builder(metaData); + ImmutableOpenMap parts = randomPart.parts(metaData); + int partCount = parts.size(); + if (partCount > 0) { + List randomParts = randomSubsetOf(randomInt(partCount - 1), randomPart.parts(metaData).keys().toArray(String.class)); + for (String part : randomParts) { + if (randomBoolean()) { + randomPart.remove(builder, part); + } else { + randomPart.put(builder, randomPart.randomChange(parts.get(part))); + } + } + } + int additionalPartCount = randomIntBetween(1, 20); + for (int i = 0; i < additionalPartCount; i++) { + String name = randomName(prefix); + randomPart.put(builder, randomPart.randomCreate(name)); + } + return builder.build(); + } + + private MetaData randomIndices(MetaData metaData) { + return randomParts(metaData, "index", new RandomPart() { + + @Override + public ImmutableOpenMap parts(MetaData metaData) { + return metaData.indices(); + } + + @Override + public MetaData.Builder put(MetaData.Builder builder, IndexMetaData part) { + return builder.put(part, true); + } + + @Override + public MetaData.Builder remove(MetaData.Builder builder, String name) { + return builder.remove(name); + } + + @Override + public IndexMetaData randomCreate(String name) { + IndexMetaData.Builder builder = IndexMetaData.builder(name); + ImmutableSettings.Builder settingsBuilder = ImmutableSettings.builder(); + setRandomSettings(getRandom(), settingsBuilder); + settingsBuilder.put(randomSettings(ImmutableSettings.EMPTY)).put(IndexMetaData.SETTING_VERSION_CREATED, randomVersion(random())); + builder.settings(settingsBuilder); + builder.numberOfShards(randomIntBetween(1, 10)).numberOfReplicas(randomInt(10)); + int aliasCount = randomInt(10); + if (randomBoolean()) { + builder.putCustom(IndexWarmersMetaData.TYPE, randomWarmers()); + } + for (int i = 0; i < aliasCount; i++) { + builder.putAlias(randomAlias()); + } + return builder.build(); + } + + @Override + public IndexMetaData randomChange(IndexMetaData part) { + IndexMetaData.Builder builder = IndexMetaData.builder(part); + switch (randomIntBetween(0, 3)) { + case 0: + builder.settings(ImmutableSettings.builder().put(part.settings()).put(randomSettings(ImmutableSettings.EMPTY))); + break; + case 1: + if (randomBoolean() && part.aliases().isEmpty() == false) { + builder.removeAlias(randomFrom(part.aliases().keys().toArray(String.class))); + } else { + builder.putAlias(AliasMetaData.builder(randomAsciiOfLength(10))); + } + break; + case 2: + builder.settings(ImmutableSettings.builder().put(part.settings()).put(IndexMetaData.SETTING_UUID, Strings.randomBase64UUID())); + break; + case 3: + builder.putCustom(IndexWarmersMetaData.TYPE, randomWarmers()); + break; + default: + throw new IllegalArgumentException("Shouldn't be here"); + } + return builder.build(); + } + }); + } + + private IndexWarmersMetaData randomWarmers() { + if (randomBoolean()) { + return new IndexWarmersMetaData( + new IndexWarmersMetaData.Entry( + randomName("warm"), + new String[]{randomName("type")}, + randomBoolean(), + new BytesArray(randomAsciiOfLength(1000))) + ); + } else { + return new IndexWarmersMetaData(); + } + } + + private MetaData randomTemplates(MetaData metaData) { + return randomParts(metaData, "template", new RandomPart() { + @Override + public ImmutableOpenMap parts(MetaData metaData) { + return metaData.templates(); + } + + @Override + public MetaData.Builder put(MetaData.Builder builder, IndexTemplateMetaData part) { + return builder.put(part); + } + + @Override + public MetaData.Builder remove(MetaData.Builder builder, String name) { + return builder.removeTemplate(name); + } + + @Override + public IndexTemplateMetaData randomCreate(String name) { + IndexTemplateMetaData.Builder builder = IndexTemplateMetaData.builder(name); + builder.order(randomInt(1000)) + .template(randomName("temp")) + .settings(randomSettings(ImmutableSettings.EMPTY)); + int aliasCount = randomIntBetween(0, 10); + for (int i = 0; i < aliasCount; i++) { + builder.putAlias(randomAlias()); + } + if (randomBoolean()) { + builder.putCustom(IndexWarmersMetaData.TYPE, randomWarmers()); + } + return builder.build(); + } + + @Override + public IndexTemplateMetaData randomChange(IndexTemplateMetaData part) { + IndexTemplateMetaData.Builder builder = new IndexTemplateMetaData.Builder(part); + builder.order(randomInt(1000)); + return builder.build(); + } + }); + } + + private AliasMetaData randomAlias() { + AliasMetaData.Builder builder = newAliasMetaDataBuilder(randomName("alias")); + if (randomBoolean()) { + builder.filter(FilterBuilders.termFilter("test", randomRealisticUnicodeOfCodepointLength(10)).toString()); + } + if (randomBoolean()) { + builder.routing(randomAsciiOfLength(10)); + } + return builder.build(); + } + + private MetaData randomMetaDataCustoms(final MetaData metaData) { + return randomParts(metaData, "custom", new RandomPart() { + + @Override + public ImmutableOpenMap parts(MetaData metaData) { + return metaData.customs(); + } + + @Override + public MetaData.Builder put(MetaData.Builder builder, MetaData.Custom part) { + if (part instanceof SnapshotMetaData) { + return builder.putCustom(SnapshotMetaData.TYPE, part); + } else if (part instanceof RepositoriesMetaData) { + return builder.putCustom(RepositoriesMetaData.TYPE, part); + } else if (part instanceof RestoreMetaData) { + return builder.putCustom(RestoreMetaData.TYPE, part); + } + throw new IllegalArgumentException("Unknown custom part " + part); + } + + @Override + public MetaData.Builder remove(MetaData.Builder builder, String name) { + return builder.removeCustom(name); + } + + @Override + public MetaData.Custom randomCreate(String name) { + switch (randomIntBetween(0, 2)) { + case 0: + return new SnapshotMetaData(new SnapshotMetaData.Entry( + new SnapshotId(randomName("repo"), randomName("snap")), + randomBoolean(), + SnapshotMetaData.State.fromValue((byte) randomIntBetween(0, 6)), + ImmutableList.of(), + Math.abs(randomLong()), + ImmutableMap.of())); + case 1: + return new RepositoriesMetaData(); + case 2: + return new RestoreMetaData(new RestoreMetaData.Entry( + new SnapshotId(randomName("repo"), randomName("snap")), + RestoreMetaData.State.fromValue((byte) randomIntBetween(0, 3)), + ImmutableList.of(), + ImmutableMap.of())); + default: + throw new IllegalArgumentException("Shouldn't be here"); + } + } + + @Override + public MetaData.Custom randomChange(MetaData.Custom part) { + return part; + } + }); + } + + private String randomName(String prefix) { + return prefix + Strings.randomBase64UUID(getRandom()); + } +} \ No newline at end of file diff --git a/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java b/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java index cbbff463f20..83a27850591 100644 --- a/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java +++ b/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java @@ -81,7 +81,7 @@ public class ClusterSerializationTests extends ElasticsearchAllocationTestCase { RoutingTable source = strategy.reroute(clusterState).routingTable(); BytesStreamOutput outStream = new BytesStreamOutput(); - RoutingTable.Builder.writeTo(source, outStream); + source.writeTo(outStream); BytesStreamInput inStream = new BytesStreamInput(outStream.bytes().toBytes()); RoutingTable target = RoutingTable.Builder.readFrom(inStream); diff --git a/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java b/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java new file mode 100644 index 00000000000..d87d900a0e8 --- /dev/null +++ b/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java @@ -0,0 +1,127 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster.serialization; + +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.cluster.Diff; +import org.elasticsearch.cluster.DiffableUtils; +import org.elasticsearch.cluster.DiffableUtils.KeyedReader; +import org.elasticsearch.cluster.AbstractDiffable; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.io.stream.*; +import org.elasticsearch.test.ElasticsearchTestCase; +import org.junit.Test; + +import java.io.IOException; +import java.util.Map; + +import static com.google.common.collect.Maps.newHashMap; +import static org.hamcrest.CoreMatchers.equalTo; + +public class DiffableTests extends ElasticsearchTestCase { + + @Test + public void testImmutableMapDiff() throws IOException { + ImmutableMap.Builder builder = ImmutableMap.builder(); + builder.put("foo", new TestDiffable("1")); + builder.put("bar", new TestDiffable("2")); + builder.put("baz", new TestDiffable("3")); + ImmutableMap before = builder.build(); + Map map = newHashMap(); + map.putAll(before); + map.remove("bar"); + map.put("baz", new TestDiffable("4")); + map.put("new", new TestDiffable("5")); + ImmutableMap after = ImmutableMap.copyOf(map); + Diff diff = DiffableUtils.diff(before, after); + BytesStreamOutput out = new BytesStreamOutput(); + diff.writeTo(out); + BytesStreamInput in = new BytesStreamInput(out.bytes()); + ImmutableMap serialized = DiffableUtils.readImmutableMapDiff(in, TestDiffable.PROTO).apply(before); + assertThat(serialized.size(), equalTo(3)); + assertThat(serialized.get("foo").value(), equalTo("1")); + assertThat(serialized.get("baz").value(), equalTo("4")); + assertThat(serialized.get("new").value(), equalTo("5")); + } + + @Test + public void testImmutableOpenMapDiff() throws IOException { + ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); + builder.put("foo", new TestDiffable("1")); + builder.put("bar", new TestDiffable("2")); + builder.put("baz", new TestDiffable("3")); + ImmutableOpenMap before = builder.build(); + builder = ImmutableOpenMap.builder(before); + builder.remove("bar"); + builder.put("baz", new TestDiffable("4")); + builder.put("new", new TestDiffable("5")); + ImmutableOpenMap after = builder.build(); + Diff diff = DiffableUtils.diff(before, after); + BytesStreamOutput out = new BytesStreamOutput(); + diff.writeTo(out); + BytesStreamInput in = new BytesStreamInput(out.bytes()); + ImmutableOpenMap serialized = DiffableUtils.readImmutableOpenMapDiff(in, new KeyedReader() { + @Override + public TestDiffable readFrom(StreamInput in, String key) throws IOException { + return new TestDiffable(in.readString()); + } + + @Override + public Diff readDiffFrom(StreamInput in, String key) throws IOException { + return AbstractDiffable.readDiffFrom(new StreamableReader() { + @Override + public TestDiffable readFrom(StreamInput in) throws IOException { + return new TestDiffable(in.readString()); + } + }, in); + } + }).apply(before); + assertThat(serialized.size(), equalTo(3)); + assertThat(serialized.get("foo").value(), equalTo("1")); + assertThat(serialized.get("baz").value(), equalTo("4")); + assertThat(serialized.get("new").value(), equalTo("5")); + + } + public static class TestDiffable extends AbstractDiffable { + + public static final TestDiffable PROTO = new TestDiffable(""); + + private final String value; + + public TestDiffable(String value) { + this.value = value; + } + + public String value() { + return value; + } + + @Override + public TestDiffable readFrom(StreamInput in) throws IOException { + return new TestDiffable(in.readString()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(value); + } + } + +} diff --git a/src/test/java/org/elasticsearch/common/xcontent/XContentTestUtils.java b/src/test/java/org/elasticsearch/common/xcontent/XContentTestUtils.java new file mode 100644 index 00000000000..9ebffe58783 --- /dev/null +++ b/src/test/java/org/elasticsearch/common/xcontent/XContentTestUtils.java @@ -0,0 +1,100 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.xcontent; + +import com.carrotsearch.ant.tasks.junit4.dependencies.com.google.common.collect.Lists; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; + +public final class XContentTestUtils { + private XContentTestUtils() { + + } + + public static Map convertToMap(ToXContent part) throws IOException { + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.startObject(); + part.toXContent(builder, EMPTY_PARAMS); + builder.endObject(); + return XContentHelper.convertToMap(builder.bytes(), false).v2(); + } + + + /** + * Compares to maps generated from XContentObjects. The order of elements in arrays is ignored + */ + public static boolean mapsEqualIgnoringArrayOrder(Map first, Map second) { + if (first.size() != second.size()) { + return false; + } + + for (String key : first.keySet()) { + if (objectsEqualIgnoringArrayOrder(first.get(key), second.get(key)) == false) { + return false; + } + } + return true; + } + + @SuppressWarnings("unchecked") + private static boolean objectsEqualIgnoringArrayOrder(Object first, Object second) { + if (first == null ) { + return second == null; + } else if (first instanceof List) { + if (second instanceof List) { + List secondList = Lists.newArrayList((List) second); + List firstList = (List) first; + if (firstList.size() == secondList.size()) { + for (Object firstObj : firstList) { + boolean found = false; + for (Object secondObj : secondList) { + if (objectsEqualIgnoringArrayOrder(firstObj, secondObj)) { + secondList.remove(secondObj); + found = true; + break; + } + } + if (found == false) { + return false; + } + } + return secondList.isEmpty(); + } else { + return false; + } + } else { + return false; + } + } else if (first instanceof Map) { + if (second instanceof Map) { + return mapsEqualIgnoringArrayOrder((Map) first, (Map) second); + } else { + return false; + } + } else { + return first.equals(second); + } + } + +} diff --git a/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryTests.java b/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryTests.java index f265869ec75..f1e7a249c59 100644 --- a/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryTests.java +++ b/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope; import org.elasticsearch.test.ElasticsearchIntegrationTest.Scope; import org.elasticsearch.test.discovery.ClusterDiscoveryConfiguration; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.Before; import org.junit.Test; diff --git a/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryTests.java b/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryTests.java index 58e177b1115..228faa8cf4d 100644 --- a/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryTests.java +++ b/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryTests.java @@ -32,9 +32,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Priority; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.compress.CompressorFactory; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; @@ -196,12 +193,7 @@ public class ZenDiscoveryTests extends ElasticsearchIntegrationTest { .put(new DiscoveryNode("abc", new LocalTransportAddress("abc"), Version.CURRENT)).masterNodeId("abc"); ClusterState.Builder builder = ClusterState.builder(state); builder.nodes(nodes); - BytesStreamOutput bStream = new BytesStreamOutput(); - StreamOutput stream = CompressorFactory.defaultCompressor().streamOutput(bStream); - stream.setVersion(node.version()); - ClusterState.Builder.writeTo(builder.build(), stream); - stream.close(); - BytesReference bytes = bStream.bytes(); + BytesReference bytes = PublishClusterStateAction.serializeFullClusterState(builder.build(), node.version()); final CountDownLatch latch = new CountDownLatch(1); final AtomicReference reference = new AtomicReference<>(); diff --git a/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java b/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java index c5adf8cb50e..c97fa5b789d 100644 --- a/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java +++ b/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java @@ -443,11 +443,11 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { new MappingMetaData.Id(null), new MappingMetaData.Routing(false, null), timestamp, false); BytesStreamOutput out = new BytesStreamOutput(); - MappingMetaData.writeTo(expected, out); + expected.writeTo(out); out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); assertThat(metaData, is(expected)); } @@ -460,11 +460,11 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { new MappingMetaData.Id(null), new MappingMetaData.Routing(false, null), timestamp, false); BytesStreamOutput out = new BytesStreamOutput(); - MappingMetaData.writeTo(expected, out); + expected.writeTo(out); out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); assertThat(metaData, is(expected)); } @@ -477,11 +477,11 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { new MappingMetaData.Id(null), new MappingMetaData.Routing(false, null), timestamp, false); BytesStreamOutput out = new BytesStreamOutput(); - MappingMetaData.writeTo(expected, out); + expected.writeTo(out); out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); assertThat(metaData, is(expected)); } diff --git a/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationTests.java b/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationTests.java index 386c778b07e..9e05d915803 100644 --- a/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationTests.java +++ b/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.disruption.SlowClusterStateProcessing; import org.junit.Test; +import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.util.Arrays; @@ -68,6 +69,12 @@ public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest { .build(); } + @Override + protected void ensureClusterStateConsistency() throws IOException { + // testShardActiveElseWhere might change the state of a non-master node + // so we cannot check state consistency of this cluster + } + @Test public void indexCleanup() throws Exception { final String masterNode = internalCluster().startNode(ImmutableSettings.builder().put("node.data", false)); diff --git a/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateTests.java b/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateTests.java index ce96576ce15..bd664694c9f 100644 --- a/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateTests.java +++ b/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateTests.java @@ -284,6 +284,7 @@ public class SimpleIndexTemplateTests extends ElasticsearchIntegrationTest { } @Test + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/8802") public void testBrokenMapping() throws Exception { // clean all templates setup by the framework. client().admin().indices().prepareDeleteTemplate("*").get(); diff --git a/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java b/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java index ff8264fdc03..8d569275aea 100644 --- a/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java +++ b/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java @@ -38,7 +38,9 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.metadata.MetaData.Custom; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Priority; import org.elasticsearch.common.io.stream.StreamInput; @@ -748,7 +750,7 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests )); } - public static abstract class TestCustomMetaData implements MetaData.Custom { + public static abstract class TestCustomMetaData extends AbstractDiffable implements MetaData.Custom { private final String data; protected TestCustomMetaData(String data) { @@ -776,194 +778,182 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests return data.hashCode(); } - public static abstract class TestCustomMetaDataFactory extends MetaData.Custom.Factory { + protected abstract TestCustomMetaData newTestCustomMetaData(String data); - protected abstract TestCustomMetaData newTestCustomMetaData(String data); + @Override + public Custom readFrom(StreamInput in) throws IOException { + return newTestCustomMetaData(in.readString()); + } - @Override - public T readFrom(StreamInput in) throws IOException { - return (T) newTestCustomMetaData(in.readString()); - } + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(getData()); + } - @Override - public void writeTo(T metadata, StreamOutput out) throws IOException { - out.writeString(metadata.getData()); - } - - @Override - public T fromXContent(XContentParser parser) throws IOException { - XContentParser.Token token; - String data = null; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - String currentFieldName = parser.currentName(); - if ("data".equals(currentFieldName)) { - if (parser.nextToken() != XContentParser.Token.VALUE_STRING) { - throw new ElasticsearchParseException("failed to parse snapshottable metadata, invalid data type"); - } - data = parser.text(); - } else { - throw new ElasticsearchParseException("failed to parse snapshottable metadata, unknown field [" + currentFieldName + "]"); + @Override + public Custom fromXContent(XContentParser parser) throws IOException { + XContentParser.Token token; + String data = null; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + String currentFieldName = parser.currentName(); + if ("data".equals(currentFieldName)) { + if (parser.nextToken() != XContentParser.Token.VALUE_STRING) { + throw new ElasticsearchParseException("failed to parse snapshottable metadata, invalid data type"); } + data = parser.text(); } else { - throw new ElasticsearchParseException("failed to parse snapshottable metadata"); + throw new ElasticsearchParseException("failed to parse snapshottable metadata, unknown field [" + currentFieldName + "]"); } + } else { + throw new ElasticsearchParseException("failed to parse snapshottable metadata"); } - if (data == null) { - throw new ElasticsearchParseException("failed to parse snapshottable metadata, data not found"); - } - return (T) newTestCustomMetaData(data); } + if (data == null) { + throw new ElasticsearchParseException("failed to parse snapshottable metadata, data not found"); + } + return newTestCustomMetaData(data); + } - @Override - public void toXContent(T metadata, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.field("data", metadata.getData()); - } + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.field("data", getData()); + return builder; } } + static { - MetaData.registerFactory(SnapshottableMetadata.TYPE, SnapshottableMetadata.FACTORY); - MetaData.registerFactory(NonSnapshottableMetadata.TYPE, NonSnapshottableMetadata.FACTORY); - MetaData.registerFactory(SnapshottableGatewayMetadata.TYPE, SnapshottableGatewayMetadata.FACTORY); - MetaData.registerFactory(NonSnapshottableGatewayMetadata.TYPE, NonSnapshottableGatewayMetadata.FACTORY); - MetaData.registerFactory(SnapshotableGatewayNoApiMetadata.TYPE, SnapshotableGatewayNoApiMetadata.FACTORY); + MetaData.registerPrototype(SnapshottableMetadata.TYPE, SnapshottableMetadata.PROTO); + MetaData.registerPrototype(NonSnapshottableMetadata.TYPE, NonSnapshottableMetadata.PROTO); + MetaData.registerPrototype(SnapshottableGatewayMetadata.TYPE, SnapshottableGatewayMetadata.PROTO); + MetaData.registerPrototype(NonSnapshottableGatewayMetadata.TYPE, NonSnapshottableGatewayMetadata.PROTO); + MetaData.registerPrototype(SnapshotableGatewayNoApiMetadata.TYPE, SnapshotableGatewayNoApiMetadata.PROTO); } public static class SnapshottableMetadata extends TestCustomMetaData { public static final String TYPE = "test_snapshottable"; - public static final Factory FACTORY = new Factory(); + public static final SnapshottableMetadata PROTO = new SnapshottableMetadata(""); public SnapshottableMetadata(String data) { super(data); } - private static class Factory extends TestCustomMetaDataFactory { + @Override + public String type() { + return TYPE; + } - @Override - public String type() { - return TYPE; - } + @Override + protected TestCustomMetaData newTestCustomMetaData(String data) { + return new SnapshottableMetadata(data); + } - @Override - protected TestCustomMetaData newTestCustomMetaData(String data) { - return new SnapshottableMetadata(data); - } - - @Override - public EnumSet context() { - return MetaData.API_AND_SNAPSHOT; - } + @Override + public EnumSet context() { + return MetaData.API_AND_SNAPSHOT; } } public static class NonSnapshottableMetadata extends TestCustomMetaData { public static final String TYPE = "test_non_snapshottable"; - public static final Factory FACTORY = new Factory(); + public static final NonSnapshottableMetadata PROTO = new NonSnapshottableMetadata(""); public NonSnapshottableMetadata(String data) { super(data); } - private static class Factory extends TestCustomMetaDataFactory { + @Override + public String type() { + return TYPE; + } - @Override - public String type() { - return TYPE; - } + @Override + protected NonSnapshottableMetadata newTestCustomMetaData(String data) { + return new NonSnapshottableMetadata(data); + } - @Override - protected NonSnapshottableMetadata newTestCustomMetaData(String data) { - return new NonSnapshottableMetadata(data); - } + @Override + public EnumSet context() { + return MetaData.API_ONLY; } } public static class SnapshottableGatewayMetadata extends TestCustomMetaData { public static final String TYPE = "test_snapshottable_gateway"; - public static final Factory FACTORY = new Factory(); + public static final SnapshottableGatewayMetadata PROTO = new SnapshottableGatewayMetadata(""); public SnapshottableGatewayMetadata(String data) { super(data); } - private static class Factory extends TestCustomMetaDataFactory { + @Override + public String type() { + return TYPE; + } - @Override - public String type() { - return TYPE; - } + @Override + protected TestCustomMetaData newTestCustomMetaData(String data) { + return new SnapshottableGatewayMetadata(data); + } - @Override - protected TestCustomMetaData newTestCustomMetaData(String data) { - return new SnapshottableGatewayMetadata(data); - } - - @Override - public EnumSet context() { - return EnumSet.of(MetaData.XContentContext.API, MetaData.XContentContext.SNAPSHOT, MetaData.XContentContext.GATEWAY); - } + @Override + public EnumSet context() { + return EnumSet.of(MetaData.XContentContext.API, MetaData.XContentContext.SNAPSHOT, MetaData.XContentContext.GATEWAY); } } public static class NonSnapshottableGatewayMetadata extends TestCustomMetaData { public static final String TYPE = "test_non_snapshottable_gateway"; - public static final Factory FACTORY = new Factory(); + public static final NonSnapshottableGatewayMetadata PROTO = new NonSnapshottableGatewayMetadata(""); public NonSnapshottableGatewayMetadata(String data) { super(data); } - private static class Factory extends TestCustomMetaDataFactory { - - @Override - public String type() { - return TYPE; - } - - @Override - protected NonSnapshottableGatewayMetadata newTestCustomMetaData(String data) { - return new NonSnapshottableGatewayMetadata(data); - } - - @Override - public EnumSet context() { - return MetaData.API_AND_GATEWAY; - } - + @Override + public String type() { + return TYPE; } + + @Override + protected NonSnapshottableGatewayMetadata newTestCustomMetaData(String data) { + return new NonSnapshottableGatewayMetadata(data); + } + + @Override + public EnumSet context() { + return MetaData.API_AND_GATEWAY; + } + } public static class SnapshotableGatewayNoApiMetadata extends TestCustomMetaData { public static final String TYPE = "test_snapshottable_gateway_no_api"; - public static final Factory FACTORY = new Factory(); + public static final SnapshotableGatewayNoApiMetadata PROTO = new SnapshotableGatewayNoApiMetadata(""); public SnapshotableGatewayNoApiMetadata(String data) { super(data); } - private static class Factory extends TestCustomMetaDataFactory { + @Override + public String type() { + return TYPE; + } - @Override - public String type() { - return TYPE; - } - - @Override - protected SnapshotableGatewayNoApiMetadata newTestCustomMetaData(String data) { - return new SnapshotableGatewayNoApiMetadata(data); - } - - @Override - public EnumSet context() { - return EnumSet.of(MetaData.XContentContext.GATEWAY, MetaData.XContentContext.SNAPSHOT); - } + @Override + protected SnapshotableGatewayNoApiMetadata newTestCustomMetaData(String data) { + return new SnapshotableGatewayNoApiMetadata(data); + } + @Override + public EnumSet context() { + return EnumSet.of(MetaData.XContentContext.GATEWAY, MetaData.XContentContext.SNAPSHOT); } } diff --git a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java index 9e68d16caa0..f30a47755ed 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java @@ -166,6 +166,8 @@ import java.util.concurrent.atomic.AtomicLong; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; +import static org.elasticsearch.common.xcontent.XContentTestUtils.convertToMap; +import static org.elasticsearch.common.xcontent.XContentTestUtils.mapsEqualIgnoringArrayOrder; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; @@ -357,7 +359,7 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase * Creates a randomized index template. This template is used to pass in randomized settings on a * per index basis. Allows to enable/disable the randomization for number of shards and replicas */ - private void randomIndexTemplate() throws IOException { + public void randomIndexTemplate() throws IOException { // TODO move settings for random directory etc here into the index based randomized settings. if (cluster().size() > 0) { @@ -647,6 +649,7 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase .transientSettings().getAsMap().size(), equalTo(0)); } ensureClusterSizeConsistency(); + ensureClusterStateConsistency(); cluster().wipe(); // wipe after to make sure we fail in the test that didn't ack the delete if (afterClass || currentClusterScope == Scope.TEST) { cluster().close(); @@ -1085,8 +1088,8 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase */ public void setMinimumMasterNodes(int n) { assertTrue(client().admin().cluster().prepareUpdateSettings().setTransientSettings( - settingsBuilder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, n)) - .get().isAcknowledged()); + settingsBuilder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, n)) + .get().isAcknowledged()); } /** @@ -1133,6 +1136,35 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase } } + /** + * Verifies that all nodes that have the same version of the cluster state as master have same cluster state + */ + protected void ensureClusterStateConsistency() throws IOException { + if (cluster() != null) { + ClusterState masterClusterState = client().admin().cluster().prepareState().all().get().getState(); + Map masterStateMap = convertToMap(masterClusterState); + int masterClusterStateSize = ClusterState.Builder.toBytes(masterClusterState).length; + for (Client client : cluster()) { + ClusterState localClusterState = client.admin().cluster().prepareState().all().setLocal(true).get().getState(); + if (masterClusterState.version() == localClusterState.version()) { + try { + assertThat(masterClusterState.uuid(), equalTo(localClusterState.uuid())); + // We cannot compare serialization bytes since serialization order of maps is not guaranteed + // but we can compare serialization sizes - they should be the same + int localClusterStateSize = ClusterState.Builder.toBytes(localClusterState).length; + assertThat(masterClusterStateSize, equalTo(localClusterStateSize)); + + // Compare JSON serialization + assertThat(mapsEqualIgnoringArrayOrder(masterStateMap, convertToMap(localClusterState)), equalTo(true)); + } catch (AssertionError error) { + logger.error("Cluster state from master:\n{}\nLocal cluster state:\n{}", masterClusterState.toString(), localClusterState.toString()); + throw error; + } + } + } + } + } + /** * Ensures the cluster is in a searchable state for the given indices. This means a searchable copy of each * shard is available on the cluster. diff --git a/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java b/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java index 1276089b182..0f71b7239e0 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java @@ -71,6 +71,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import static com.google.common.collect.Lists.newArrayList; /** * Base testcase for randomized unit testing with Elasticsearch @@ -595,4 +596,17 @@ public abstract class ElasticsearchTestCase extends LuceneTestCase { return threadGroup.getName(); } } + + /** + * Returns size random values + */ + public static List randomSubsetOf(int size, T... values) { + if (size > values.length) { + throw new IllegalArgumentException("Can\'t pick " + size + " random objects from a list of " + values.length + " objects"); + } + List list = newArrayList(values); + Collections.shuffle(list); + return list.subList(0, size); + } + }