From 061a010453551858cec8f657115f0e2a69629fb2 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 27 Apr 2015 09:07:02 +0200 Subject: [PATCH] Revert "Add support for cluster state diffs" This reverts commit d746e14cf3f6d9b25a2c6018eedca249555cf44b. --- .../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 | 222 +++---- .../cluster/metadata/RepositoryMetaData.java | 21 - .../cluster/metadata/RestoreMetaData.java | 226 +++---- .../cluster/metadata/SnapshotMetaData.java | 231 ++++--- .../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 | 8 +- .../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 | 304 ++++----- .../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 | 214 +++--- .../test/ElasticsearchIntegrationTest.java | 38 +- .../test/ElasticsearchTestCase.java | 14 - 60 files changed, 1128 insertions(+), 3818 deletions(-) delete mode 100644 src/main/java/org/elasticsearch/cluster/AbstractDiffable.java delete mode 100644 src/main/java/org/elasticsearch/cluster/Diff.java delete mode 100644 src/main/java/org/elasticsearch/cluster/Diffable.java delete mode 100644 src/main/java/org/elasticsearch/cluster/DiffableUtils.java delete mode 100644 src/main/java/org/elasticsearch/cluster/IncompatibleClusterStateVersionException.java delete mode 100644 src/main/java/org/elasticsearch/common/io/stream/StreamableReader.java delete mode 100644 src/main/java/org/elasticsearch/common/io/stream/Writeable.java delete mode 100644 src/test/java/org/elasticsearch/cluster/ClusterStateDiffPublishingTests.java delete mode 100644 src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java delete mode 100644 src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java delete 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 28f9cb1db90..79b31f620d5 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); - state.writeTo(out); + ClusterState.Builder.writeTo(state, 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 e9aa9b723fa..861a84a9e71 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.writeTo(out); + ClusterState.Builder.writeTo(clusterState, 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 150a15eacfd..fc1db98c35e 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,6 +19,7 @@ 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; @@ -28,6 +29,7 @@ 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; @@ -37,6 +39,11 @@ 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; + /** * */ @@ -77,7 +84,6 @@ 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()); } @@ -116,9 +122,10 @@ public class TransportClusterStateAction extends TransportMasterNodeReadOperatio } // Filter our metadata that shouldn't be returned by API - for(ObjectObjectCursor custom : currentState.metaData().customs()) { - if(!custom.value.context().contains(MetaData.XContentContext.API)) { - mdBuilder.removeCustom(custom.key); + for(ObjectCursor type : currentState.metaData().customs().keys()) { + Custom.Factory factory = lookupFactorySafe(type.value); + if(!factory.context().contains(MetaData.XContentContext.API)) { + mdBuilder.removeCustom(type.value); } } 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 106e864a367..765a9395afc 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.writeTo(out); + AliasMetaData.Builder.writeTo(aliasMetaData, 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 7c795546258..0be9e2767c1 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 @@ -397,11 +397,11 @@ public class CreateIndexRequest extends AcknowledgedRequest aliases((Map) entry.getValue()); } else { // maybe custom? - IndexMetaData.Custom proto = IndexMetaData.lookupPrototype(name); - if (proto != null) { + IndexMetaData.Custom.Factory factory = IndexMetaData.lookupFactory(name); + if (factory != null) { found = true; try { - customs.put(name, proto.fromMap((Map) entry.getValue())); + customs.put(name, factory.fromMap((Map) entry.getValue())); } catch (IOException e) { throw new ElasticsearchParseException("failed to parse custom metadata for [" + name + "]"); } @@ -449,7 +449,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.lookupPrototypeSafe(type).readFrom(in); + IndexMetaData.Custom customIndexMetaData = IndexMetaData.lookupFactorySafe(type).readFrom(in); customs.put(type, customIndexMetaData); } int aliasesSize = in.readVInt(); @@ -473,7 +473,7 @@ public class CreateIndexRequest extends AcknowledgedRequest out.writeVInt(customs.size()); for (Map.Entry entry : customs.entrySet()) { out.writeString(entry.getKey()); - entry.getValue().writeTo(out); + IndexMetaData.lookupFactorySafe(entry.getKey()).writeTo(entry.getValue(), 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 7080a694a11..35e6cfa4804 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.PROTO.readFrom(in)); + mappingEntryBuilder.put(in.readString(), MappingMetaData.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); - mappingEntry.value.writeTo(out); + MappingMetaData.writeTo(mappingEntry.value, 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) { - aliasEntry.writeTo(out); + AliasMetaData.Builder.writeTo(aliasEntry, 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 30e9e24c493..b27577f8da3 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.PROTO.readFrom(in)); + typeMapBuilder.put(in.readString(), MappingMetaData.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); - typeEntry.value.writeTo(out); + MappingMetaData.writeTo(typeEntry.value, 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 2ce6d8d2c1a..56de19872f2 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) { - indexTemplate.writeTo(out); + IndexTemplateMetaData.Builder.writeTo(indexTemplate, 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 608b7fa82b2..b728abf934e 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 @@ -293,10 +293,10 @@ public class PutIndexTemplateRequest extends MasterNodeOperationRequest) entry.getValue()); } else { // maybe custom? - IndexMetaData.Custom proto = IndexMetaData.lookupPrototype(name); - if (proto != null) { + IndexMetaData.Custom.Factory factory = IndexMetaData.lookupFactory(name); + if (factory != null) { try { - customs.put(name, proto.fromMap((Map) entry.getValue())); + customs.put(name, factory.fromMap((Map) entry.getValue())); } catch (IOException e) { throw new ElasticsearchParseException("failed to parse custom metadata for [" + name + "]"); } @@ -441,7 +441,7 @@ public class PutIndexTemplateRequest extends MasterNodeOperationRequest entry : customs.entrySet()) { out.writeString(entry.getKey()); - entry.getValue().writeTo(out); + IndexMetaData.lookupFactorySafe(entry.getKey()).writeTo(entry.getValue(), 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 deleted file mode 100644 index 4e6da2bd569..00000000000 --- a/src/main/java/org/elasticsearch/cluster/AbstractDiffable.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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 b092a121c07..ef4d67740dc 100644 --- a/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -23,7 +23,6 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.collect.ImmutableSet; import org.elasticsearch.ElasticsearchIllegalArgumentException; -import org.elasticsearch.cluster.DiffableUtils.KeyedReader; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -57,9 +56,7 @@ import java.util.Map; /** * */ -public class ClusterState implements ToXContent, Diffable { - - public static final ClusterState PROTO = builder(ClusterName.DEFAULT).build(); +public class ClusterState implements ToXContent { public static enum ClusterStateStatus { UNKNOWN((byte) 0), @@ -78,43 +75,47 @@ public class ClusterState implements ToXContent, Diffable { } } - public interface Custom extends Diffable, ToXContent { + public interface Custom { - String type(); + 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); + } } - private final static Map customPrototypes = new HashMap<>(); + private final static Map customFactories = new HashMap<>(); /** * Register a custom index meta data factory. Make sure to call it from a static block. */ - public static void registerPrototype(String type, Custom proto) { - customPrototypes.put(type, proto); + public static void registerFactory(String type, Custom.Factory factory) { + customFactories.put(type, factory); } @Nullable - public static T lookupPrototype(String type) { - //noinspection unchecked - return (T) customPrototypes.get(type); + public static Custom.Factory lookupFactory(String type) { + return customFactories.get(type); } - public static T lookupPrototypeSafe(String type) throws ElasticsearchIllegalArgumentException { - @SuppressWarnings("unchecked") - T proto = (T)customPrototypes.get(type); - if (proto == null) { - throw new ElasticsearchIllegalArgumentException("No custom state prototype registered for type [" + type + "]"); + public static Custom.Factory lookupFactorySafe(String type) throws ElasticsearchIllegalArgumentException { + Custom.Factory factory = customFactories.get(type); + if (factory == null) { + throw new ElasticsearchIllegalArgumentException("No custom state factory registered for type [" + type + "]"); } - return proto; + return factory; } - 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,20 +128,17 @@ public class ClusterState implements ToXContent, Diffable { private final ClusterName clusterName; - private final boolean wasReadFromDiff; - // built on demand private volatile RoutingNodes routingNodes; private volatile ClusterStateStatus status; - 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(long version, ClusterState state) { + this(state.clusterName, version, state.metaData(), state.routingTable(), state.nodes(), state.blocks(), state.customs()); } - public ClusterState(ClusterName clusterName, long version, String uuid, MetaData metaData, RoutingTable routingTable, DiscoveryNodes nodes, ClusterBlocks blocks, ImmutableOpenMap customs, boolean wasReadFromDiff) { + public ClusterState(ClusterName clusterName, long version, MetaData metaData, RoutingTable routingTable, DiscoveryNodes nodes, ClusterBlocks blocks, ImmutableOpenMap customs) { this.version = version; - this.uuid = uuid; this.clusterName = clusterName; this.metaData = metaData; this.routingTable = routingTable; @@ -148,7 +146,6 @@ public class ClusterState implements ToXContent, Diffable { this.blocks = blocks; this.customs = customs; this.status = ClusterStateStatus.UNKNOWN; - this.wasReadFromDiff = wasReadFromDiff; } public ClusterStateStatus status() { @@ -168,14 +165,6 @@ public class ClusterState implements ToXContent, Diffable { 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; } @@ -228,11 +217,6 @@ public class ClusterState implements ToXContent, Diffable { 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 @@ -248,8 +232,6 @@ public class ClusterState implements ToXContent, Diffable { 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()); @@ -321,13 +303,14 @@ public class ClusterState implements ToXContent, Diffable { } } + + @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)) { @@ -452,7 +435,7 @@ public class ClusterState implements ToXContent, Diffable { for (ObjectObjectCursor cursor : metaData.customs()) { builder.startObject(cursor.key); - cursor.value.toXContent(builder, params); + MetaData.lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); builder.endObject(); } @@ -491,7 +474,7 @@ public class ClusterState implements ToXContent, Diffable { builder.startObject("nodes"); for (RoutingNode routingNode : readOnlyRoutingNodes()) { - builder.startArray(routingNode.nodeId() == null ? "null" : routingNode.nodeId(), XContentBuilder.FieldCaseConversion.NONE); + builder.startArray(routingNode.nodeId(), XContentBuilder.FieldCaseConversion.NONE); for (ShardRouting shardRouting : routingNode) { shardRouting.toXContent(builder, params); } @@ -504,7 +487,7 @@ public class ClusterState implements ToXContent, Diffable { if (metrics.contains(Metric.CUSTOMS)) { for (ObjectObjectCursor cursor : customs) { builder.startObject(cursor.key); - cursor.value.toXContent(builder, params); + lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); builder.endObject(); } } @@ -524,25 +507,21 @@ public class ClusterState implements ToXContent, Diffable { 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) { @@ -596,17 +575,6 @@ public class ClusterState implements ToXContent, Diffable { 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); } @@ -621,26 +589,13 @@ public class ClusterState implements ToXContent, Diffable { 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() { - if (UNKNOWN_UUID.equals(uuid)) { - uuid = Strings.randomBase64UUID(); - } - return new ClusterState(clusterName, version, uuid, metaData, routingTable, nodes, blocks, customs.build(), fromDiff); + return new ClusterState(clusterName, version, metaData, routingTable, nodes, blocks, customs.build()); } public static byte[] toBytes(ClusterState state) throws IOException { BytesStreamOutput os = new BytesStreamOutput(); - state.writeTo(os); + writeTo(state, os); return os.bytes().toBytes(); } @@ -652,152 +607,39 @@ public class ClusterState implements ToXContent, Diffable { 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 { - 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) { + ClusterName clusterName = ClusterName.readClusterName(in); Builder builder = new Builder(clusterName); - if (toUuid.equals(state.uuid)) { - // no need to read the rest - cluster state didn't change - return state; + 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 (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 deleted file mode 100644 index 2e571f43bca..00000000000 --- a/src/main/java/org/elasticsearch/cluster/Diff.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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 deleted file mode 100644 index 7ce60047a2b..00000000000 --- a/src/main/java/org/elasticsearch/cluster/Diffable.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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 deleted file mode 100644 index 4e912a34f97..00000000000 --- a/src/main/java/org/elasticsearch/cluster/DiffableUtils.java +++ /dev/null @@ -1,283 +0,0 @@ -/* - * 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 deleted file mode 100644 index 92f5897bf2e..00000000000 --- a/src/main/java/org/elasticsearch/cluster/IncompatibleClusterStateVersionException.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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 95c0ba7127e..bb7d332de4f 100644 --- a/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java +++ b/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java @@ -23,7 +23,6 @@ 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; @@ -37,12 +36,10 @@ import java.util.Set; /** * Represents current cluster level blocks to block dirty operations done against the cluster. */ -public class ClusterBlocks extends AbstractDiffable { +public class ClusterBlocks { 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; @@ -206,43 +203,6 @@ public class ClusterBlocks extends AbstractDiffable { 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()); @@ -353,7 +313,38 @@ public class ClusterBlocks extends AbstractDiffable { } public static ClusterBlocks readClusterBlocks(StreamInput in) throws IOException { - return PROTO.readFrom(in); + 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(); } } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java index 0f7e55c8087..008935ec026 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java @@ -21,7 +21,6 @@ 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; @@ -39,9 +38,7 @@ import java.util.Set; /** * */ -public class AliasMetaData extends AbstractDiffable { - - public static final AliasMetaData PROTO = new AliasMetaData("", null, null, null); +public class AliasMetaData { private final String alias; @@ -149,48 +146,6 @@ public class AliasMetaData extends AbstractDiffable { 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; @@ -339,12 +294,44 @@ public class AliasMetaData extends AbstractDiffable { return builder.build(); } - public void writeTo(AliasMetaData aliasMetaData, StreamOutput out) throws IOException { - aliasMetaData.writeTo(out); + 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 static AliasMetaData readFrom(StreamInput in) throws IOException { - return PROTO.readFrom(in); + 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); } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index 2005de524bd..1543151fad0 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -25,9 +25,6 @@ import com.google.common.collect.ImmutableMap; import org.elasticsearch.ElasticsearchIllegalArgumentException; import org.elasticsearch.ElasticsearchIllegalStateException; 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; @@ -64,54 +61,60 @@ import static org.elasticsearch.common.settings.ImmutableSettings.*; /** * */ -public class IndexMetaData implements Diffable { +public class IndexMetaData { - 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 extends Diffable, ToXContent { + public interface Custom { String type(); - Custom fromMap(Map map) throws IOException; + interface Factory { - Custom fromXContent(XContentParser parser) throws IOException; + String type(); - /** - * 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); + 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); + } } - public static Map customPrototypes = new HashMap<>(); + public static Map customFactories = new HashMap<>(); static { // register non plugin custom metadata - registerPrototype(IndexWarmersMetaData.TYPE, IndexWarmersMetaData.PROTO); + registerFactory(IndexWarmersMetaData.TYPE, IndexWarmersMetaData.FACTORY); } /** * Register a custom index meta data factory. Make sure to call it from a static block. */ - public static void registerPrototype(String type, Custom proto) { - customPrototypes.put(type, proto); + public static void registerFactory(String type, Custom.Factory factory) { + customFactories.put(type, factory); } @Nullable - public static T lookupPrototype(String type) { - //noinspection unchecked - return (T) customPrototypes.get(type); + public static Custom.Factory lookupFactory(String type) { + return customFactories.get(type); } - public static T lookupPrototypeSafe(String type) throws ElasticsearchIllegalArgumentException { - //noinspection unchecked - T proto = (T) customPrototypes.get(type); - if (proto == null) { - throw new ElasticsearchIllegalArgumentException("No custom metadata prototype registered for type [" + type + "]"); + public static Custom.Factory lookupFactorySafe(String type) throws ElasticsearchIllegalArgumentException { + Custom.Factory factory = customFactories.get(type); + if (factory == null) { + throw new ElasticsearchIllegalArgumentException("No custom index metadata factoy registered for type [" + type + "]"); } - return proto; + return factory; } 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)); @@ -450,9 +453,7 @@ public class IndexMetaData implements Diffable { if (state != that.state) { return false; } - if (!customs.equals(that.customs)) { - return false; - } + return true; } @@ -466,126 +467,6 @@ public class IndexMetaData implements Diffable { 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); } @@ -781,7 +662,7 @@ public class IndexMetaData implements Diffable { for (ObjectObjectCursor cursor : indexMetaData.customs()) { builder.startObject(cursor.key, XContentBuilder.FieldCaseConversion.NONE); - cursor.value.toXContent(builder, params); + lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); builder.endObject(); } @@ -828,13 +709,12 @@ public class IndexMetaData implements Diffable { } } else { // check if its a custom index metadata - Custom proto = lookupPrototype(currentFieldName); - if (proto == null) { + Custom.Factory factory = lookupFactory(currentFieldName); + if (factory == null) { //TODO warn parser.skipChildren(); } else { - Custom custom = proto.fromXContent(parser); - builder.putCustom(custom.type(), custom); + builder.putCustom(factory.type(), factory.fromXContent(parser)); } } } else if (token == XContentParser.Token.START_ARRAY) { @@ -863,7 +743,47 @@ public class IndexMetaData implements Diffable { } public static IndexMetaData readFrom(StreamInput in) throws IOException { - return PROTO.readFrom(in); + 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); + } } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java index 54150ee6a1e..582e008550d 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.cluster.AbstractDiffable; +import org.elasticsearch.Version; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.compress.CompressedString; @@ -42,9 +42,7 @@ import java.util.Set; /** * */ -public class IndexTemplateMetaData extends AbstractDiffable { - - public static final IndexTemplateMetaData PROTO = IndexTemplateMetaData.builder("").build(); +public class IndexTemplateMetaData { private final String name; @@ -163,57 +161,11 @@ public class IndexTemplateMetaData extends AbstractDiffable 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.customPrototypes.keySet()); + VALID_FIELDS.addAll(IndexMetaData.customFactories.keySet()); } private String name; @@ -353,7 +305,7 @@ public class IndexTemplateMetaData extends AbstractDiffable cursor : indexTemplateMetaData.customs()) { builder.startObject(cursor.key, XContentBuilder.FieldCaseConversion.NONE); - cursor.value.toXContent(builder, params); + IndexMetaData.lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); builder.endObject(); } @@ -395,13 +347,12 @@ public class IndexTemplateMetaData extends AbstractDiffable factory = IndexMetaData.lookupFactory(currentFieldName); + if (factory == null) { //TODO warn parser.skipChildren(); } else { - IndexMetaData.Custom custom = proto.fromXContent(parser); - builder.putCustom(custom.type(), custom); + builder.putCustom(factory.type(), factory.fromXContent(parser)); } } } else if (token == XContentParser.Token.START_ARRAY) { @@ -450,7 +401,47 @@ public class IndexTemplateMetaData extends AbstractDiffable 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); + } } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java index 0959a4612c1..f2ace98caeb 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java @@ -19,11 +19,9 @@ package org.elasticsearch.cluster.metadata; -import com.google.common.collect.Maps; import org.elasticsearch.ElasticsearchIllegalStateException; 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; @@ -41,18 +39,14 @@ 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 extends AbstractDiffable { - - public static final MappingMetaData PROTO = new MappingMetaData(); +public class MappingMetaData { public static class Id { @@ -324,15 +318,6 @@ public class MappingMetaData extends AbstractDiffable { initMappers(withoutType); } - private MappingMetaData() { - this.type = ""; - try { - this.source = new CompressedString(""); - } catch (IOException ex) { - throw new ElasticsearchIllegalStateException("Cannot create MappingMetaData prototype", ex); - } - } - private void initMappers(Map withoutType) { if (withoutType.containsKey("_id")) { String path = null; @@ -548,35 +533,34 @@ public class MappingMetaData extends AbstractDiffable { } } - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(type()); - source().writeTo(out); + public static void writeTo(MappingMetaData mappingMd, StreamOutput out) throws IOException { + out.writeString(mappingMd.type()); + mappingMd.source().writeTo(out); // id - if (id().hasPath()) { + if (mappingMd.id().hasPath()) { out.writeBoolean(true); - out.writeString(id().path()); + out.writeString(mappingMd.id().path()); } else { out.writeBoolean(false); } // routing - out.writeBoolean(routing().required()); - if (routing().hasPath()) { + out.writeBoolean(mappingMd.routing().required()); + if (mappingMd.routing().hasPath()) { out.writeBoolean(true); - out.writeString(routing().path()); + out.writeString(mappingMd.routing().path()); } else { out.writeBoolean(false); } // timestamp - out.writeBoolean(timestamp().enabled()); - out.writeOptionalString(timestamp().path()); - out.writeString(timestamp().format()); - out.writeOptionalString(timestamp().defaultTimestamp()); + out.writeBoolean(mappingMd.timestamp().enabled()); + out.writeOptionalString(mappingMd.timestamp().path()); + out.writeString(mappingMd.timestamp().format()); + out.writeOptionalString(mappingMd.timestamp().defaultTimestamp()); // TODO Remove the test in elasticsearch 2.0.0 if (out.getVersion().onOrAfter(Version.V_1_5_0)) { - out.writeOptionalBoolean(timestamp().ignoreMissing()); + out.writeOptionalBoolean(mappingMd.timestamp().ignoreMissing()); } - out.writeBoolean(hasParentField()); + out.writeBoolean(mappingMd.hasParentField()); } @Override @@ -605,7 +589,7 @@ public class MappingMetaData extends AbstractDiffable { return result; } - public MappingMetaData readFrom(StreamInput in) throws IOException { + public static 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 4f20e1212cc..51793b1d27b 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -26,9 +26,7 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.base.Predicate; import com.google.common.collect.*; import org.elasticsearch.ElasticsearchIllegalArgumentException; -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; @@ -58,9 +56,7 @@ import static org.elasticsearch.common.settings.ImmutableSettings.*; /** * */ -public class MetaData implements Iterable, Diffable { - - public static final MetaData PROTO = builder().build(); +public class MetaData implements Iterable { public static final String ALL = "_all"; @@ -72,51 +68,60 @@ public class MetaData implements Iterable, Diffable { 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 extends Diffable, ToXContent { + public interface Custom { - String type(); + abstract class Factory { - Custom fromXContent(XContentParser parser) throws IOException; + public abstract String type(); - EnumSet context(); + 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; + } + } } - public static Map customPrototypes = new HashMap<>(); + public static Map customFactories = new HashMap<>(); static { // register non plugin custom metadata - registerPrototype(RepositoriesMetaData.TYPE, RepositoriesMetaData.PROTO); - registerPrototype(SnapshotMetaData.TYPE, SnapshotMetaData.PROTO); - registerPrototype(RestoreMetaData.TYPE, RestoreMetaData.PROTO); + registerFactory(RepositoriesMetaData.TYPE, RepositoriesMetaData.FACTORY); + registerFactory(SnapshotMetaData.TYPE, SnapshotMetaData.FACTORY); + registerFactory(RestoreMetaData.TYPE, RestoreMetaData.FACTORY); } /** * Register a custom index meta data factory. Make sure to call it from a static block. */ - public static void registerPrototype(String type, Custom proto) { - customPrototypes.put(type, proto); + public static void registerFactory(String type, Custom.Factory factory) { + customFactories.put(type, factory); } @Nullable - public static T lookupPrototype(String type) { - //noinspection unchecked - return (T) customPrototypes.get(type); + public static Custom.Factory lookupFactory(String type) { + return customFactories.get(type); } - public static T lookupPrototypeSafe(String type) throws ElasticsearchIllegalArgumentException { - //noinspection unchecked - T proto = (T) customPrototypes.get(type); - if (proto == null) { - throw new ElasticsearchIllegalArgumentException("No custom metadata prototype registered for type [" + type + "]"); + public static Custom.Factory lookupFactorySafe(String type) throws ElasticsearchIllegalArgumentException { + Custom.Factory factory = customFactories.get(type); + if (factory == null) { + throw new ElasticsearchIllegalArgumentException("No custom index metadata factory registered for type [" + type + "]"); } - return proto; + return factory; } @@ -640,14 +645,14 @@ public class MetaData implements Iterable, Diffable { /** * 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 - * @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 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. + * indices options don't allow such a case. */ public String[] concreteIndices(IndicesOptions indicesOptions, String... aliasesOrIndices) throws IndexMissingException, ElasticsearchIllegalArgumentException { if (indicesOptions.expandWildcardsOpen() || indicesOptions.expandWildcardsClosed()) { @@ -1135,14 +1140,14 @@ public class MetaData implements Iterable, Diffable { // Check if any persistent metadata needs to be saved int customCount1 = 0; for (ObjectObjectCursor cursor : metaData1.customs) { - if (customPrototypes.get(cursor.key).context().contains(XContentContext.GATEWAY)) { + if (customFactories.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 (customPrototypes.get(cursor.key).context().contains(XContentContext.GATEWAY)) { + if (customFactories.get(cursor.key).context().contains(XContentContext.GATEWAY)) { customCount2++; } } @@ -1150,129 +1155,6 @@ public class MetaData implements Iterable, Diffable { 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(); } @@ -1344,11 +1226,6 @@ public class MetaData implements Iterable, Diffable { return this; } - public Builder indices(ImmutableOpenMap indices) { - this.indices.putAll(indices); - return this; - } - public Builder put(IndexTemplateMetaData.Builder template) { return put(template.build()); } @@ -1363,11 +1240,6 @@ public class MetaData implements Iterable, Diffable { return this; } - public Builder templates(ImmutableOpenMap templates) { - this.templates.putAll(templates); - return this; - } - public Custom getCustom(String type) { return customs.get(type); } @@ -1382,11 +1254,6 @@ public class MetaData implements Iterable, Diffable { 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); @@ -1439,11 +1306,6 @@ public class MetaData implements Iterable, Diffable { return this; } - public Builder uuid(String uuid) { - this.uuid = uuid; - return this; - } - public Builder generateUuidIfNeeded() { if (uuid.equals("_na_")) { uuid = Strings.randomBase64UUID(); @@ -1502,10 +1364,10 @@ public class MetaData implements Iterable, Diffable { } for (ObjectObjectCursor cursor : metaData.customs()) { - Custom proto = lookupPrototypeSafe(cursor.key); - if (proto.context().contains(context)) { + Custom.Factory factory = lookupFactorySafe(cursor.key); + if (factory.context().contains(context)) { builder.startObject(cursor.key); - cursor.value.toXContent(builder, params); + factory.toXContent(cursor.value, builder, params); builder.endObject(); } } @@ -1549,13 +1411,12 @@ public class MetaData implements Iterable, Diffable { } } else { // check if its a custom index metadata - Custom proto = lookupPrototype(currentFieldName); - if (proto == null) { + Custom.Factory factory = lookupFactory(currentFieldName); + if (factory == null) { //TODO warn parser.skipChildren(); } else { - Custom custom = proto.fromXContent(parser); - builder.putCustom(custom.type(), custom); + builder.putCustom(factory.type(), factory.fromXContent(parser)); } } } else if (token.isValue()) { @@ -1570,7 +1431,46 @@ public class MetaData implements Iterable, Diffable { } public static MetaData readFrom(StreamInput in) throws IOException { - return PROTO.readFrom(in); + 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); + } } } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index f4077966485..244c598f0a3 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -273,7 +273,7 @@ public class MetaDataCreateIndexService extends AbstractComponent { if (existing == null) { customs.put(type, custom); } else { - IndexMetaData.Custom merged = existing.mergeWith(custom); + IndexMetaData.Custom merged = IndexMetaData.lookupFactorySafe(type).merge(existing, 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 51cd5db086b..81b11fc14b1 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java @@ -21,8 +21,6 @@ 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; @@ -41,11 +39,11 @@ import java.util.Map; /** * Contains metadata about registered snapshot repositories */ -public class RepositoriesMetaData extends AbstractDiffable implements MetaData.Custom { +public class RepositoriesMetaData implements MetaData.Custom { public static final String TYPE = "repositories"; - public static final RepositoriesMetaData PROTO = new RepositoriesMetaData(); + public static final Factory FACTORY = new Factory(); private final ImmutableList repositories; @@ -82,132 +80,122 @@ public class RepositoriesMetaData extends AbstractDiffable implements Me 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(); - } - /** - * {@inheritDoc} + * Repository metadata factory */ - @Override - public String type() { - return TYPE; - } + public static class Factory extends MetaData.Custom.Factory { - /** - * {@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); + /** + * {@inheritDoc} + */ + @Override + public String type() { + return TYPE; } - return new RepositoriesMetaData(repository); - } - /** - * {@inheritDoc} - */ - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeVInt(repositories.size()); - for (RepositoryMetaData repository : repositories) { - repository.writeTo(out); + /** + * {@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 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"); + /** + * {@inheritDoc} + */ + @Override + public void writeTo(RepositoriesMetaData repositories, StreamOutput out) throws IOException { + out.writeVInt(repositories.repositories().size()); + for (RepositoryMetaData repository : repositories.repositories()) { + repository.writeTo(out); } } - 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); + /** + * {@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()])); } - 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()); + /** + * {@inheritDoc} + */ + @Override + public void toXContent(RepositoriesMetaData customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException { + for (RepositoryMetaData repository : customIndexMetaData.repositories()) { + toXContent(repository, builder, params); + } } - builder.endObject(); - builder.endObject(); + @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(); + } } + } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java index a283f1f43c1..ea50b30ba88 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java @@ -99,25 +99,4 @@ 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 6dbde85c158..373d5ff858c 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/RestoreMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/RestoreMetaData.java @@ -22,7 +22,6 @@ package org.elasticsearch.cluster.metadata; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.elasticsearch.ElasticsearchIllegalArgumentException; -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; @@ -31,17 +30,16 @@ 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 extends AbstractDiffable implements MetaData.Custom { +public class RestoreMetaData implements MetaData.Custom { public static final String TYPE = "restore"; - public static final RestoreMetaData PROTO = new RestoreMetaData(); + public static final Factory FACTORY = new Factory(); private final ImmutableList entries; @@ -397,122 +395,124 @@ public class RestoreMetaData extends AbstractDiffable implement } /** - * {@inheritDoc} + * Restore metadata factory */ - @Override - public String type() { - return TYPE; - } + public static class Factory extends MetaData.Custom.Factory { - /** - * {@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 String type() { + return TYPE; } - return new RestoreMetaData(entries); - } - /** - * {@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); - } - out.writeVInt(entry.shards().size()); - for (Map.Entry shardEntry : entry.shards().entrySet()) { - shardEntry.getKey().writeTo(out); - shardEntry.getValue().writeTo(out); - } - } - } - - /** - * {@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()); + /** + * {@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()); + } + 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); } - builder.endObject(); } } - builder.endArray(); - builder.endObject(); + /** + * {@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); + } + 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); + } + } + 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 5010fcab5ac..b759fe5daeb 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/SnapshotMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/SnapshotMetaData.java @@ -22,8 +22,6 @@ package org.elasticsearch.cluster.metadata; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.elasticsearch.ElasticsearchIllegalArgumentException; -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; @@ -33,7 +31,6 @@ 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; @@ -41,10 +38,10 @@ import static com.google.common.collect.Maps.newHashMap; /** * Meta data about snapshots that are currently executing */ -public class SnapshotMetaData extends AbstractDiffable implements MetaData.Custom { +public class SnapshotMetaData implements MetaData.Custom { public static final String TYPE = "snapshots"; - public static final SnapshotMetaData PROTO = new SnapshotMetaData(); + public static final Factory FACTORY = new Factory(); @Override public boolean equals(Object o) { @@ -333,123 +330,123 @@ public class SnapshotMetaData extends AbstractDiffable implements MetaDa return null; } - @Override - public String type() { - return TYPE; - } - @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()); + 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. } - return new SnapshotMetaData(entries); - } - @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); - } - 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()); - } - } - } - - @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()); + @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()); + } + 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()); } - builder.endObject(); } } - builder.endArray(); - builder.endObject(); + + @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); + } + 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); + } + } + 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 61b5b876536..0a4986476e5 100644 --- a/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.UnmodifiableIterator; import org.elasticsearch.ElasticsearchIllegalArgumentException; 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; @@ -46,10 +45,9 @@ 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 extends AbstractDiffable implements Iterable { +public class DiscoveryNodes 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; @@ -570,44 +568,6 @@ public class DiscoveryNodes extends AbstractDiffable implements } } - 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(); } @@ -672,8 +632,37 @@ public class DiscoveryNodes extends AbstractDiffable implements 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 { - return PROTO.readFrom(in, localNode); + 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(); } } } diff --git a/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java index 239f5113781..5f0356d3572 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java @@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import com.google.common.collect.UnmodifiableIterator; import org.elasticsearch.ElasticsearchIllegalStateException; -import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.collect.ImmutableOpenIntMap; @@ -57,9 +56,7 @@ import static com.google.common.collect.Lists.newArrayList; * represented as {@link ShardRouting}. *

*/ -public class IndexRoutingTable extends AbstractDiffable implements Iterable { - - public static final IndexRoutingTable PROTO = builder("").build(); +public class IndexRoutingTable implements Iterable { private final String index; private final ShardShuffler shuffler; @@ -318,51 +315,9 @@ public class IndexRoutingTable extends AbstractDiffable imple 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); } @@ -384,7 +339,30 @@ public class IndexRoutingTable extends AbstractDiffable imple * @throws IOException if something happens during read */ public static IndexRoutingTable readFrom(StreamInput in) throws IOException { - return PROTO.readFrom(in); + 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); + } } /** diff --git a/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java index 2371b96f5b0..00e50b76129 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java @@ -347,28 +347,6 @@ 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 25a8bac2f88..9f1b5db6c6b 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.*; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.io.stream.StreamInput; @@ -44,9 +44,7 @@ import static com.google.common.collect.Maps.newHashMap; * * @see IndexRoutingTable */ -public class RoutingTable implements Iterable, Diffable { - - public static RoutingTable PROTO = builder().build(); +public class RoutingTable implements Iterable { public static final RoutingTable EMPTY_ROUTING_TABLE = builder().build(); @@ -256,66 +254,6 @@ public class RoutingTable implements Iterable, Diffable 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(); } @@ -465,11 +403,6 @@ public class RoutingTable implements Iterable, Diffable indicesRouting) { - this.indicesRouting.putAll(indicesRouting); - return this; - } - public Builder remove(String index) { indicesRouting.remove(index); return this; @@ -489,7 +422,23 @@ public class RoutingTable implements Iterable, Diffable, Diffable { - /** - * 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 deleted file mode 100644 index 9025315dc43..00000000000 --- a/src/main/java/org/elasticsearch/common/io/stream/Writeable.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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 36b8e5da6f5..dfd51e6348f 100644 --- a/src/main/java/org/elasticsearch/discovery/Discovery.java +++ b/src/main/java/org/elasticsearch/discovery/Discovery.java @@ -19,7 +19,6 @@ 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; @@ -60,7 +59,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(ClusterChangedEvent clusterChangedEvent, AckListener ackListener); + void publish(ClusterState clusterState, 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 e6a3668921b..f73f2bbb593 100644 --- a/src/main/java/org/elasticsearch/discovery/DiscoveryService.java +++ b/src/main/java/org/elasticsearch/discovery/DiscoveryService.java @@ -21,7 +21,6 @@ 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; @@ -133,9 +132,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) { @@ -280,7 +274,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem } @Override - public void publish(ClusterChangedEvent clusterChangedEvent, final Discovery.AckListener ackListener) { + public void publish(ClusterState clusterState, final Discovery.AckListener ackListener) { if (!master) { throw new ElasticsearchIllegalStateException("Shouldn't publish state when not master"); } @@ -293,7 +287,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem } nodesToPublishTo.add(localDiscovery.localNode); } - publish(members, clusterChangedEvent, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); + publish(members, clusterState, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); } } @@ -306,47 +300,17 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem return members.toArray(new LocalDiscovery[members.size()]); } - private void publish(LocalDiscovery[] members, ClusterChangedEvent clusterChangedEvent, final BlockingClusterStatePublishResponseHandler publishResponseHandler) { + private void publish(LocalDiscovery[] members, ClusterState clusterState, final BlockingClusterStatePublishResponseHandler publishResponseHandler) { try { // we do the marshaling intentionally, to check it works well... - byte[] clusterStateBytes = null; - byte[] clusterStateDiffBytes = null; + final byte[] clusterStateBytes = Builder.toBytes(clusterState); - ClusterState clusterState = clusterChangedEvent.state(); for (final LocalDiscovery discovery : members) { if (discovery.master) { continue; } - 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; - + final ClusterState nodeSpecificClusterState = ClusterState.Builder.fromBytes(clusterStateBytes, discovery.localNode); 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 8a6382dcf94..9ad4945844a 100644 --- a/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -331,12 +331,12 @@ public class ZenDiscovery extends AbstractLifecycleComponent implemen @Override - public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) { - if (!clusterChangedEvent.state().getNodes().localNodeMaster()) { + public void publish(ClusterState clusterState, AckListener ackListener) { + if (!clusterState.getNodes().localNodeMaster()) { throw new ElasticsearchIllegalStateException("Shouldn't publish state when not master"); } - nodesFD.updateNodesAndPing(clusterChangedEvent.state()); - publishClusterState.publish(clusterChangedEvent, ackListener); + nodesFD.updateNodesAndPing(clusterState); + publishClusterState.publish(clusterState, 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 c4ad8895e79..fd1ba85c25c 100644 --- a/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java +++ b/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java @@ -21,12 +21,8 @@ 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; @@ -44,13 +40,10 @@ 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; /** * @@ -90,43 +83,73 @@ public class PublishClusterStateAction extends AbstractComponent { transportService.removeHandler(ACTION_NAME); } - public void publish(ClusterChangedEvent clusterChangedEvent, final Discovery.AckListener ackListener) { - Set nodesToPublishTo = new HashSet<>(clusterChangedEvent.state().nodes().size()); + public void publish(ClusterState clusterState, final Discovery.AckListener ackListener) { + Set nodesToPublishTo = new HashSet<>(clusterState.nodes().size()); DiscoveryNode localNode = nodesProvider.nodes().localNode(); - for (final DiscoveryNode node : clusterChangedEvent.state().nodes()) { + for (final DiscoveryNode node : clusterState.nodes()) { if (node.equals(localNode)) { continue; } nodesToPublishTo.add(node); } - publish(clusterChangedEvent, nodesToPublishTo, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); + publish(clusterState, nodesToPublishTo, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); } - private void publish(final ClusterChangedEvent clusterChangedEvent, final Set nodesToPublishTo, + private void publish(final ClusterState clusterState, 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... - // 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); + 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; } - sendClusterStateDiff(clusterState, diff, serializedDiffs, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler); + } + 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); } } @@ -148,107 +171,7 @@ 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 { @@ -260,24 +183,11 @@ public class PublishClusterStateAction extends AbstractComponent { in = request.bytes().streamInput(); } in.setVersion(request.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); - } - + ClusterState clusterState = ClusterState.Builder.readFrom(in, nodesProvider.nodes().localNode()); + clusterState.status(ClusterState.ClusterStateStatus.RECEIVED); + logger.debug("received cluster state version {}", clusterState.version()); try { - listener.onNewClusterState(lastSeenClusterState, new NewClusterStateListener.NewStateProcessed() { + listener.onNewClusterState(clusterState, new NewClusterStateListener.NewStateProcessed() { @Override public void onNewClusterStateProcessed() { try { @@ -297,7 +207,7 @@ public class PublishClusterStateAction extends AbstractComponent { } }); } catch (Exception e) { - logger.warn("unexpected error while processing cluster state version [{}]", e, lastSeenClusterState.version()); + logger.warn("unexpected error while processing cluster state version [{}]", e, clusterState.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 139b5763489..cd15bccdc4a 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 5538ef6d043..43dec7edb51 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.writeTo(out); + IndexMetaData.Builder.writeTo(indexMetaData, out); } } } diff --git a/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java b/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java index d7334046de0..77ab900ce90 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.writeTo(out); + MetaData.Builder.writeTo(metaData, 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 85b46925b5f..be4e1b4e3f3 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.toXContent(repositoryMetaData, builder, request); + RepositoriesMetaData.FACTORY.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 d2653bc1745..dc800f37062 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 @@ -147,7 +147,7 @@ public class RestGetIndicesAction extends BaseRestHandler { builder.startObject(Fields.WARMERS); if (warmers != null) { for (IndexWarmersMetaData.Entry warmer : warmers) { - IndexWarmersMetaData.toXContent(warmer, builder, params); + IndexWarmersMetaData.FACTORY.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 be83ccbe4b5..7023eecedd4 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.toXContent(warmerEntry, builder, request); + IndexWarmersMetaData.FACTORY.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 ef1ef44ffb9..de56f823eac 100644 --- a/src/main/java/org/elasticsearch/search/warmer/IndexWarmersMetaData.java +++ b/src/main/java/org/elasticsearch/search/warmer/IndexWarmersMetaData.java @@ -22,9 +22,7 @@ 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; @@ -35,33 +33,16 @@ 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 extends AbstractDiffable implements IndexMetaData.Custom { +public class IndexWarmersMetaData implements IndexMetaData.Custom { public static final String TYPE = "warmers"; - 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 final Factory FACTORY = new Factory(); public static class Entry { private final String name; @@ -93,29 +74,6 @@ public class IndexWarmersMetaData extends AbstractDiffable 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; @@ -134,143 +92,149 @@ public class IndexWarmersMetaData extends AbstractDiffable 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); - } + public static class Factory implements IndexMetaData.Custom.Factory { - @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 { - out.writeBoolean(true); - out.writeBytesReference(entry.source()); - } - out.writeOptionalBoolean(entry.queryCache()); + @Override + public String type() { + return TYPE; } - } - @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); + @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; - 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()); + 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(); } } - } 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()); + } else { + Map mapping = XContentFactory.xContent(entry.source()).createParser(entry.source()).mapOrderedAndClose(); + builder.map(mapping); + } + 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; } } - 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); } } - if (!found) { - entries.add(secondEntry); - } + return new IndexWarmersMetaData(entries.toArray(new Entry[entries.size()])); } - 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 deleted file mode 100644 index 33008fd63d2..00000000000 --- a/src/test/java/org/elasticsearch/cluster/ClusterStateDiffPublishingTests.java +++ /dev/null @@ -1,625 +0,0 @@ -/* - * 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 deleted file mode 100644 index 84df1eaf209..00000000000 --- a/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java +++ /dev/null @@ -1,534 +0,0 @@ -/* - * 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 83a27850591..cbbff463f20 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(); - source.writeTo(outStream); + RoutingTable.Builder.writeTo(source, 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 deleted file mode 100644 index d87d900a0e8..00000000000 --- a/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * 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 deleted file mode 100644 index 9ebffe58783..00000000000 --- a/src/test/java/org/elasticsearch/common/xcontent/XContentTestUtils.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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 0850cd5e095..430690ae146 100644 --- a/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryTests.java +++ b/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryTests.java @@ -28,7 +28,6 @@ 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 228faa8cf4d..58e177b1115 100644 --- a/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryTests.java +++ b/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryTests.java @@ -32,6 +32,9 @@ 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; @@ -193,7 +196,12 @@ 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); - BytesReference bytes = PublishClusterStateAction.serializeFullClusterState(builder.build(), node.version()); + 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(); 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 c97fa5b789d..c5adf8cb50e 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(); - expected.writeTo(out); + MappingMetaData.writeTo(expected, out); out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.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(); - expected.writeTo(out); + MappingMetaData.writeTo(expected, out); out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.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(); - expected.writeTo(out); + MappingMetaData.writeTo(expected, out); out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.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 3cc8a0cfe20..e1efe59776d 100644 --- a/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationTests.java +++ b/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationTests.java @@ -41,7 +41,6 @@ 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; @@ -62,12 +61,6 @@ public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest { return ImmutableSettings.settingsBuilder().put(super.nodeSettings(nodeOrdinal)).put("path.data", "").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 230c6cae3ce..1c3f8f8c9ca 100644 --- a/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateTests.java +++ b/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateTests.java @@ -285,7 +285,6 @@ 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 8d569275aea..ff8264fdc03 100644 --- a/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java +++ b/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java @@ -38,9 +38,7 @@ 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; @@ -750,7 +748,7 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests )); } - public static abstract class TestCustomMetaData extends AbstractDiffable implements MetaData.Custom { + public static abstract class TestCustomMetaData implements MetaData.Custom { private final String data; protected TestCustomMetaData(String data) { @@ -778,182 +776,194 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests return data.hashCode(); } - protected abstract TestCustomMetaData newTestCustomMetaData(String data); + public static abstract class TestCustomMetaDataFactory extends MetaData.Custom.Factory { - @Override - public Custom readFrom(StreamInput in) throws IOException { - return newTestCustomMetaData(in.readString()); - } + protected abstract TestCustomMetaData newTestCustomMetaData(String data); - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(getData()); - } + @Override + public T readFrom(StreamInput in) throws IOException { + return (T) newTestCustomMetaData(in.readString()); + } - @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"); + @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 + "]"); } - data = parser.text(); } else { - throw new ElasticsearchParseException("failed to parse snapshottable metadata, unknown field [" + currentFieldName + "]"); + throw new ElasticsearchParseException("failed to parse snapshottable metadata"); } - } 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 XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.field("data", getData()); - return builder; + @Override + public void toXContent(T metadata, XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.field("data", metadata.getData()); + } } } - static { - 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); + 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); } public static class SnapshottableMetadata extends TestCustomMetaData { public static final String TYPE = "test_snapshottable"; - public static final SnapshottableMetadata PROTO = new SnapshottableMetadata(""); + public static final Factory FACTORY = new Factory(); public SnapshottableMetadata(String data) { super(data); } - @Override - public String type() { - return TYPE; - } + private static class Factory extends TestCustomMetaDataFactory { - @Override - protected TestCustomMetaData newTestCustomMetaData(String data) { - return new SnapshottableMetadata(data); - } + @Override + public String type() { + return TYPE; + } - @Override - public EnumSet context() { - return MetaData.API_AND_SNAPSHOT; + @Override + protected TestCustomMetaData newTestCustomMetaData(String data) { + return new SnapshottableMetadata(data); + } + + @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 NonSnapshottableMetadata PROTO = new NonSnapshottableMetadata(""); + public static final Factory FACTORY = new Factory(); public NonSnapshottableMetadata(String data) { super(data); } - @Override - public String type() { - return TYPE; - } + private static class Factory extends TestCustomMetaDataFactory { - @Override - protected NonSnapshottableMetadata newTestCustomMetaData(String data) { - return new NonSnapshottableMetadata(data); - } + @Override + public String type() { + return TYPE; + } - @Override - public EnumSet context() { - return MetaData.API_ONLY; + @Override + protected NonSnapshottableMetadata newTestCustomMetaData(String data) { + return new NonSnapshottableMetadata(data); + } } } public static class SnapshottableGatewayMetadata extends TestCustomMetaData { public static final String TYPE = "test_snapshottable_gateway"; - public static final SnapshottableGatewayMetadata PROTO = new SnapshottableGatewayMetadata(""); + public static final Factory FACTORY = new Factory(); public SnapshottableGatewayMetadata(String data) { super(data); } - @Override - public String type() { - return TYPE; - } + private static class Factory extends TestCustomMetaDataFactory { - @Override - protected TestCustomMetaData newTestCustomMetaData(String data) { - return new SnapshottableGatewayMetadata(data); - } + @Override + public String type() { + return TYPE; + } - @Override - public EnumSet context() { - return EnumSet.of(MetaData.XContentContext.API, MetaData.XContentContext.SNAPSHOT, MetaData.XContentContext.GATEWAY); + @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); + } } } public static class NonSnapshottableGatewayMetadata extends TestCustomMetaData { public static final String TYPE = "test_non_snapshottable_gateway"; - public static final NonSnapshottableGatewayMetadata PROTO = new NonSnapshottableGatewayMetadata(""); + public static final Factory FACTORY = new Factory(); public NonSnapshottableGatewayMetadata(String data) { super(data); } - @Override - public String type() { - return TYPE; - } + private static class Factory extends TestCustomMetaDataFactory { - @Override - protected NonSnapshottableGatewayMetadata newTestCustomMetaData(String data) { - return new NonSnapshottableGatewayMetadata(data); - } + @Override + public String type() { + return TYPE; + } - @Override - public EnumSet context() { - return MetaData.API_AND_GATEWAY; - } + @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 SnapshotableGatewayNoApiMetadata PROTO = new SnapshotableGatewayNoApiMetadata(""); + public static final Factory FACTORY = new Factory(); public SnapshotableGatewayNoApiMetadata(String data) { super(data); } - @Override - public String type() { - return TYPE; - } + private static class Factory extends TestCustomMetaDataFactory { - @Override - protected SnapshotableGatewayNoApiMetadata newTestCustomMetaData(String data) { - return new SnapshotableGatewayNoApiMetadata(data); - } + @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 - 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 be318020fc0..1c0c11bb5cd 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java @@ -166,8 +166,6 @@ 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; @@ -359,7 +357,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 */ - public void randomIndexTemplate() throws IOException { + private void randomIndexTemplate() throws IOException { // TODO move settings for random directory etc here into the index based randomized settings. if (cluster().size() > 0) { @@ -652,7 +650,6 @@ 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(); @@ -1091,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()); } /** @@ -1139,35 +1136,6 @@ 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 8c7ae1955f3..55b4b15af01 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java @@ -71,7 +71,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import static com.google.common.collect.Lists.newArrayList; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllFilesClosed; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllSearchersClosed; @@ -618,17 +617,4 @@ 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); - } - }