Remove PROTO-based custom cluster state components

Switches custom cluster state components from PROTO-based de-serialization to named objects based de-serialization
This commit is contained in:
Igor Motov 2016-12-27 22:31:34 -05:00
parent dea8cee70f
commit ca90d9ea82
115 changed files with 1090 additions and 751 deletions

View File

@ -59,7 +59,7 @@ public class ClusterRerouteResponse extends AcknowledgedResponse {
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
state = ClusterState.Builder.readFrom(in, null);
state = ClusterState.readFrom(in, null);
readAcknowledged(in);
explanations = RoutingExplanations.readFrom(in);
}

View File

@ -52,7 +52,7 @@ public class ClusterStateResponse extends ActionResponse {
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
clusterName = new ClusterName(in);
clusterState = ClusterState.Builder.readFrom(in, null);
clusterState = ClusterState.readFrom(in, null);
}
@Override

View File

@ -57,7 +57,7 @@ public class GetAliasesResponse extends ActionResponse {
int valueSize = in.readVInt();
List<AliasMetaData> value = new ArrayList<>(valueSize);
for (int j = 0; j < valueSize; j++) {
value.add(AliasMetaData.Builder.readFrom(in));
value.add(new AliasMetaData(in));
}
aliasesBuilder.put(key, Collections.unmodifiableList(value));
}

View File

@ -104,7 +104,7 @@ public class GetIndexResponse extends ActionResponse {
int valueSize = in.readVInt();
ImmutableOpenMap.Builder<String, MappingMetaData> mappingEntryBuilder = ImmutableOpenMap.builder();
for (int j = 0; j < valueSize; j++) {
mappingEntryBuilder.put(in.readString(), MappingMetaData.PROTO.readFrom(in));
mappingEntryBuilder.put(in.readString(), new MappingMetaData(in));
}
mappingsMapBuilder.put(key, mappingEntryBuilder.build());
}
@ -116,7 +116,7 @@ public class GetIndexResponse extends ActionResponse {
int valueSize = in.readVInt();
List<AliasMetaData> aliasEntryBuilder = new ArrayList<>();
for (int j = 0; j < valueSize; j++) {
aliasEntryBuilder.add(AliasMetaData.Builder.readFrom(in));
aliasEntryBuilder.add(new AliasMetaData(in));
}
aliasesMapBuilder.put(key, Collections.unmodifiableList(aliasEntryBuilder));
}

View File

@ -57,7 +57,7 @@ public class GetMappingsResponse extends ActionResponse {
int valueSize = in.readVInt();
ImmutableOpenMap.Builder<String, MappingMetaData> typeMapBuilder = ImmutableOpenMap.builder();
for (int j = 0; j < valueSize; j++) {
typeMapBuilder.put(in.readString(), MappingMetaData.PROTO.readFrom(in));
typeMapBuilder.put(in.readString(), new MappingMetaData(in));
}
indexMapBuilder.put(key, typeMapBuilder.build());
}

View File

@ -52,7 +52,7 @@ public class GetIndexTemplatesResponse extends ActionResponse implements ToXCont
int size = in.readVInt();
indexTemplates = new ArrayList<>(size);
for (int i = 0 ; i < size ; i++) {
indexTemplates.add(0, IndexTemplateMetaData.Builder.readFrom(in));
indexTemplates.add(0, IndexTemplateMetaData.readFrom(in));
}
}

View File

@ -52,7 +52,7 @@ public class GetPipelineResponse extends ActionResponse implements StatusToXCont
int size = in.readVInt();
pipelines = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
pipelines.add(PipelineConfiguration.readPipelineConfiguration(in));
pipelines.add(PipelineConfiguration.readFrom(in));
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.client.support.AbstractClient;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.inject.Injector;
@ -140,6 +141,7 @@ public abstract class TransportClient extends AbstractClient {
List<NamedWriteableRegistry.Entry> entries = new ArrayList<>();
entries.addAll(NetworkModule.getNamedWriteables());
entries.addAll(searchModule.getNamedWriteables());
entries.addAll(ClusterModule.getNamedWriteables());
entries.addAll(pluginsService.filterPlugins(Plugin.class).stream()
.flatMap(p -> p.getNamedWriteables().stream())
.collect(Collectors.toList()));

View File

@ -40,12 +40,7 @@ public abstract class AbstractDiffable<T extends Diffable<T>> implements Diffabl
}
}
@Override
public Diff<T> readDiffFrom(StreamInput in) throws IOException {
return new CompleteDiff<>(this, in);
}
public static <T extends Diffable<T>> Diff<T> readDiffFrom(T reader, StreamInput in) throws IOException {
public static <T extends Diffable<T>> Diff<T> readDiffFrom(Reader<T> reader, StreamInput in) throws IOException {
return new CompleteDiff<T>(reader, in);
}
@ -71,9 +66,9 @@ public abstract class AbstractDiffable<T extends Diffable<T>> implements Diffabl
/**
* Read simple diff from the stream
*/
public CompleteDiff(Diffable<T> reader, StreamInput in) throws IOException {
public CompleteDiff(Reader<T> reader, StreamInput in) throws IOException {
if (in.readBoolean()) {
this.part = reader.readFrom(in);
this.part = reader.read(in);
} else {
this.part = null;
}

View File

@ -0,0 +1,114 @@
/*
* 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.NamedWriteable;
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. Comparing to AbstractDiffable, this class also works with NamedWriteables
*/
public abstract class AbstractNamedDiffable<T extends Diffable<T> & NamedWriteable> implements Diffable<T>, NamedWriteable {
@Override
public Diff<T> diff(T previousState) {
if (this.get().equals(previousState)) {
return new CompleteNamedDiff<>(previousState.getWriteableName());
} else {
return new CompleteNamedDiff<>(get());
}
}
public static <T extends Diffable<T> & NamedWriteable> NamedDiff<T> readDiffFrom(Class<? extends T> tClass, String name, StreamInput in)
throws IOException {
return new CompleteNamedDiff<>(tClass, name, in);
}
private static class CompleteNamedDiff<T extends Diffable<T> & NamedWriteable> implements NamedDiff<T> {
@Nullable
private final T part;
private final String name;
/**
* Creates simple diff with changes
*/
public CompleteNamedDiff(T part) {
this.part = part;
this.name = part.getWriteableName();
}
/**
* Creates simple diff without changes
*/
public CompleteNamedDiff(String name) {
this.part = null;
this.name = name;
}
/**
* Read simple diff from the stream
*/
public CompleteNamedDiff(Class<? extends T> tClass, String name, StreamInput in) throws IOException {
if (in.readBoolean()) {
this.part = in.readNamedWriteable(tClass, name);
} else {
this.part = null;
}
this.name = name;
}
@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;
}
}
@Override
public String getWriteableName() {
return name;
}
}
@SuppressWarnings("unchecked")
public T get() {
return (T) this;
}
}

View File

@ -22,7 +22,9 @@ package org.elasticsearch.cluster;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.action.index.NodeMappingRefreshAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.IndexGraveyard;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService;
import org.elasticsearch.cluster.metadata.MetaDataDeleteIndexService;
import org.elasticsearch.cluster.metadata.MetaDataIndexAliasesService;
@ -30,6 +32,7 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
import org.elasticsearch.cluster.metadata.MetaDataIndexTemplateService;
import org.elasticsearch.cluster.metadata.MetaDataMappingService;
import org.elasticsearch.cluster.metadata.MetaDataUpdateSettingsService;
import org.elasticsearch.cluster.metadata.RepositoriesMetaData;
import org.elasticsearch.cluster.routing.DelayedAllocationService;
import org.elasticsearch.cluster.routing.RoutingService;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
@ -52,15 +55,25 @@ import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocatio
import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry.Entry;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.gateway.GatewayAllocator;
import org.elasticsearch.ingest.IngestMetadata;
import org.elasticsearch.plugins.ClusterPlugin;
import org.elasticsearch.script.ScriptMetaData;
import org.elasticsearch.tasks.TaskResultsService;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.LinkedHashMap;
@ -94,6 +107,52 @@ public class ClusterModule extends AbstractModule {
indexNameExpressionResolver = new IndexNameExpressionResolver(settings);
}
public static List<Entry> getNamedWriteables() {
List<Entry> entries = new ArrayList<>();
// Cluster State
registerClusterCustom(entries, SnapshotsInProgress.TYPE, SnapshotsInProgress::new, SnapshotsInProgress::readDiffFrom);
registerClusterCustom(entries, RestoreInProgress.TYPE, RestoreInProgress::new, RestoreInProgress::readDiffFrom);
registerClusterCustom(entries, SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress::new,
SnapshotDeletionsInProgress::readDiffFrom);
// Metadata
registerMetaDataCustom(entries, RepositoriesMetaData.TYPE, RepositoriesMetaData::new, RepositoriesMetaData::readDiffFrom);
registerMetaDataCustom(entries, IngestMetadata.TYPE, IngestMetadata::new, IngestMetadata::readDiffFrom);
registerMetaDataCustom(entries, ScriptMetaData.TYPE, ScriptMetaData::new, ScriptMetaData::readDiffFrom);
registerMetaDataCustom(entries, IndexGraveyard.TYPE, IndexGraveyard::new, IndexGraveyard::readDiffFrom);
return entries;
}
public static List<NamedXContentRegistry.Entry> getNamedXWriteables() {
List<NamedXContentRegistry.Entry> entries = new ArrayList<>();
// Metadata
entries.add(new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField(RepositoriesMetaData.TYPE),
RepositoriesMetaData::fromXContent));
entries.add(new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField(IngestMetadata.TYPE),
IngestMetadata::fromXContent));
entries.add(new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField(ScriptMetaData.TYPE),
ScriptMetaData::fromXContent));
entries.add(new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField(IndexGraveyard.TYPE),
IndexGraveyard::fromXContent));
return entries;
}
private static <T extends ClusterState.Custom> void registerClusterCustom(List<Entry> entries, String name, Reader<? extends T> reader,
Reader<NamedDiff> diffReader) {
registerCustom(entries, ClusterState.Custom.class, name, reader, diffReader);
}
private static <T extends MetaData.Custom> void registerMetaDataCustom(List<Entry> entries, String name, Reader<? extends T> reader,
Reader<NamedDiff> diffReader) {
registerCustom(entries, MetaData.Custom.class, name, reader, diffReader);
}
private static <T extends NamedWriteable> void registerCustom(List<Entry> entries, Class<T> category, String name,
Reader<? extends T> reader, Reader<NamedDiff> diffReader) {
entries.add(new Entry(category, name, reader));
entries.add(new Entry(NamedDiff.class, name, diffReader));
}
public IndexNameExpressionResolver getIndexNameExpressionResolver() {
return indexNameExpressionResolver;
}

View File

@ -43,8 +43,12 @@ import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
@ -86,36 +90,10 @@ import java.util.Set;
*/
public class ClusterState implements ToXContent, Diffable<ClusterState> {
public static final ClusterState PROTO = builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).build();
public static final ClusterState EMPTY_STATE = builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).build();
public interface Custom extends Diffable<Custom>, ToXContent {
public interface Custom extends Diffable<Custom>, ToXContent, NamedWriteable {
String type();
}
private static final Map<String, Custom> customPrototypes = 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);
}
static {
// register non plugin custom parts
registerPrototype(SnapshotsInProgress.TYPE, SnapshotsInProgress.PROTO);
registerPrototype(RestoreInProgress.TYPE, RestoreInProgress.PROTO);
registerPrototype(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.PROTO);
}
public static <T extends Custom> T lookupPrototype(String type) {
@SuppressWarnings("unchecked")
T proto = (T) customPrototypes.get(type);
if (proto == null) {
throw new IllegalArgumentException("No custom state prototype registered for type [" + type + "], node likely missing plugins");
}
return proto;
}
public static final String UNKNOWN_UUID = "_na_";
@ -659,53 +637,39 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
* @param data input bytes
* @param localNode used to set the local node in the cluster state.
*/
public static ClusterState fromBytes(byte[] data, DiscoveryNode localNode) throws IOException {
return readFrom(StreamInput.wrap(data), localNode);
public static ClusterState fromBytes(byte[] data, DiscoveryNode localNode, NamedWriteableRegistry registry) throws IOException {
StreamInput in = new NamedWriteableAwareStreamInput(StreamInput.wrap(data), registry);
return readFrom(in, localNode);
}
/**
* @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) {
public Diff<ClusterState> diff(ClusterState previousState) {
return new ClusterStateDiff(previousState, this);
}
@Override
public Diff<ClusterState> readDiffFrom(StreamInput in) throws IOException {
return new ClusterStateDiff(in, this);
public static Diff<ClusterState> readDiffFrom(StreamInput in, DiscoveryNode localNode) throws IOException {
return new ClusterStateDiff(in, localNode);
}
public ClusterState readFrom(StreamInput in, DiscoveryNode localNode) throws IOException {
public static ClusterState readFrom(StreamInput in, DiscoveryNode localNode) throws IOException {
ClusterName clusterName = new ClusterName(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);
builder.metaData = MetaData.readFrom(in);
builder.routingTable = RoutingTable.readFrom(in);
builder.nodes = DiscoveryNodes.readFrom(in, localNode);
builder.blocks = new ClusterBlocks(in);
int customSize = in.readVInt();
for (int i = 0; i < customSize; i++) {
String type = in.readString();
Custom customIndexMetaData = lookupPrototype(type).readFrom(in);
builder.putCustom(type, customIndexMetaData);
Custom customIndexMetaData = in.readNamedWriteable(Custom.class);
builder.putCustom(customIndexMetaData.getWriteableName(), customIndexMetaData);
}
return builder.build();
}
@Override
public ClusterState readFrom(StreamInput in) throws IOException {
return readFrom(in, nodes.getLocalNode());
}
@Override
public void writeTo(StreamOutput out) throws IOException {
clusterName.writeTo(out);
@ -727,8 +691,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
if (omitSnapshotDeletions && cursor.key.equals(SnapshotDeletionsInProgress.TYPE)) {
continue;
}
out.writeString(cursor.key);
cursor.value.writeTo(out);
out.writeNamedWriteable(cursor.value);
}
}
@ -764,25 +727,26 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
customs = DiffableUtils.diff(before.customs, after.customs, DiffableUtils.getStringKeySerializer());
}
public ClusterStateDiff(StreamInput in, ClusterState proto) throws IOException {
public ClusterStateDiff(StreamInput in, DiscoveryNode localNode) throws IOException {
clusterName = new ClusterName(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);
routingTable = RoutingTable.readDiffFrom(in);
nodes = DiscoveryNodes.readDiffFrom(in, localNode);
metaData = MetaData.readDiffFrom(in);
blocks = ClusterBlocks.readDiffFrom(in);
customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(),
new DiffableUtils.DiffableValueSerializer<String, Custom>() {
@Override
public Custom read(StreamInput in, String key) throws IOException {
return lookupPrototype(key).readFrom(in);
return in.readNamedWriteable(Custom.class, key);
}
@SuppressWarnings("unchecked")
@Override
public Diff<Custom> readDiff(StreamInput in, String key) throws IOException {
return lookupPrototype(key).readDiffFrom(in);
return in.readNamedWriteable(NamedDiff.class, key);
}
});
}

View File

@ -34,13 +34,4 @@ public interface Diffable<T> extends Writeable {
*/
Diff<T> diff(T previousState);
/**
* Reads the {@link org.elasticsearch.cluster.Diff} from StreamInput
*/
Diff<T> readDiffFrom(StreamInput in) throws IOException;
/**
* Reads an object of this type from the provided {@linkplain StreamInput}. The receiving instance remains unchanged.
*/
T readFrom(StreamInput in) throws IOException;
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.collect.ImmutableOpenIntMap;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import java.io.IOException;
import java.util.ArrayList;
@ -135,22 +136,22 @@ public final class DiffableUtils {
/**
* Loads an object that represents difference between two ImmutableOpenMaps of Diffable objects using Diffable proto object
*/
public static <K, T extends Diffable<T>> MapDiff<K, T, ImmutableOpenMap<K, T>> readImmutableOpenMapDiff(StreamInput in, KeySerializer<K> keySerializer, T proto) throws IOException {
return new ImmutableOpenMapDiff<>(in, keySerializer, new DiffablePrototypeValueReader<>(proto));
public static <K, T extends Diffable<T>> MapDiff<K, T, ImmutableOpenMap<K, T>> readImmutableOpenMapDiff(StreamInput in, KeySerializer<K> keySerializer, Reader<T> reader, Reader<Diff<T>> diffReader) throws IOException {
return new ImmutableOpenMapDiff<>(in, keySerializer, new DiffableValueReader<>(reader, diffReader));
}
/**
* Loads an object that represents difference between two ImmutableOpenIntMaps of Diffable objects using Diffable proto object
*/
public static <T extends Diffable<T>> MapDiff<Integer, T, ImmutableOpenIntMap<T>> readImmutableOpenIntMapDiff(StreamInput in, KeySerializer<Integer> keySerializer, T proto) throws IOException {
return new ImmutableOpenIntMapDiff<>(in, keySerializer, new DiffablePrototypeValueReader<>(proto));
public static <T extends Diffable<T>> MapDiff<Integer, T, ImmutableOpenIntMap<T>> readImmutableOpenIntMapDiff(StreamInput in, KeySerializer<Integer> keySerializer, Reader<T> reader, Reader<Diff<T>> diffReader) throws IOException {
return new ImmutableOpenIntMapDiff<>(in, keySerializer, new DiffableValueReader<>(reader, diffReader));
}
/**
* Loads an object that represents difference between two Maps of Diffable objects using Diffable proto object
*/
public static <K, T extends Diffable<T>> MapDiff<K, T, Map<K, T>> readJdkMapDiff(StreamInput in, KeySerializer<K> keySerializer, T proto) throws IOException {
return new JdkMapDiff<>(in, keySerializer, new DiffablePrototypeValueReader<>(proto));
public static <K, T extends Diffable<T>> MapDiff<K, T, Map<K, T>> readJdkMapDiff(StreamInput in, KeySerializer<K> keySerializer, Reader<T> reader, Reader<Diff<T>> diffReader) throws IOException {
return new JdkMapDiff<>(in, keySerializer, new DiffableValueReader<>(reader, diffReader));
}
/**
@ -629,25 +630,27 @@ public final class DiffableUtils {
}
/**
* Implementation of the ValueSerializer that uses a prototype object for reading operations
* Implementation of the ValueSerializer that wraps value and diff readers.
*
* Note: this implementation is ignoring the key.
*/
public static class DiffablePrototypeValueReader<K, V extends Diffable<V>> extends DiffableValueSerializer<K, V> {
private final V proto;
public static class DiffableValueReader<K, V extends Diffable<V>> extends DiffableValueSerializer<K, V> {
private final Reader<V> reader;
private final Reader<Diff<V>> diffReader;
public DiffablePrototypeValueReader(V proto) {
this.proto = proto;
public DiffableValueReader(Reader<V> reader, Reader<Diff<V>> diffReader) {
this.reader = reader;
this.diffReader = diffReader;
}
@Override
public V read(StreamInput in, K key) throws IOException {
return proto.readFrom(in);
return reader.read(in);
}
@Override
public Diff<V> readDiff(StreamInput in, K key) throws IOException {
return proto.readDiffFrom(in);
return diffReader.read(in);
}
}

View File

@ -17,18 +17,13 @@
* under the License.
*/
package org.elasticsearch.common.xcontent;
package org.elasticsearch.cluster;
import org.elasticsearch.common.ParseFieldMatcher;
import java.io.IOException;
import org.elasticsearch.common.io.stream.NamedWriteable;
/**
* Indicates that the class supports XContent deserialization.
* Diff that also support NamedWriteable interface
*/
public interface FromXContentBuilder<T> {
/**
* Parses an object with the type T from parser
*/
T fromXContent(XContentParser parser, ParseFieldMatcher parseFieldMatcher) throws IOException;
public interface NamedDiff<T extends Diffable<T>> extends Diff<T>, NamedWriteable {
}

View File

@ -39,12 +39,10 @@ import java.util.Objects;
/**
* Meta data about restore processes that are currently executing
*/
public class RestoreInProgress extends AbstractDiffable<Custom> implements Custom {
public class RestoreInProgress extends AbstractNamedDiffable<Custom> implements Custom {
public static final String TYPE = "restore";
public static final RestoreInProgress PROTO = new RestoreInProgress();
private final List<Entry> entries;
/**
@ -377,15 +375,15 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
* {@inheritDoc}
*/
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
/**
* {@inheritDoc}
*/
@Override
public RestoreInProgress readFrom(StreamInput in) throws IOException {
public static NamedDiff<Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(Custom.class, TYPE, in);
}
public RestoreInProgress(StreamInput in) throws IOException {
Entry[] entries = new Entry[in.readVInt()];
for (int i = 0; i < entries.length; i++) {
Snapshot snapshot = new Snapshot(in);
@ -404,7 +402,7 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
}
entries[i] = new Entry(snapshot, state, Collections.unmodifiableList(indexBuilder), builder.build());
}
return new RestoreInProgress(entries);
this.entries = Arrays.asList(entries);
}
/**

View File

@ -36,10 +36,9 @@ import java.util.Objects;
/**
* A class that represents the snapshot deletions that are in progress in the cluster.
*/
public class SnapshotDeletionsInProgress extends AbstractDiffable<Custom> implements Custom {
public class SnapshotDeletionsInProgress extends AbstractNamedDiffable<Custom> implements Custom {
public static final String TYPE = "snapshot_deletions";
public static final SnapshotDeletionsInProgress PROTO = new SnapshotDeletionsInProgress(Collections.emptyList());
// the version where SnapshotDeletionsInProgress was introduced
public static final Version VERSION_INTRODUCED = Version.V_5_2_0_UNRELEASED;
@ -98,7 +97,7 @@ public class SnapshotDeletionsInProgress extends AbstractDiffable<Custom> implem
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@ -120,16 +119,15 @@ public class SnapshotDeletionsInProgress extends AbstractDiffable<Custom> implem
return 31 + entries.hashCode();
}
@Override
public Custom readFrom(StreamInput in) throws IOException {
return new SnapshotDeletionsInProgress(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeList(entries);
}
public static NamedDiff<Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(Custom.class, TYPE, in);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startArray(TYPE);

View File

@ -44,11 +44,9 @@ import java.util.Map;
/**
* Meta data about snapshots that are currently executing
*/
public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Custom {
public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implements Custom {
public static final String TYPE = "snapshots";
public static final SnapshotsInProgress PROTO = new SnapshotsInProgress();
// denotes an undefined repository state id, which will happen when receiving a cluster state with
// a snapshot in progress from a pre 5.2.x node
public static final long UNDEFINED_REPOSITORY_STATE_ID = -2L;
@ -377,12 +375,15 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@Override
public SnapshotsInProgress readFrom(StreamInput in) throws IOException {
public static NamedDiff<Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(Custom.class, TYPE, in);
}
public SnapshotsInProgress(StreamInput in) throws IOException {
Entry[] entries = new Entry[in.readVInt()];
for (int i = 0; i < entries.length; i++) {
Snapshot snapshot = new Snapshot(in);
@ -416,7 +417,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
repositoryStateId,
builder.build());
}
return new SnapshotsInProgress(entries);
this.entries = Arrays.asList(entries);
}
@Override

View File

@ -21,6 +21,7 @@ package org.elasticsearch.cluster.block;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
import org.elasticsearch.common.collect.ImmutableOpenMap;
@ -48,8 +49,6 @@ import static java.util.stream.Stream.concat;
public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
public static final ClusterBlocks EMPTY_CLUSTER_BLOCK = new ClusterBlocks(emptySet(), ImmutableOpenMap.of());
public static final ClusterBlocks PROTO = EMPTY_CLUSTER_BLOCK;
private final Set<ClusterBlock> global;
private final ImmutableOpenMap<String, Set<ClusterBlock>> indicesBlocks;
@ -59,23 +58,7 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
ClusterBlocks(Set<ClusterBlock> global, ImmutableOpenMap<String, Set<ClusterBlock>> indicesBlocks) {
this.global = global;
this.indicesBlocks = indicesBlocks;
levelHolders = new ImmutableLevelHolder[ClusterBlockLevel.values().length];
for (final ClusterBlockLevel level : ClusterBlockLevel.values()) {
Predicate<ClusterBlock> containsLevel = block -> block.contains(level);
Set<ClusterBlock> newGlobal = unmodifiableSet(global.stream()
.filter(containsLevel)
.collect(toSet()));
ImmutableOpenMap.Builder<String, Set<ClusterBlock>> indicesBuilder = ImmutableOpenMap.builder();
for (ObjectObjectCursor<String, Set<ClusterBlock>> entry : indicesBlocks) {
indicesBuilder.put(entry.key, unmodifiableSet(entry.value.stream()
.filter(containsLevel)
.collect(toSet())));
}
levelHolders[level.id()] = new ImmutableLevelHolder(newGlobal, indicesBuilder.build());
}
levelHolders = generateLevelHolders(global, indicesBlocks);
}
public Set<ClusterBlock> global() {
@ -98,6 +81,27 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
return indices(level).getOrDefault(index, emptySet());
}
private static ImmutableLevelHolder[] generateLevelHolders(Set<ClusterBlock> global,
ImmutableOpenMap<String, Set<ClusterBlock>> indicesBlocks) {
ImmutableLevelHolder[] levelHolders = new ImmutableLevelHolder[ClusterBlockLevel.values().length];
for (final ClusterBlockLevel level : ClusterBlockLevel.values()) {
Predicate<ClusterBlock> containsLevel = block -> block.contains(level);
Set<ClusterBlock> newGlobal = unmodifiableSet(global.stream()
.filter(containsLevel)
.collect(toSet()));
ImmutableOpenMap.Builder<String, Set<ClusterBlock>> indicesBuilder = ImmutableOpenMap.builder();
for (ObjectObjectCursor<String, Set<ClusterBlock>> entry : indicesBlocks) {
indicesBuilder.put(entry.key, unmodifiableSet(entry.value.stream()
.filter(containsLevel)
.collect(toSet())));
}
levelHolders[level.id()] = new ImmutableLevelHolder(newGlobal, indicesBuilder.build());
}
return levelHolders;
}
/**
* Returns <tt>true</tt> if one of the global blocks as its disable state persistence flag set.
*/
@ -239,15 +243,16 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
}
}
@Override
public ClusterBlocks readFrom(StreamInput in) throws IOException {
public ClusterBlocks(StreamInput in) throws IOException {
Set<ClusterBlock> global = readBlockSet(in);
int size = in.readVInt();
ImmutableOpenMap.Builder<String, Set<ClusterBlock>> indicesBuilder = ImmutableOpenMap.builder(size);
for (int j = 0; j < size; j++) {
indicesBuilder.put(in.readString().intern(), readBlockSet(in));
}
return new ClusterBlocks(global, indicesBuilder.build());
this.global = global;
this.indicesBlocks = indicesBuilder.build();
levelHolders = generateLevelHolders(global, indicesBlocks);
}
private static Set<ClusterBlock> readBlockSet(StreamInput in) throws IOException {
@ -259,6 +264,10 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
return unmodifiableSet(blocks);
}
public static Diff<ClusterBlocks> readDiffFrom(StreamInput in) throws IOException {
return AbstractDiffable.readDiffFrom(ClusterBlocks::new, in);
}
static class ImmutableLevelHolder {
static final ImmutableLevelHolder EMPTY = new ImmutableLevelHolder(emptySet(), ImmutableOpenMap.of());
@ -383,9 +392,5 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
}
return new ClusterBlocks(unmodifiableSet(new HashSet<>(global)), indicesBuilder.build());
}
public static ClusterBlocks readClusterBlocks(StreamInput in) throws IOException {
return PROTO.readFrom(in);
}
}
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.cluster.metadata;
import org.elasticsearch.ElasticsearchGenerationException;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.compress.CompressedXContent;
@ -41,8 +42,6 @@ import static java.util.Collections.emptySet;
public class AliasMetaData extends AbstractDiffable<AliasMetaData> {
public static final AliasMetaData PROTO = new AliasMetaData("", null, null, null);
private final String alias;
private final CompressedXContent filter;
@ -173,22 +172,29 @@ public class AliasMetaData extends AbstractDiffable<AliasMetaData> {
}
@Override
public AliasMetaData readFrom(StreamInput in) throws IOException {
String alias = in.readString();
CompressedXContent filter = null;
public AliasMetaData(StreamInput in) throws IOException {
alias = in.readString();
if (in.readBoolean()) {
filter = CompressedXContent.readCompressedString(in);
} else {
filter = null;
}
String indexRouting = null;
if (in.readBoolean()) {
indexRouting = in.readString();
} else {
indexRouting = null;
}
String searchRouting = null;
if (in.readBoolean()) {
searchRouting = in.readString();
searchRoutingValues = Collections.unmodifiableSet(Strings.splitStringByCommaToSet(searchRouting));
} else {
searchRouting = null;
searchRoutingValues = emptySet();
}
return new AliasMetaData(alias, filter, indexRouting, searchRouting);
}
public static Diff<AliasMetaData> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(AliasMetaData::new, in);
}
public static class Builder {
@ -327,14 +333,6 @@ public class AliasMetaData extends AbstractDiffable<AliasMetaData> {
}
return builder.build();
}
public void writeTo(AliasMetaData aliasMetaData, StreamOutput out) throws IOException {
aliasMetaData.writeTo(out);
}
public static AliasMetaData readFrom(StreamInput in) throws IOException {
return PROTO.readFrom(in);
}
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParseFieldMatcherSupplier;
@ -67,7 +68,6 @@ public final class IndexGraveyard implements MetaData.Custom {
500, // the default maximum number of tombstones
Setting.Property.NodeScope);
public static final IndexGraveyard PROTO = new IndexGraveyard(new ArrayList<>());
public static final String TYPE = "index-graveyard";
private static final ParseField TOMBSTONES_FIELD = new ParseField("tombstones");
private static final ObjectParser<List<Tombstone>, ParseFieldMatcherSupplier> GRAVEYARD_PARSER;
@ -83,7 +83,7 @@ public final class IndexGraveyard implements MetaData.Custom {
tombstones = Collections.unmodifiableList(list);
}
private IndexGraveyard(final StreamInput in) throws IOException {
public IndexGraveyard(final StreamInput in) throws IOException {
final int queueSize = in.readVInt();
List<Tombstone> tombstones = new ArrayList<>(queueSize);
for (int i = 0; i < queueSize; i++) {
@ -92,12 +92,8 @@ public final class IndexGraveyard implements MetaData.Custom {
this.tombstones = Collections.unmodifiableList(tombstones);
}
public static IndexGraveyard fromStream(final StreamInput in) throws IOException {
return new IndexGraveyard(in);
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@ -144,7 +140,7 @@ public final class IndexGraveyard implements MetaData.Custom {
return builder.endArray();
}
public IndexGraveyard fromXContent(final XContentParser parser) throws IOException {
public static IndexGraveyard fromXContent(final XContentParser parser) throws IOException {
return new IndexGraveyard(GRAVEYARD_PARSER.parse(parser, () -> ParseFieldMatcher.STRICT));
}
@ -161,19 +157,13 @@ public final class IndexGraveyard implements MetaData.Custom {
}
}
@Override
public IndexGraveyard readFrom(final StreamInput in) throws IOException {
return new IndexGraveyard(in);
}
@Override
@SuppressWarnings("unchecked")
public Diff<MetaData.Custom> diff(final MetaData.Custom previous) {
return new IndexGraveyardDiff((IndexGraveyard) previous, this);
}
@Override
public Diff<MetaData.Custom> readDiffFrom(final StreamInput in) throws IOException {
public static NamedDiff<MetaData.Custom> readDiffFrom(final StreamInput in) throws IOException {
return new IndexGraveyardDiff(in);
}
@ -273,7 +263,7 @@ public final class IndexGraveyard implements MetaData.Custom {
/**
* A class representing a diff of two IndexGraveyard objects.
*/
public static final class IndexGraveyardDiff implements Diff<MetaData.Custom> {
public static final class IndexGraveyardDiff implements NamedDiff<MetaData.Custom> {
private final List<Tombstone> added;
private final int removedCount;
@ -349,6 +339,11 @@ public final class IndexGraveyard implements MetaData.Custom {
public int getRemovedCount() {
return removedCount;
}
@Override
public String getWriteableName() {
return TYPE;
}
}
/**

View File

@ -23,7 +23,6 @@ import com.carrotsearch.hppc.LongArrayList;
import com.carrotsearch.hppc.cursors.IntObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.cluster.Diff;
@ -34,7 +33,6 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.node.DiscoveryNodeFilters;
import org.elasticsearch.cluster.routing.allocation.IndexMetaDataUpdater;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.collect.ImmutableOpenIntMap;
import org.elasticsearch.common.collect.ImmutableOpenMap;
@ -46,7 +44,6 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.xcontent.FromXContentBuilder;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -78,8 +75,12 @@ import static org.elasticsearch.cluster.node.DiscoveryNodeFilters.OpType.OR;
import static org.elasticsearch.common.settings.Settings.readSettingsFromStream;
import static org.elasticsearch.common.settings.Settings.writeSettingsToStream;
public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuilder<IndexMetaData>, ToXContent {
public class IndexMetaData implements Diffable<IndexMetaData>, ToXContent {
/**
* This class will be removed in v7.0
*/
@Deprecated
public interface Custom extends Diffable<Custom>, ToXContent {
String type();
@ -88,6 +89,16 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
Custom fromXContent(XContentParser parser) throws IOException;
/**
* Reads the {@link org.elasticsearch.cluster.Diff} from StreamInput
*/
Diff<Custom> readDiffFrom(StreamInput in) throws IOException;
/**
* Reads an object of this type from the provided {@linkplain StreamInput}. The receiving instance remains unchanged.
*/
Custom readFrom(StreamInput in) throws IOException;
/**
* Merges from this to another, with this being more important, i.e., if something exists in this and another,
* this will prevail.
@ -249,10 +260,6 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
Setting.Property.Dynamic,
Setting.Property.IndexScope);
public static final IndexMetaData PROTO = IndexMetaData.builder("")
.settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT))
.numberOfShards(1).numberOfReplicas(0).build();
public static final String KEY_IN_SYNC_ALLOCATIONS = "in_sync_allocations";
static final String KEY_VERSION = "version";
static final String KEY_ROUTING_NUM_SHARDS = "routing_num_shards";
@ -567,13 +574,11 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
return new IndexMetaDataDiff(previousState, this);
}
@Override
public Diff<IndexMetaData> readDiffFrom(StreamInput in) throws IOException {
public static Diff<IndexMetaData> readDiffFrom(StreamInput in) throws IOException {
return new IndexMetaDataDiff(in);
}
@Override
public IndexMetaData fromXContent(XContentParser parser, ParseFieldMatcher parseFieldMatcher) throws IOException {
public static IndexMetaData fromXContent(XContentParser parser) throws IOException {
return Builder.fromXContent(parser);
}
@ -617,8 +622,10 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
state = State.fromId(in.readByte());
settings = Settings.readSettingsFromStream(in);
primaryTerms = in.readVLongArray();
mappings = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), MappingMetaData.PROTO);
aliases = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), AliasMetaData.PROTO);
mappings = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), MappingMetaData::new,
MappingMetaData::readDiffFrom);
aliases = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), AliasMetaData::new,
AliasMetaData::readDiffFrom);
customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(),
new DiffableUtils.DiffableValueSerializer<String, Custom>() {
@Override
@ -626,6 +633,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
return lookupPrototypeSafe(key).readFrom(in);
}
@SuppressWarnings("unchecked")
@Override
public Diff<Custom> readDiff(StreamInput in, String key) throws IOException {
return lookupPrototypeSafe(key).readDiffFrom(in);
@ -665,8 +673,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
}
}
@Override
public IndexMetaData readFrom(StreamInput in) throws IOException {
public static IndexMetaData readFrom(StreamInput in) throws IOException {
Builder builder = new Builder(in.readString());
builder.version(in.readLong());
builder.setRoutingNumShards(in.readInt());
@ -675,12 +682,12 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
builder.primaryTerms(in.readVLongArray());
int mappingsSize = in.readVInt();
for (int i = 0; i < mappingsSize; i++) {
MappingMetaData mappingMd = MappingMetaData.PROTO.readFrom(in);
MappingMetaData mappingMd = new MappingMetaData(in);
builder.putMapping(mappingMd);
}
int aliasesSize = in.readVInt();
for (int i = 0; i < aliasesSize; i++) {
AliasMetaData aliasMd = AliasMetaData.Builder.readFrom(in);
AliasMetaData aliasMd = new AliasMetaData(in);
builder.putAlias(aliasMd);
}
int customSize = in.readVInt();
@ -1200,10 +1207,6 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
}
return builder.build();
}
public static IndexMetaData readFrom(StreamInput in) throws IOException {
return PROTO.readFrom(in);
}
}
/**

View File

@ -23,6 +23,7 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.collect.ImmutableOpenMap;
@ -51,7 +52,6 @@ import java.util.Set;
public class IndexTemplateMetaData extends AbstractDiffable<IndexTemplateMetaData> {
public static final IndexTemplateMetaData PROTO = IndexTemplateMetaData.builder("").build();
private static final DeprecationLogger DEPRECATION_LOGGER = new DeprecationLogger(Loggers.getLogger(IndexTemplateMetaData.class));
private final String name;
@ -206,8 +206,7 @@ public class IndexTemplateMetaData extends AbstractDiffable<IndexTemplateMetaDat
return result;
}
@Override
public IndexTemplateMetaData readFrom(StreamInput in) throws IOException {
public static IndexTemplateMetaData readFrom(StreamInput in) throws IOException {
Builder builder = new Builder(in.readString());
builder.order(in.readInt());
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
@ -222,7 +221,7 @@ public class IndexTemplateMetaData extends AbstractDiffable<IndexTemplateMetaDat
}
int aliasesSize = in.readVInt();
for (int i = 0; i < aliasesSize; i++) {
AliasMetaData aliasMd = AliasMetaData.Builder.readFrom(in);
AliasMetaData aliasMd = new AliasMetaData(in);
builder.putAlias(aliasMd);
}
int customSize = in.readVInt();
@ -237,6 +236,10 @@ public class IndexTemplateMetaData extends AbstractDiffable<IndexTemplateMetaDat
return builder.build();
}
public static Diff<IndexTemplateMetaData> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(IndexTemplateMetaData::readFrom, in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
@ -525,10 +528,6 @@ public class IndexTemplateMetaData extends AbstractDiffable<IndexTemplateMetaDat
return null;
}
public static IndexTemplateMetaData readFrom(StreamInput in) throws IOException {
return PROTO.readFrom(in);
}
}
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.cluster.metadata;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -40,8 +41,6 @@ import static org.elasticsearch.common.xcontent.support.XContentMapValues.lenien
*/
public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
public static final MappingMetaData PROTO = new MappingMetaData();
public static class Routing {
public static final Routing EMPTY = new Routing(false);
@ -228,11 +227,11 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
return result;
}
public MappingMetaData readFrom(StreamInput in) throws IOException {
String type = in.readString();
CompressedXContent source = CompressedXContent.readCompressedString(in);
public MappingMetaData(StreamInput in) throws IOException {
type = in.readString();
source = CompressedXContent.readCompressedString(in);
// routing
Routing routing = new Routing(in.readBoolean());
routing = new Routing(in.readBoolean());
if (in.getVersion().before(Version.V_6_0_0_alpha1_UNRELEASED)) {
// timestamp
boolean enabled = in.readBoolean();
@ -243,9 +242,11 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
in.readOptionalString(); // defaultTimestamp
in.readOptionalBoolean(); // ignoreMissing
}
hasParentField = in.readBoolean();
}
final boolean hasParentField = in.readBoolean();
return new MappingMetaData(type, source, routing, hasParentField);
public static Diff<MappingMetaData> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(MappingMetaData::new, in);
}
}

View File

@ -27,37 +27,32 @@ import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.Diffable;
import org.elasticsearch.cluster.DiffableUtils;
import org.elasticsearch.cluster.InternalClusterInfoService;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.collect.HppcMaps;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.xcontent.FromXContentBuilder;
import org.elasticsearch.common.xcontent.NamedXContentRegistry.UnknownNamedObjectException;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.ingest.IngestMetadata;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.script.ScriptMetaData;
import java.io.IOException;
import java.util.ArrayList;
@ -69,18 +64,15 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import static java.util.Collections.unmodifiableSet;
import static org.elasticsearch.common.settings.Settings.readSettingsFromStream;
import static org.elasticsearch.common.settings.Settings.writeSettingsToStream;
import static org.elasticsearch.common.util.set.Sets.newHashSet;
public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, FromXContentBuilder<MetaData>, ToXContent {
public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, ToXContent {
public static final MetaData PROTO = builder().build();
private static final Logger logger = Loggers.getLogger(MetaData.class);
public static final String ALL = "_all";
@ -118,48 +110,11 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
*/
public static EnumSet<XContentContext> ALL_CONTEXTS = EnumSet.allOf(XContentContext.class);
public interface Custom extends Diffable<Custom>, ToXContent {
String type();
Custom fromXContent(XContentParser parser) throws IOException;
public interface Custom extends Diffable<Custom>, ToXContent, NamedWriteable {
EnumSet<XContentContext> context();
}
public static Map<String, Custom> customPrototypes = new HashMap<>();
static {
// register non plugin custom metadata
registerPrototype(RepositoriesMetaData.TYPE, RepositoriesMetaData.PROTO);
registerPrototype(IngestMetadata.TYPE, IngestMetadata.PROTO);
registerPrototype(ScriptMetaData.TYPE, ScriptMetaData.PROTO);
registerPrototype(IndexGraveyard.TYPE, IndexGraveyard.PROTO);
}
/**
* 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);
}
@Nullable
public static <T extends Custom> T lookupPrototype(String type) {
//noinspection unchecked
return (T) customPrototypes.get(type);
}
public static <T extends Custom> T lookupPrototypeSafe(String type) {
//noinspection unchecked
T proto = (T) customPrototypes.get(type);
if (proto == null) {
throw new IllegalArgumentException("No custom metadata prototype registered for type [" + type + "], node likely missing plugins");
}
return proto;
}
public static final Setting<Boolean> SETTING_READ_ONLY_SETTING =
Setting.boolSetting("cluster.blocks.read_only", false, Property.Dynamic, Property.NodeScope);
@ -596,14 +551,14 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
// Check if any persistent metadata needs to be saved
int customCount1 = 0;
for (ObjectObjectCursor<String, Custom> cursor : metaData1.customs) {
if (customPrototypes.get(cursor.key).context().contains(XContentContext.GATEWAY)) {
if (cursor.value.context().contains(XContentContext.GATEWAY)) {
if (!cursor.value.equals(metaData2.custom(cursor.key))) return false;
customCount1++;
}
}
int customCount2 = 0;
for (ObjectObjectCursor<String, Custom> cursor : metaData2.customs) {
if (customPrototypes.get(cursor.key).context().contains(XContentContext.GATEWAY)) {
for (ObjectCursor<Custom> cursor : metaData2.customs.values()) {
if (cursor.value.context().contains(XContentContext.GATEWAY)) {
customCount2++;
}
}
@ -616,13 +571,11 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
return new MetaDataDiff(previousState, this);
}
@Override
public Diff<MetaData> readDiffFrom(StreamInput in) throws IOException {
public static Diff<MetaData> readDiffFrom(StreamInput in) throws IOException {
return new MetaDataDiff(in);
}
@Override
public MetaData fromXContent(XContentParser parser, ParseFieldMatcher parseFieldMatcher) throws IOException {
public static MetaData fromXContent(XContentParser parser) throws IOException {
return Builder.fromXContent(parser);
}
@ -659,18 +612,21 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
version = in.readLong();
transientSettings = Settings.readSettingsFromStream(in);
persistentSettings = Settings.readSettingsFromStream(in);
indices = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexMetaData.PROTO);
templates = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexTemplateMetaData.PROTO);
indices = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexMetaData::readFrom,
IndexMetaData::readDiffFrom);
templates = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexTemplateMetaData::readFrom,
IndexTemplateMetaData::readDiffFrom);
customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(),
new DiffableUtils.DiffableValueSerializer<String, Custom>() {
@Override
public Custom read(StreamInput in, String key) throws IOException {
return lookupPrototypeSafe(key).readFrom(in);
return in.readNamedWriteable(Custom.class, key);
}
@SuppressWarnings("unchecked")
@Override
public Diff<Custom> readDiff(StreamInput in, String key) throws IOException {
return lookupPrototypeSafe(key).readDiffFrom(in);
return in.readNamedWriteable(NamedDiff.class, key);
}
});
}
@ -700,8 +656,7 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
}
}
@Override
public MetaData readFrom(StreamInput in) throws IOException {
public static MetaData readFrom(StreamInput in) throws IOException {
Builder builder = new Builder();
builder.version = in.readLong();
builder.clusterUUID = in.readString();
@ -709,17 +664,16 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
builder.persistentSettings(readSettingsFromStream(in));
int size = in.readVInt();
for (int i = 0; i < size; i++) {
builder.put(IndexMetaData.Builder.readFrom(in), false);
builder.put(IndexMetaData.readFrom(in), false);
}
size = in.readVInt();
for (int i = 0; i < size; i++) {
builder.put(IndexTemplateMetaData.Builder.readFrom(in));
builder.put(IndexTemplateMetaData.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);
Custom customIndexMetaData = in.readNamedWriteable(Custom.class);
builder.putCustom(customIndexMetaData.getWriteableName(), customIndexMetaData);
}
return builder.build();
}
@ -740,8 +694,7 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
}
out.writeVInt(customs.size());
for (ObjectObjectCursor<String, Custom> cursor : customs) {
out.writeString(cursor.key);
cursor.value.writeTo(out);
out.writeNamedWriteable(cursor.value);
}
}
@ -1051,8 +1004,7 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
}
for (ObjectObjectCursor<String, Custom> cursor : metaData.customs()) {
Custom proto = lookupPrototypeSafe(cursor.key);
if (proto.context().contains(context)) {
if (cursor.value.context().contains(context)) {
builder.startObject(cursor.key);
cursor.value.toXContent(builder, params);
builder.endObject();
@ -1103,14 +1055,12 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
builder.put(IndexTemplateMetaData.Builder.fromXContent(parser, parser.currentName()));
}
} else {
// check if its a custom index metadata
Custom proto = lookupPrototype(currentFieldName);
if (proto == null) {
//TODO warn
try {
Custom custom = parser.namedObject(Custom.class, currentFieldName, null);
builder.putCustom(custom.getWriteableName(), custom);
} catch (UnknownNamedObjectException ex) {
logger.warn("Skipping unknown custom object with type {}", currentFieldName);
parser.skipChildren();
} else {
Custom custom = proto.fromXContent(parser);
builder.putCustom(custom.type(), custom);
}
}
} else if (token.isValue()) {
@ -1127,10 +1077,6 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
}
return builder.build();
}
public static MetaData readFrom(StreamInput in) throws IOException {
return PROTO.readFrom(in);
}
}
private static final ToXContent.Params FORMAT_PARAMS;

View File

@ -21,6 +21,9 @@ package org.elasticsearch.cluster.metadata;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.AbstractNamedDiffable;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.cluster.metadata.MetaData.Custom;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -39,12 +42,10 @@ import java.util.List;
/**
* Contains metadata about registered snapshot repositories
*/
public class RepositoriesMetaData extends AbstractDiffable<Custom> implements MetaData.Custom {
public class RepositoriesMetaData extends AbstractNamedDiffable<Custom> implements Custom {
public static final String TYPE = "repositories";
public static final RepositoriesMetaData PROTO = new RepositoriesMetaData();
private final List<RepositoryMetaData> repositories;
/**
@ -100,20 +101,20 @@ public class RepositoriesMetaData extends AbstractDiffable<Custom> implements Me
* {@inheritDoc}
*/
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
/**
* {@inheritDoc}
*/
@Override
public Custom readFrom(StreamInput in) throws IOException {
public RepositoriesMetaData(StreamInput in) throws IOException {
RepositoryMetaData[] repository = new RepositoryMetaData[in.readVInt()];
for (int i = 0; i < repository.length; i++) {
repository[i] = RepositoryMetaData.readFrom(in);
repository[i] = new RepositoryMetaData(in);
}
return new RepositoriesMetaData(repository);
this.repositories = Arrays.asList(repository);
}
public static NamedDiff<Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(Custom.class, TYPE, in);
}
/**
@ -127,11 +128,7 @@ public class RepositoriesMetaData extends AbstractDiffable<Custom> implements Me
}
}
/**
* {@inheritDoc}
*/
@Override
public RepositoriesMetaData fromXContent(XContentParser parser) throws IOException {
public static RepositoriesMetaData fromXContent(XContentParser parser) throws IOException {
XContentParser.Token token;
List<RepositoryMetaData> repository = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {

View File

@ -73,17 +73,10 @@ public class RepositoryMetaData {
}
/**
* Reads repository metadata from stream input
*
* @param in stream input
* @return repository metadata
*/
public static RepositoryMetaData readFrom(StreamInput in) throws IOException {
String name = in.readString();
String type = in.readString();
Settings settings = Settings.readSettingsFromStream(in);
return new RepositoryMetaData(name, type, settings);
public RepositoryMetaData(StreamInput in) throws IOException {
name = in.readString();
type = in.readString();
settings = Settings.readSettingsFromStream(in);
}
/**

View File

@ -24,6 +24,7 @@ import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.ImmutableOpenMap;
@ -46,7 +47,6 @@ import java.util.Map;
public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements Iterable<DiscoveryNode> {
public static final DiscoveryNodes EMPTY_NODES = builder().build();
public static final DiscoveryNodes PROTO = EMPTY_NODES;
private final ImmutableOpenMap<String, DiscoveryNode> nodes;
private final ImmutableOpenMap<String, DiscoveryNode> dataNodes;
@ -523,7 +523,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
}
}
private DiscoveryNodes readFrom(StreamInput in, DiscoveryNode localNode) throws IOException {
public static DiscoveryNodes readFrom(StreamInput in, DiscoveryNode localNode) throws IOException {
Builder builder = new Builder();
if (in.readBoolean()) {
builder.masterNodeId(in.readString());
@ -546,9 +546,8 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
return builder.build();
}
@Override
public DiscoveryNodes readFrom(StreamInput in) throws IOException {
return readFrom(in, getLocalNode());
public static Diff<DiscoveryNodes> readDiffFrom(StreamInput in, DiscoveryNode localNode) throws IOException {
return AbstractDiffable.readDiffFrom(in1 -> readFrom(in1, localNode), in);
}
public static Builder builder() {
@ -678,10 +677,6 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
);
}
public static DiscoveryNodes readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException {
return PROTO.readFrom(in, localNode);
}
public boolean isLocalNodeElectedMaster() {
return masterNodeId != null && masterNodeId.equals(localNodeId);
}

View File

@ -25,6 +25,7 @@ import com.carrotsearch.hppc.cursors.IntObjectCursor;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.RecoverySource.LocalShardsRecoverySource;
@ -63,8 +64,6 @@ import java.util.Set;
*/
public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> implements Iterable<IndexShardRoutingTable> {
public static final IndexRoutingTable PROTO = builder(new Index("", "_na_")).build();
private final Index index;
private final ShardShuffler shuffler;
@ -319,8 +318,7 @@ public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> imple
return result;
}
@Override
public IndexRoutingTable readFrom(StreamInput in) throws IOException {
public static IndexRoutingTable readFrom(StreamInput in) throws IOException {
Index index = new Index(in);
Builder builder = new Builder(index);
@ -332,6 +330,10 @@ public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> imple
return builder.build();
}
public static Diff<IndexRoutingTable> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(IndexRoutingTable::readFrom, in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
index.writeTo(out);
@ -354,17 +356,6 @@ public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> imple
this.index = index;
}
/**
* Reads an {@link IndexRoutingTable} from an {@link StreamInput}
*
* @param in {@link StreamInput} to read the {@link IndexRoutingTable} from
* @return {@link IndexRoutingTable} read
* @throws IOException if something happens during read
*/
public static IndexRoutingTable readFrom(StreamInput in) throws IOException {
return PROTO.readFrom(in);
}
/**
* Initializes a new empty index, as if it was created from an API.
*/

View File

@ -56,8 +56,6 @@ import java.util.function.Predicate;
*/
public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<RoutingTable> {
public static RoutingTable PROTO = builder().build();
public static final RoutingTable EMPTY_ROUTING_TABLE = builder().build();
private final long version;
@ -349,18 +347,16 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
return new RoutingTableDiff(previousState, this);
}
@Override
public Diff<RoutingTable> readDiffFrom(StreamInput in) throws IOException {
public static Diff<RoutingTable> readDiffFrom(StreamInput in) throws IOException {
return new RoutingTableDiff(in);
}
@Override
public RoutingTable readFrom(StreamInput in) throws IOException {
public static RoutingTable readFrom(StreamInput in) throws IOException {
Builder builder = new Builder();
builder.version = in.readLong();
int size = in.readVInt();
for (int i = 0; i < size; i++) {
IndexRoutingTable index = IndexRoutingTable.Builder.readFrom(in);
IndexRoutingTable index = IndexRoutingTable.readFrom(in);
builder.add(index);
}
@ -389,7 +385,8 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
public RoutingTableDiff(StreamInput in) throws IOException {
version = in.readLong();
indicesRouting = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexRoutingTable.PROTO);
indicesRouting = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexRoutingTable::readFrom,
IndexRoutingTable::readDiffFrom);
}
@Override
@ -607,10 +604,6 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
indicesRouting = null;
return table;
}
public static RoutingTable readFrom(StreamInput in) throws IOException {
return PROTO.readFrom(in);
}
}
@Override

View File

@ -36,14 +36,20 @@ public class NamedWriteableAwareStreamInput extends FilterStreamInput {
@Override
public <C extends NamedWriteable> C readNamedWriteable(Class<C> categoryClass) throws IOException {
String name = readString();
return readNamedWriteable(categoryClass, name);
}
@Override
public <C extends NamedWriteable> C readNamedWriteable(@SuppressWarnings("unused") Class<C> categoryClass,
@SuppressWarnings("unused") String name) throws IOException {
Writeable.Reader<? extends C> reader = namedWriteableRegistry.getReader(categoryClass, name);
C c = reader.read(this);
if (c == null) {
throw new IOException(
"Writeable.Reader [" + reader + "] returned null which is not allowed and probably means it screwed up the stream.");
"Writeable.Reader [" + reader + "] returned null which is not allowed and probably means it screwed up the stream.");
}
assert name.equals(c.getWriteableName()) : c + " claims to have a different name [" + c.getWriteableName()
+ "] than it was read from [" + name + "].";
+ "] than it was read from [" + name + "].";
return c;
}
}

View File

@ -825,6 +825,22 @@ public abstract class StreamInput extends InputStream {
throw new UnsupportedOperationException("can't read named writeable from StreamInput");
}
/**
* Reads a {@link NamedWriteable} from the current stream with the given name. It is assumed that the caller obtained the name
* from other source, so it's not read from the stream. The name is used for looking for
* the corresponding entry in the registry by name, so that the proper object can be read and returned.
* Default implementation throws {@link UnsupportedOperationException} as StreamInput doesn't hold a registry.
* Use {@link FilterInputStream} instead which wraps a stream and supports a {@link NamedWriteableRegistry} too.
*
* Prefer {@link StreamInput#readNamedWriteable(Class)} and {@link StreamOutput#writeNamedWriteable(NamedWriteable)} unless you
* have a compelling reason to use this method instead.
*/
@Nullable
public <C extends NamedWriteable> C readNamedWriteable(@SuppressWarnings("unused") Class<C> categoryClass,
@SuppressWarnings("unused") String name) throws IOException {
throw new UnsupportedOperationException("can't read named writeable from StreamInput");
}
/**
* Reads an optional {@link NamedWriteable}.
*/

View File

@ -26,6 +26,7 @@ import org.apache.lucene.util.IOUtils;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
@ -86,7 +87,7 @@ public class IndexFolderUpgrader {
void upgrade(final String indexFolderName) throws IOException {
for (NodeEnvironment.NodePath nodePath : nodeEnv.nodePaths()) {
final Path indexFolderPath = nodePath.indicesPath.resolve(indexFolderName);
final IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger, indexFolderPath);
final IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY, indexFolderPath);
if (indexMetaData != null) {
final Index index = indexMetaData.getIndex();
if (needsUpgrade(index, indexFolderName)) {

View File

@ -30,6 +30,7 @@ import java.util.function.Supplier;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
@ -54,8 +55,9 @@ public class DiscoveryModule {
private final Discovery discovery;
public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportService transportService, NetworkService networkService,
ClusterService clusterService, List<DiscoveryPlugin> plugins) {
public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, NetworkService networkService, ClusterService clusterService,
List<DiscoveryPlugin> plugins) {
final UnicastHostsProvider hostsProvider;
Map<String, Supplier<UnicastHostsProvider>> hostProviders = new HashMap<>();
@ -78,10 +80,12 @@ public class DiscoveryModule {
}
Map<String, Supplier<Discovery>> discoveryTypes = new HashMap<>();
discoveryTypes.put("zen", () -> new ZenDiscovery(settings, threadPool, transportService, clusterService, hostsProvider));
discoveryTypes.put("zen",
() -> new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider));
discoveryTypes.put("none", () -> new NoneDiscovery(settings, clusterService, clusterService.getClusterSettings()));
for (DiscoveryPlugin plugin : plugins) {
plugin.getDiscoveryTypes(threadPool, transportService, clusterService, hostsProvider).entrySet().forEach(entry -> {
plugin.getDiscoveryTypes(threadPool, transportService, namedWriteableRegistry,
clusterService, hostsProvider).entrySet().forEach(entry -> {
if (discoveryTypes.put(entry.getKey(), entry.getValue()) != null) {
throw new IllegalArgumentException("Cannot register discovery type [" + entry.getKey() + "] twice");
}

View File

@ -170,7 +170,7 @@ public class MembershipAction extends AbstractComponent {
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
this.state = ClusterState.Builder.readFrom(in, localNode.get());
this.state = ClusterState.readFrom(in, localNode.get());
}
@Override

View File

@ -34,6 +34,8 @@ import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.compress.Compressor;
import org.elasticsearch.common.compress.CompressorFactory;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
@ -79,6 +81,7 @@ public class PublishClusterStateAction extends AbstractComponent {
}
private final TransportService transportService;
private final NamedWriteableRegistry namedWriteableRegistry;
private final Supplier<ClusterState> clusterStateSupplier;
private final NewPendingClusterStateListener newPendingClusterStatelistener;
private final DiscoverySettings discoverySettings;
@ -88,12 +91,14 @@ public class PublishClusterStateAction extends AbstractComponent {
public PublishClusterStateAction(
Settings settings,
TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry,
Supplier<ClusterState> clusterStateSupplier,
NewPendingClusterStateListener listener,
DiscoverySettings discoverySettings,
ClusterName clusterName) {
super(settings);
this.transportService = transportService;
this.namedWriteableRegistry = namedWriteableRegistry;
this.clusterStateSupplier = clusterStateSupplier;
this.newPendingClusterStatelistener = listener;
this.discoverySettings = discoverySettings;
@ -376,15 +381,16 @@ public class PublishClusterStateAction extends AbstractComponent {
} else {
in = request.bytes().streamInput();
}
in = new NamedWriteableAwareStreamInput(in, namedWriteableRegistry);
in.setVersion(request.version());
synchronized (lastSeenClusterStateMutex) {
final ClusterState incomingState;
// If true we received full cluster state - otherwise diffs
if (in.readBoolean()) {
incomingState = ClusterState.Builder.readFrom(in, clusterStateSupplier.get().nodes().getLocalNode());
incomingState = ClusterState.readFrom(in, clusterStateSupplier.get().nodes().getLocalNode());
logger.debug("received full cluster state version [{}] with size [{}]", incomingState.version(), request.bytes().length());
} else if (lastSeenClusterState != null) {
Diff<ClusterState> diff = lastSeenClusterState.readDiffFrom(in);
Diff<ClusterState> diff = ClusterState.readDiffFrom(in, lastSeenClusterState.nodes().getLocalNode());
incomingState = diff.apply(lastSeenClusterState);
logger.debug("received diff cluster state version [{}] with uuid [{}], diff size [{}]",
incomingState.version(), incomingState.stateUUID(), request.bytes().length());

View File

@ -43,6 +43,7 @@ import org.elasticsearch.common.Priority;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.component.Lifecycle;
import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasables;
@ -102,6 +103,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
public static final String DISCOVERY_REJOIN_ACTION_NAME = "internal:discovery/zen/rejoin";
private final TransportService transportService;
private final NamedWriteableRegistry namedWriteableRegistry;
private final ClusterService clusterService;
private AllocationService allocationService;
private final ClusterName clusterName;
@ -138,11 +140,13 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
private volatile NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor;
public ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) {
super(settings);
this.clusterService = clusterService;
this.clusterName = clusterService.getClusterName();
this.transportService = transportService;
this.namedWriteableRegistry = namedWriteableRegistry;
this.discoverySettings = new DiscoverySettings(settings, clusterService.getClusterSettings());
this.zenPing = newZenPing(settings, threadPool, transportService, hostsProvider);
this.electMaster = new ElectMasterService(settings);
@ -179,6 +183,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
new PublishClusterStateAction(
settings,
transportService,
namedWriteableRegistry,
clusterService::state,
new NewPendingClusterStateListener(),
discoverySettings,

View File

@ -45,6 +45,7 @@ import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
@ -372,7 +373,7 @@ public final class NodeEnvironment implements Closeable {
private static NodeMetaData loadOrCreateNodeMetaData(Settings settings, Logger logger,
NodePath... nodePaths) throws IOException {
final Path[] paths = Arrays.stream(nodePaths).map(np -> np.path).toArray(Path[]::new);
NodeMetaData metaData = NodeMetaData.FORMAT.loadLatestState(logger, paths);
NodeMetaData metaData = NodeMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY, paths);
if (metaData == null) {
metaData = new NodeMetaData(generateNodeId(settings));
}

View File

@ -223,7 +223,7 @@ public class LocalAllocateDangledIndices extends AbstractComponent {
fromNode = new DiscoveryNode(in);
indices = new IndexMetaData[in.readVInt()];
for (int i = 0; i < indices.length; i++) {
indices[i] = IndexMetaData.Builder.readFrom(in);
indices[i] = IndexMetaData.readFrom(in);
}
}

View File

@ -182,7 +182,7 @@ public abstract class MetaDataStateFormat<T> {
* Reads the state from a given file and compares the expected version against the actual version of
* the state.
*/
public final T read(Path file) throws IOException {
public final T read(NamedXContentRegistry namedXContentRegistry, Path file) throws IOException {
try (Directory dir = newDirectory(file.getParent())) {
try (final IndexInput indexInput = dir.openInput(file.getFileName().toString(), IOContext.DEFAULT)) {
// We checksum the entire file before we even go and parse it. If it's corrupted we barf right here.
@ -197,8 +197,7 @@ public abstract class MetaDataStateFormat<T> {
long filePointer = indexInput.getFilePointer();
long contentSize = indexInput.length() - CodecUtil.footerLength() - filePointer;
try (IndexInput slice = indexInput.slice("state_xcontent", filePointer, contentSize)) {
// It is safe to use EMPTY here because this never uses namedObject
try (XContentParser parser = XContentFactory.xContent(xContentType).createParser(NamedXContentRegistry.EMPTY,
try (XContentParser parser = XContentFactory.xContent(xContentType).createParser(namedXContentRegistry,
new InputStreamIndexInput(slice, contentSize))) {
return fromXContent(parser);
}
@ -262,7 +261,7 @@ public abstract class MetaDataStateFormat<T> {
* @param dataLocations the data-locations to try.
* @return the latest state or <code>null</code> if no state was found.
*/
public T loadLatestState(Logger logger, Path... dataLocations) throws IOException {
public T loadLatestState(Logger logger, NamedXContentRegistry namedXContentRegistry, Path... dataLocations) throws IOException {
List<PathAndStateId> files = new ArrayList<>();
long maxStateId = -1;
boolean maxStateIdIsLegacy = true;
@ -313,15 +312,14 @@ public abstract class MetaDataStateFormat<T> {
logger.debug("{}: no data for [{}], ignoring...", prefix, stateFile.toAbsolutePath());
continue;
}
// EMPTY is safe here because no parser uses namedObject
try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, new BytesArray(data))) {
try (XContentParser parser = XContentHelper.createParser(namedXContentRegistry, new BytesArray(data))) {
state = fromXContent(parser);
}
if (state == null) {
logger.debug("{}: no data for [{}], ignoring...", prefix, stateFile.toAbsolutePath());
}
} else {
state = read(stateFile);
state = read(namedXContentRegistry, stateFile);
logger.trace("state id [{}] read from [{}]", id, stateFile.getFileName());
}
return state;

View File

@ -26,6 +26,7 @@ import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.Index;
@ -40,10 +41,12 @@ import java.util.function.Predicate;
public class MetaStateService extends AbstractComponent {
private final NodeEnvironment nodeEnv;
private final NamedXContentRegistry namedXContentRegistry;
public MetaStateService(Settings settings, NodeEnvironment nodeEnv) {
public MetaStateService(Settings settings, NodeEnvironment nodeEnv, NamedXContentRegistry namedXContentRegistry) {
super(settings);
this.nodeEnv = nodeEnv;
this.namedXContentRegistry = namedXContentRegistry;
}
/**
@ -59,7 +62,8 @@ public class MetaStateService extends AbstractComponent {
metaDataBuilder = MetaData.builder();
}
for (String indexFolderName : nodeEnv.availableIndexFolders()) {
IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger, nodeEnv.resolveIndexFolder(indexFolderName));
IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger, namedXContentRegistry,
nodeEnv.resolveIndexFolder(indexFolderName));
if (indexMetaData != null) {
metaDataBuilder.put(indexMetaData, false);
} else {
@ -74,7 +78,7 @@ public class MetaStateService extends AbstractComponent {
*/
@Nullable
public IndexMetaData loadIndexState(Index index) throws IOException {
return IndexMetaData.FORMAT.loadLatestState(logger, nodeEnv.indexPaths(index));
return IndexMetaData.FORMAT.loadLatestState(logger, namedXContentRegistry, nodeEnv.indexPaths(index));
}
/**
@ -86,7 +90,7 @@ public class MetaStateService extends AbstractComponent {
if (excludeIndexPathIdsPredicate.test(indexFolderName)) {
continue;
}
IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger,
IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger, namedXContentRegistry,
nodeEnv.resolveIndexFolder(indexFolderName));
if (indexMetaData != null) {
final String indexPathId = indexMetaData.getIndex().getUUID();
@ -106,7 +110,7 @@ public class MetaStateService extends AbstractComponent {
* Loads the global state, *without* index state, see {@link #loadFullState()} for that.
*/
MetaData loadGlobalState() throws IOException {
return MetaData.FORMAT.loadLatestState(logger, nodeEnv.nodeDataPaths());
return MetaData.FORMAT.loadLatestState(logger, namedXContentRegistry, nodeEnv.nodeDataPaths());
}
/**

View File

@ -185,7 +185,7 @@ public class TransportNodesListGatewayMetaState extends TransportNodesAction<Tra
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
if (in.readBoolean()) {
metaData = MetaData.Builder.readFrom(in);
metaData = MetaData.readFrom(in);
}
}

View File

@ -39,6 +39,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
@ -115,7 +116,7 @@ public class TransportNodesListGatewayStartedShards extends
try {
final ShardId shardId = request.getShardId();
logger.trace("{} loading local shard state info", shardId);
ShardStateMetaData shardStateMetaData = ShardStateMetaData.FORMAT.loadLatestState(logger,
ShardStateMetaData shardStateMetaData = ShardStateMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY,
nodeEnv.availableShardPaths(request.shardId));
if (shardStateMetaData != null) {
IndexMetaData metaData = clusterService.state().metaData().index(shardId.getIndex());
@ -123,7 +124,8 @@ public class TransportNodesListGatewayStartedShards extends
// we may send this requests while processing the cluster state that recovered the index
// sometimes the request comes in before the local node processed that cluster state
// in such cases we can load it from disk
metaData = IndexMetaData.FORMAT.loadLatestState(logger, nodeEnv.indexPaths(shardId.getIndex()));
metaData = IndexMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY,
nodeEnv.indexPaths(shardId.getIndex()));
}
if (metaData == null) {
ElasticsearchException e = new ElasticsearchException("failed to find local IndexMetaData");

View File

@ -34,8 +34,6 @@ public class SeqNoStats implements ToXContent, Writeable {
private static final String LOCAL_CHECKPOINT = "local_checkpoint";
private static final String GLOBAL_CHECKPOINT = "global_checkpoint";
public static final SeqNoStats PROTO = new SeqNoStats(0, 0, 0);
private final long maxSeqNo;
private final long localCheckpoint;
private final long globalCheckpoint;

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.shard;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock;
import org.elasticsearch.index.IndexSettings;
@ -113,7 +114,8 @@ public final class ShardPath {
final Path[] paths = env.availableShardPaths(shardId);
Path loadedPath = null;
for (Path path : paths) {
ShardStateMetaData load = ShardStateMetaData.FORMAT.loadLatestState(logger, path);
// EMPTY is safe here because we never call namedObject
ShardStateMetaData load = ShardStateMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY, path);
if (load != null) {
if (load.indexUUID.equals(indexUUID) == false && IndexMetaData.INDEX_UUID_NA_VALUE.equals(load.indexUUID) == false) {
logger.warn("{} found shard on path: [{}] with a different index UUID - this shard seems to be leftover from a different index with the same name. Remove the leftover shard in order to reuse the path with the current index", shardId, path);
@ -150,7 +152,8 @@ public final class ShardPath {
final String indexUUID = indexSettings.getUUID();
final Path[] paths = env.availableShardPaths(lock.getShardId());
for (Path path : paths) {
ShardStateMetaData load = ShardStateMetaData.FORMAT.loadLatestState(logger, path);
// EMPTY is safe here because we never call namedObject
ShardStateMetaData load = ShardStateMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY, path);
if (load != null) {
if (load.indexUUID.equals(indexUUID) == false && IndexMetaData.INDEX_UUID_NA_VALUE.equals(load.indexUUID) == false) {
logger.warn("{} deleting leftover shard on path: [{}] with a different index UUID", lock.getShardId(), path);

View File

@ -27,7 +27,6 @@ import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.FromXContentBuilder;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -41,9 +40,7 @@ import java.util.List;
/**
* Shard snapshot metadata
*/
public class BlobStoreIndexShardSnapshot implements ToXContent, FromXContentBuilder<BlobStoreIndexShardSnapshot> {
public static final BlobStoreIndexShardSnapshot PROTO = new BlobStoreIndexShardSnapshot();
public class BlobStoreIndexShardSnapshot implements ToXContent {
/**
* Information about snapshotted file
@ -478,6 +475,7 @@ public class BlobStoreIndexShardSnapshot implements ToXContent, FromXContentBuil
private static final ParseField PARSE_NUMBER_OF_FILES = new ParseField("number_of_files");
private static final ParseField PARSE_TOTAL_SIZE = new ParseField("total_size");
private static final ParseField PARSE_FILES = new ParseField("files");
private static final ParseFieldMatcher parseFieldMatcher = ParseFieldMatcher.EMPTY;
/**
* Serializes shard snapshot metadata info into JSON
@ -507,8 +505,7 @@ public class BlobStoreIndexShardSnapshot implements ToXContent, FromXContentBuil
* @param parser parser
* @return shard snapshot metadata
*/
public BlobStoreIndexShardSnapshot fromXContent(XContentParser parser, ParseFieldMatcher parseFieldMatcher) throws IOException {
public static BlobStoreIndexShardSnapshot fromXContent(XContentParser parser) throws IOException {
String snapshot = null;
long indexVersion = -1;
long startTime = 0;

View File

@ -22,7 +22,6 @@ package org.elasticsearch.index.snapshots.blobstore;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.FromXContentBuilder;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -44,9 +43,7 @@ import static java.util.Collections.unmodifiableMap;
* This class is used to find files that were already snapshotted and clear out files that no longer referenced by any
* snapshots
*/
public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, ToXContent, FromXContentBuilder<BlobStoreIndexShardSnapshots> {
public static final BlobStoreIndexShardSnapshots PROTO = new BlobStoreIndexShardSnapshots();
public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, ToXContent {
private final List<SnapshotFiles> shardSnapshots;
private final Map<String, FileInfo> files;
@ -156,6 +153,7 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
static final ParseField FILES = new ParseField("files");
static final ParseField SNAPSHOTS = new ParseField("snapshots");
}
private static final ParseFieldMatcher parseFieldMatcher = ParseFieldMatcher.EMPTY;
/**
* Writes index file for the shard in the following format.
@ -232,8 +230,7 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
return builder;
}
@Override
public BlobStoreIndexShardSnapshots fromXContent(XContentParser parser, ParseFieldMatcher parseFieldMatcher) throws IOException {
public static BlobStoreIndexShardSnapshots fromXContent(XContentParser parser) throws IOException {
XContentParser.Token token = parser.currentToken();
if (token == null) { // New parser
token = parser.nextToken();

View File

@ -39,6 +39,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.gateway.AsyncShardFetch;
import org.elasticsearch.index.IndexService;
@ -130,7 +131,8 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesAction<T
// we may send this requests while processing the cluster state that recovered the index
// sometimes the request comes in before the local node processed that cluster state
// in such cases we can load it from disk
metaData = IndexMetaData.FORMAT.loadLatestState(logger, nodeEnv.indexPaths(shardId.getIndex()));
metaData = IndexMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY,
nodeEnv.indexPaths(shardId.getIndex()));
}
if (metaData == null) {
logger.trace("{} node doesn't have meta data for the requests index, responding with empty", shardId);

View File

@ -21,6 +21,7 @@ package org.elasticsearch.ingest;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.DiffableUtils;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
@ -45,7 +46,6 @@ import java.util.Map;
public final class IngestMetadata implements MetaData.Custom {
public static final String TYPE = "ingest";
public static final IngestMetadata PROTO = new IngestMetadata();
private static final ParseField PIPELINES_FIELD = new ParseField("pipeline");
private static final ObjectParser<List<PipelineConfiguration>, ParseFieldMatcherSupplier> INGEST_METADATA_PARSER = new ObjectParser<>(
"ingest_metadata", ArrayList::new);
@ -67,7 +67,7 @@ public final class IngestMetadata implements MetaData.Custom {
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@ -75,15 +75,14 @@ public final class IngestMetadata implements MetaData.Custom {
return pipelines;
}
@Override
public IngestMetadata readFrom(StreamInput in) throws IOException {
public IngestMetadata(StreamInput in) throws IOException {
int size = in.readVInt();
Map<String, PipelineConfiguration> pipelines = new HashMap<>(size);
for (int i = 0; i < size; i++) {
PipelineConfiguration pipeline = PipelineConfiguration.readPipelineConfiguration(in);
PipelineConfiguration pipeline = PipelineConfiguration.readFrom(in);
pipelines.put(pipeline.getId(), pipeline);
}
return new IngestMetadata(pipelines);
this.pipelines = Collections.unmodifiableMap(pipelines);
}
@Override
@ -94,8 +93,7 @@ public final class IngestMetadata implements MetaData.Custom {
}
}
@Override
public IngestMetadata fromXContent(XContentParser parser) throws IOException {
public static IngestMetadata fromXContent(XContentParser parser) throws IOException {
Map<String, PipelineConfiguration> pipelines = new HashMap<>();
List<PipelineConfiguration> configs = INGEST_METADATA_PARSER.parse(parser, () -> ParseFieldMatcher.STRICT);
for (PipelineConfiguration pipeline : configs) {
@ -124,12 +122,11 @@ public final class IngestMetadata implements MetaData.Custom {
return new IngestMetadataDiff((IngestMetadata) before, this);
}
@Override
public Diff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {
public static NamedDiff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {
return new IngestMetadataDiff(in);
}
static class IngestMetadataDiff implements Diff<MetaData.Custom> {
static class IngestMetadataDiff implements NamedDiff<MetaData.Custom> {
final Diff<Map<String, PipelineConfiguration>> pipelines;
@ -138,7 +135,8 @@ public final class IngestMetadata implements MetaData.Custom {
}
public IngestMetadataDiff(StreamInput in) throws IOException {
pipelines = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), PipelineConfiguration.PROTOTYPE);
pipelines = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), PipelineConfiguration::readFrom,
PipelineConfiguration::readDiffFrom);
}
@Override
@ -150,6 +148,11 @@ public final class IngestMetadata implements MetaData.Custom {
public void writeTo(StreamOutput out) throws IOException {
pipelines.writeTo(out);
}
@Override
public String getWriteableName() {
return TYPE;
}
}
@Override

View File

@ -20,6 +20,7 @@
package org.elasticsearch.ingest;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcherSupplier;
import org.elasticsearch.common.bytes.BytesReference;
@ -39,11 +40,6 @@ import java.util.Map;
*/
public final class PipelineConfiguration extends AbstractDiffable<PipelineConfiguration> implements ToXContent {
static final PipelineConfiguration PROTOTYPE = new PipelineConfiguration(null, null);
public static PipelineConfiguration readPipelineConfiguration(StreamInput in) throws IOException {
return PROTOTYPE.readFrom(in);
}
private static final ObjectParser<Builder, ParseFieldMatcherSupplier> PARSER = new ObjectParser<>("pipeline_config", Builder::new);
static {
PARSER.declareString(Builder::setId, new ParseField("id"));
@ -103,11 +99,14 @@ public final class PipelineConfiguration extends AbstractDiffable<PipelineConfig
return builder;
}
@Override
public PipelineConfiguration readFrom(StreamInput in) throws IOException {
public static PipelineConfiguration readFrom(StreamInput in) throws IOException {
return new PipelineConfiguration(in.readString(), in.readBytesReference());
}
public static Diff<PipelineConfiguration> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(PipelineConfiguration::readFrom, in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(id);

View File

@ -325,9 +325,6 @@ public class Node implements Closeable {
final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool);
clusterService.addListener(scriptModule.getScriptService());
resourcesToClose.add(clusterService);
final TribeService tribeService = new TribeService(settings, clusterService, nodeId,
s -> newTribeClientNode(s, classpathPlugins));
resourcesToClose.add(tribeService);
final IngestService ingestService = new IngestService(settings, threadPool, this.environment,
scriptModule.getScriptService(), analysisModule.getAnalysisRegistry(), pluginsService.filterPlugins(IngestPlugin.class));
final ClusterInfoService clusterInfoService = newClusterInfoService(settings, clusterService, threadPool, client);
@ -349,7 +346,6 @@ public class Node implements Closeable {
settingsModule.getClusterSettings(), threadPool, pluginsService.filterPlugins(ActionPlugin.class));
modules.add(actionModule);
modules.add(new GatewayModule());
modules.add(new RepositoriesModule(this.environment, pluginsService.filterPlugins(RepositoryPlugin.class)));
CircuitBreakerService circuitBreakerService = createCircuitBreakerService(settingsModule.getSettings(),
settingsModule.getClusterSettings());
resourcesToClose.add(circuitBreakerService);
@ -361,15 +357,21 @@ public class Node implements Closeable {
indicesModule.getNamedWriteables().stream(),
searchModule.getNamedWriteables().stream(),
pluginsService.filterPlugins(Plugin.class).stream()
.flatMap(p -> p.getNamedWriteables().stream()))
.flatMap(p -> p.getNamedWriteables().stream()),
ClusterModule.getNamedWriteables().stream())
.flatMap(Function.identity()).collect(Collectors.toList());
final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(namedWriteables);
NamedXContentRegistry xContentRegistry = new NamedXContentRegistry(Stream.of(
searchModule.getNamedXContents().stream(),
pluginsService.filterPlugins(Plugin.class).stream()
.flatMap(p -> p.getNamedXContent().stream())
).flatMap(Function.identity()).collect(toList()));
final MetaStateService metaStateService = new MetaStateService(settings, nodeEnvironment);
searchModule.getNamedXContents().stream(),
pluginsService.filterPlugins(Plugin.class).stream()
.flatMap(p -> p.getNamedXContent().stream()),
ClusterModule.getNamedXWriteables().stream())
.flatMap(Function.identity()).collect(toList()));
final TribeService tribeService = new TribeService(settings, clusterService, nodeId, namedWriteableRegistry,
s -> newTribeClientNode(s, classpathPlugins));
resourcesToClose.add(tribeService);
modules.add(new RepositoriesModule(this.environment, pluginsService.filterPlugins(RepositoryPlugin.class), xContentRegistry));
final MetaStateService metaStateService = new MetaStateService(settings, nodeEnvironment, xContentRegistry);
final IndicesService indicesService = new IndicesService(settings, pluginsService, nodeEnvironment, xContentRegistry,
settingsModule.getClusterSettings(), analysisModule.getAnalysisRegistry(),
clusterModule.getIndexNameExpressionResolver(), indicesModule.getMapperRegistry(), namedWriteableRegistry,
@ -407,7 +409,7 @@ public class Node implements Closeable {
}
final DiscoveryModule discoveryModule = new DiscoveryModule(this.settings, threadPool, transportService,
networkService, clusterService, pluginsService.filterPlugins(DiscoveryPlugin.class));
namedWriteableRegistry, networkService, clusterService, pluginsService.filterPlugins(DiscoveryPlugin.class));
modules.add(b -> {
b.bind(SearchRequestParsers.class).toInstance(searchModule.getSearchRequestParsers());
b.bind(SearchExtRegistry.class).toInstance(searchModule.getSearchExtRegistry());

View File

@ -24,6 +24,7 @@ import java.util.Map;
import java.util.function.Supplier;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery;
@ -60,6 +61,7 @@ public interface DiscoveryPlugin {
* @param hostsProvider Use to find configured hosts which should be pinged for initial discovery
*/
default Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) {
return Collections.emptyMap();
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.plugins;
import java.util.Collections;
import java.util.Map;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.repositories.Repository;
@ -38,7 +39,7 @@ public interface RepositoryPlugin {
* The key of the returned {@link Map} is the type name of the repository and
* the value is a factory to construct the {@link Repository} interface.
*/
default Map<String, Repository.Factory> getRepositories(Environment env) {
default Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) {
return Collections.emptyMap();
}
}

View File

@ -28,6 +28,7 @@ import org.elasticsearch.action.admin.cluster.snapshots.status.TransportNodesSna
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.binder.LinkedBindingBuilder;
import org.elasticsearch.common.inject.multibindings.MapBinder;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.plugins.RepositoryPlugin;
import org.elasticsearch.repositories.fs.FsRepository;
@ -43,13 +44,13 @@ public class RepositoriesModule extends AbstractModule {
private final Map<String, Repository.Factory> repositoryTypes;
public RepositoriesModule(Environment env, List<RepositoryPlugin> repoPlugins) {
public RepositoriesModule(Environment env, List<RepositoryPlugin> repoPlugins, NamedXContentRegistry namedXContentRegistry) {
Map<String, Repository.Factory> factories = new HashMap<>();
factories.put(FsRepository.TYPE, (metadata) -> new FsRepository(metadata, env));
factories.put(URLRepository.TYPE, (metadata) -> new URLRepository(metadata, env));
factories.put(FsRepository.TYPE, (metadata) -> new FsRepository(metadata, env, namedXContentRegistry));
factories.put(URLRepository.TYPE, (metadata) -> new URLRepository(metadata, env, namedXContentRegistry));
for (RepositoryPlugin repoPlugin : repoPlugins) {
Map<String, Repository.Factory> newRepoTypes = repoPlugin.getRepositories(env);
Map<String, Repository.Factory> newRepoTypes = repoPlugin.getRepositories(env, namedXContentRegistry);
for (Map.Entry<String, Repository.Factory> entry : newRepoTypes.entrySet()) {
if (factories.put(entry.getKey(), entry.getValue()) != null) {
throw new IllegalArgumentException("Repository type [" + entry.getKey() + "] is already registered");

View File

@ -19,11 +19,10 @@
package org.elasticsearch.repositories.blobstore;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.blobstore.BlobContainer;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.FromXContentBuilder;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.NamedXContentRegistry.FromXContent;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
@ -41,9 +40,9 @@ public abstract class BlobStoreFormat<T extends ToXContent> {
protected final String blobNameFormat;
protected final FromXContentBuilder<T> reader;
protected final FromXContent<T> reader;
protected final ParseFieldMatcher parseFieldMatcher;
protected final NamedXContentRegistry namedXContentRegistry;
// Serialization parameters to specify correct context for metadata serialization
protected static final ToXContent.Params SNAPSHOT_ONLY_FORMAT_PARAMS;
@ -61,12 +60,11 @@ public abstract class BlobStoreFormat<T extends ToXContent> {
/**
* @param blobNameFormat format of the blobname in {@link String#format(Locale, String, Object...)} format
* @param reader the prototype object that can deserialize objects with type T
* @param parseFieldMatcher parse field matcher
*/
protected BlobStoreFormat(String blobNameFormat, FromXContentBuilder<T> reader, ParseFieldMatcher parseFieldMatcher) {
protected BlobStoreFormat(String blobNameFormat, FromXContent<T> reader, NamedXContentRegistry namedXContentRegistry) {
this.reader = reader;
this.blobNameFormat = blobNameFormat;
this.parseFieldMatcher = parseFieldMatcher;
this.namedXContentRegistry = namedXContentRegistry;
}
/**
@ -110,9 +108,8 @@ public abstract class BlobStoreFormat<T extends ToXContent> {
}
protected T read(BytesReference bytes) throws IOException {
// EMPTY is safe here because no reader calls namedObject
try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, bytes)) {
T obj = reader.fromXContent(parser, parseFieldMatcher);
try (XContentParser parser = XContentHelper.createParser(namedXContentRegistry, bytes)) {
T obj = reader.fromXContent(parser);
return obj;
}
}

View File

@ -45,7 +45,6 @@ import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.RepositoryMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.blobstore.BlobContainer;
@ -168,6 +167,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
protected final RepositoryMetaData metadata;
protected final NamedXContentRegistry namedXContentRegistry;
private static final int BUFFER_SIZE = 4096;
private static final String SNAPSHOT_PREFIX = "snap-";
@ -214,8 +215,6 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
private final boolean readOnly;
private final ParseFieldMatcher parseFieldMatcher;
private final ChecksumBlobStoreFormat<BlobStoreIndexShardSnapshot> indexShardSnapshotFormat;
private final ChecksumBlobStoreFormat<BlobStoreIndexShardSnapshots> indexShardSnapshotsFormat;
@ -226,25 +225,29 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
* @param metadata The metadata for this repository including name and settings
* @param globalSettings Settings for the node this repository object is created on
*/
protected BlobStoreRepository(RepositoryMetaData metadata, Settings globalSettings) {
protected BlobStoreRepository(RepositoryMetaData metadata, Settings globalSettings, NamedXContentRegistry namedXContentRegistry) {
super(globalSettings);
this.metadata = metadata;
parseFieldMatcher = new ParseFieldMatcher(settings);
this.namedXContentRegistry = namedXContentRegistry;
snapshotRateLimiter = getRateLimiter(metadata.settings(), "max_snapshot_bytes_per_sec", new ByteSizeValue(40, ByteSizeUnit.MB));
restoreRateLimiter = getRateLimiter(metadata.settings(), "max_restore_bytes_per_sec", new ByteSizeValue(40, ByteSizeUnit.MB));
readOnly = metadata.settings().getAsBoolean("readonly", false);
indexShardSnapshotFormat = new ChecksumBlobStoreFormat<>(SNAPSHOT_CODEC, SNAPSHOT_NAME_FORMAT, BlobStoreIndexShardSnapshot.PROTO, parseFieldMatcher, isCompress());
indexShardSnapshotsFormat = new ChecksumBlobStoreFormat<>(SNAPSHOT_INDEX_CODEC, SNAPSHOT_INDEX_NAME_FORMAT, BlobStoreIndexShardSnapshots.PROTO, parseFieldMatcher, isCompress());
indexShardSnapshotFormat = new ChecksumBlobStoreFormat<>(SNAPSHOT_CODEC, SNAPSHOT_NAME_FORMAT,
BlobStoreIndexShardSnapshot::fromXContent, namedXContentRegistry, isCompress());
indexShardSnapshotsFormat = new ChecksumBlobStoreFormat<>(SNAPSHOT_INDEX_CODEC, SNAPSHOT_INDEX_NAME_FORMAT,
BlobStoreIndexShardSnapshots::fromXContent, namedXContentRegistry, isCompress());
}
@Override
protected void doStart() {
this.snapshotsBlobContainer = blobStore().blobContainer(basePath());
ParseFieldMatcher parseFieldMatcher = new ParseFieldMatcher(settings);
globalMetaDataFormat = new ChecksumBlobStoreFormat<>(METADATA_CODEC, METADATA_NAME_FORMAT, MetaData.PROTO, parseFieldMatcher, isCompress());
indexMetaDataFormat = new ChecksumBlobStoreFormat<>(INDEX_METADATA_CODEC, METADATA_NAME_FORMAT, IndexMetaData.PROTO, parseFieldMatcher, isCompress());
snapshotFormat = new ChecksumBlobStoreFormat<>(SNAPSHOT_CODEC, SNAPSHOT_NAME_FORMAT, SnapshotInfo.PROTO, parseFieldMatcher, isCompress());
globalMetaDataFormat = new ChecksumBlobStoreFormat<>(METADATA_CODEC, METADATA_NAME_FORMAT,
MetaData::fromXContent, namedXContentRegistry, isCompress());
indexMetaDataFormat = new ChecksumBlobStoreFormat<>(INDEX_METADATA_CODEC, METADATA_NAME_FORMAT,
IndexMetaData::fromXContent, namedXContentRegistry, isCompress());
snapshotFormat = new ChecksumBlobStoreFormat<>(SNAPSHOT_CODEC, SNAPSHOT_NAME_FORMAT,
SnapshotInfo::fromXContent, namedXContentRegistry, isCompress());
}
@Override

View File

@ -23,7 +23,6 @@ import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.store.OutputStreamIndexOutput;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.blobstore.BlobContainer;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
@ -33,7 +32,8 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.store.ByteArrayIndexInput;
import org.elasticsearch.common.lucene.store.IndexOutputOutputStream;
import org.elasticsearch.common.xcontent.FromXContentBuilder;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.NamedXContentRegistry.FromXContent;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -73,8 +73,9 @@ public class ChecksumBlobStoreFormat<T extends ToXContent> extends BlobStoreForm
* @param compress true if the content should be compressed
* @param xContentType content type that should be used for write operations
*/
public ChecksumBlobStoreFormat(String codec, String blobNameFormat, FromXContentBuilder<T> reader, ParseFieldMatcher parseFieldMatcher, boolean compress, XContentType xContentType) {
super(blobNameFormat, reader, parseFieldMatcher);
public ChecksumBlobStoreFormat(String codec, String blobNameFormat, FromXContent<T> reader,
NamedXContentRegistry namedXContentRegistry, boolean compress, XContentType xContentType) {
super(blobNameFormat, reader, namedXContentRegistry);
this.xContentType = xContentType;
this.compress = compress;
this.codec = codec;
@ -86,8 +87,9 @@ public class ChecksumBlobStoreFormat<T extends ToXContent> extends BlobStoreForm
* @param reader prototype object that can deserialize T from XContent
* @param compress true if the content should be compressed
*/
public ChecksumBlobStoreFormat(String codec, String blobNameFormat, FromXContentBuilder<T> reader, ParseFieldMatcher parseFieldMatcher, boolean compress) {
this(codec, blobNameFormat, reader, parseFieldMatcher, compress, DEFAULT_X_CONTENT_TYPE);
public ChecksumBlobStoreFormat(String codec, String blobNameFormat, FromXContent<T> reader,
NamedXContentRegistry namedXContentRegistry, boolean compress) {
this(codec, blobNameFormat, reader, namedXContentRegistry, compress, DEFAULT_X_CONTENT_TYPE);
}
/**

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.blobstore.fs.FsBlobStore;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.repositories.RepositoryException;
import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
@ -72,8 +73,9 @@ public class FsRepository extends BlobStoreRepository {
/**
* Constructs a shared file system repository.
*/
public FsRepository(RepositoryMetaData metadata, Environment environment) throws IOException {
super(metadata, environment.settings());
public FsRepository(RepositoryMetaData metadata, Environment environment,
NamedXContentRegistry namedXContentRegistry) throws IOException {
super(metadata, environment.settings(), namedXContentRegistry);
String location = REPOSITORIES_LOCATION_SETTING.get(metadata.settings());
if (location.isEmpty()) {
logger.warn("the repository location is missing, it should point to a shared file system location that is available on all master and data nodes");

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.blobstore.url.URLBlobStore;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.util.URIPattern;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.repositories.RepositoryException;
import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
@ -77,8 +78,9 @@ public class URLRepository extends BlobStoreRepository {
/**
* Constructs a read-only URL-based repository
*/
public URLRepository(RepositoryMetaData metadata, Environment environment) throws IOException {
super(metadata, environment.settings());
public URLRepository(RepositoryMetaData metadata, Environment environment,
NamedXContentRegistry namedXContentRegistry) throws IOException {
super(metadata, environment.settings(), namedXContentRegistry);
if (URL_SETTING.exists(metadata.settings()) == false && REPOSITORIES_URL_SETTING.exists(settings) == false) {
throw new RepositoryException(metadata.name(), "missing url");

View File

@ -22,6 +22,7 @@ import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.DiffableUtils;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.bytes.BytesArray;
@ -45,7 +46,6 @@ import java.util.Map;
public final class ScriptMetaData implements MetaData.Custom {
public static final String TYPE = "stored_scripts";
public static final ScriptMetaData PROTO = new ScriptMetaData(Collections.emptyMap());
private final Map<String, ScriptAsBytes> scripts;
@ -108,12 +108,11 @@ public final class ScriptMetaData implements MetaData.Custom {
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@Override
public ScriptMetaData fromXContent(XContentParser parser) throws IOException {
public static ScriptMetaData fromXContent(XContentParser parser) throws IOException {
Map<String, ScriptAsBytes> scripts = new HashMap<>();
String key = null;
for (Token token = parser.nextToken(); token != Token.END_OBJECT; token = parser.nextToken()) {
@ -136,16 +135,14 @@ public final class ScriptMetaData implements MetaData.Custom {
return MetaData.ALL_CONTEXTS;
}
@Override
public ScriptMetaData readFrom(StreamInput in) throws IOException {
public ScriptMetaData(StreamInput in) throws IOException {
int size = in.readVInt();
Map<String, ScriptAsBytes> scripts = new HashMap<>();
this.scripts = new HashMap<>();
for (int i = 0; i < size; i++) {
String languageAndId = in.readString();
BytesReference script = in.readBytesReference();
scripts.put(languageAndId, new ScriptAsBytes(script));
}
return new ScriptMetaData(scripts);
}
@Override
@ -170,8 +167,7 @@ public final class ScriptMetaData implements MetaData.Custom {
return new ScriptMetadataDiff((ScriptMetaData) before, this);
}
@Override
public Diff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {
public static NamedDiff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {
return new ScriptMetadataDiff(in);
}
@ -237,7 +233,7 @@ public final class ScriptMetaData implements MetaData.Custom {
}
}
static final class ScriptMetadataDiff implements Diff<MetaData.Custom> {
static final class ScriptMetadataDiff implements NamedDiff<MetaData.Custom> {
final Diff<Map<String, ScriptAsBytes>> pipelines;
@ -246,7 +242,8 @@ public final class ScriptMetaData implements MetaData.Custom {
}
public ScriptMetadataDiff(StreamInput in) throws IOException {
pipelines = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), new ScriptAsBytes(null));
pipelines = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), ScriptAsBytes::new,
ScriptAsBytes::readDiffFrom);
}
@Override
@ -258,6 +255,11 @@ public final class ScriptMetaData implements MetaData.Custom {
public void writeTo(StreamOutput out) throws IOException {
pipelines.writeTo(out);
}
@Override
public String getWriteableName() {
return TYPE;
}
}
static final class ScriptAsBytes extends AbstractDiffable<ScriptAsBytes> {
@ -273,9 +275,12 @@ public final class ScriptMetaData implements MetaData.Custom {
out.writeBytesReference(script);
}
@Override
public ScriptAsBytes readFrom(StreamInput in) throws IOException {
return new ScriptAsBytes(in.readBytesReference());
public ScriptAsBytes(StreamInput in) throws IOException {
this(in.readBytesReference());
}
public static Diff<ScriptAsBytes> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(ScriptAsBytes::new, in);
}
@Override

View File

@ -27,7 +27,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.joda.Joda;
import org.elasticsearch.common.xcontent.FromXContentBuilder;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -42,9 +41,8 @@ import java.util.Objects;
/**
* Information about a snapshot
*/
public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent, FromXContentBuilder<SnapshotInfo>, Writeable {
public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent, Writeable {
public static final SnapshotInfo PROTO = new SnapshotInfo(new SnapshotId("", ""), Collections.emptyList(), 0);
public static final String CONTEXT_MODE_PARAM = "context_mode";
public static final String CONTEXT_MODE_SNAPSHOT = "SNAPSHOT";
private static final FormatDateTimeFormatter DATE_TIME_FORMATTER = Joda.forPattern("strictDateOptionalTime");
@ -373,11 +371,6 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
return builder;
}
@Override
public SnapshotInfo fromXContent(final XContentParser parser, final ParseFieldMatcher matcher) throws IOException {
return fromXContent(parser);
}
/**
* This method creates a SnapshotInfo from internal x-content. It does not
* handle x-content written with the external version as external x-content

View File

@ -44,6 +44,10 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.hash.MurmurHash3;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.regex.Regex;
@ -61,6 +65,7 @@ import org.elasticsearch.node.Node;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.transport.TransportSettings;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
@ -207,10 +212,13 @@ public class TribeService extends AbstractLifecycleComponent {
private final List<Node> nodes = new CopyOnWriteArrayList<>();
private final NamedWriteableRegistry namedWriteableRegistry;
public TribeService(Settings settings, ClusterService clusterService, final String tribeNodeId,
Function<Settings, Node> clientNodeBuilder) {
NamedWriteableRegistry namedWriteableRegistry, Function<Settings, Node> clientNodeBuilder) {
super(settings);
this.clusterService = clusterService;
this.namedWriteableRegistry = namedWriteableRegistry;
Map<String, Settings> nodesSettings = new HashMap<>(settings.getGroups("tribe", true));
nodesSettings.remove("blocks"); // remove prefix settings that don't indicate a client
nodesSettings.remove("on_conflict"); // remove prefix settings that don't indicate a client
@ -512,7 +520,7 @@ public class TribeService extends AbstractLifecycleComponent {
.map(ClusterState::metaData)
.map(clusterMetaData -> ((MetaData.Custom) clusterMetaData.custom(customMetaDataType)))
.filter(custom1 -> custom1 != null && custom1 instanceof MergableCustomMetaData)
.map(custom2 -> (MergableCustomMetaData) custom2)
.map(custom2 -> (MergableCustomMetaData) marshal(custom2))
.collect(Collectors.toList())
);
for (String changedCustomMetaDataType : changedCustomMetaDataTypeSet) {
@ -577,4 +585,21 @@ public class TribeService extends AbstractLifecycleComponent {
}
return changedCustomMetaDataMap;
}
/**
* Since custom metadata can be loaded by a plugin class loader that resides in a sub-node, we need to
* marshal this object into something the tribe node can work with
*/
private MetaData.Custom marshal(MetaData.Custom custom) {
try (BytesStreamOutput bytesStreamOutput = new BytesStreamOutput()){
bytesStreamOutput.writeNamedWriteable(custom);
try(StreamInput input = bytesStreamOutput.bytes().streamInput()) {
StreamInput namedInput = new NamedWriteableAwareStreamInput(input, namedWriteableRegistry);
MetaData.Custom marshaled = namedInput.readNamedWriteable(MetaData.Custom.class);
return marshaled;
}
} catch (IOException ex) {
throw new IllegalStateException("cannot marshal object with type " + custom.getWriteableName() + " to tribe node");
}
}
}

View File

@ -29,6 +29,7 @@ import org.elasticsearch.action.support.nodes.BaseNodesRequest;
import org.elasticsearch.action.support.nodes.BaseNodesResponse;
import org.elasticsearch.action.support.nodes.TransportNodesAction;
import org.elasticsearch.action.support.replication.ClusterStateCreationUtils;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNode;
@ -171,7 +172,8 @@ public abstract class TaskManagerTestCase extends ESTestCase {
clusterService = createClusterService(threadPool);
transportService = new TransportService(settings,
new MockTcpTransport(settings, threadPool, BigArrays.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService(),
new NamedWriteableRegistry(Collections.emptyList()), new NetworkService(settings, Collections.emptyList())),
new NamedWriteableRegistry(ClusterModule.getNamedWriteables()),
new NetworkService(settings, Collections.emptyList())),
threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, null) {
@Override
protected TaskManager createTaskManager() {

View File

@ -544,7 +544,7 @@ public class OldIndexBackwardsCompatibilityIT extends ESIntegTestCase {
String indexName = indexFile.replace(".zip", "").toLowerCase(Locale.ROOT).replace("unsupported-", "index-");
Path nodeDir = getNodeDir(indexFile);
logger.info("Parsing cluster state files from index [{}]", indexName);
final MetaData metaData = globalFormat.loadLatestState(logger, nodeDir);
final MetaData metaData = globalFormat.loadLatestState(logger, xContentRegistry(), nodeDir);
assertNotNull(metaData);
final Version version = Version.fromString(indexName.substring("index-".length()));
@ -555,7 +555,7 @@ public class OldIndexBackwardsCompatibilityIT extends ESIntegTestCase {
dataDir = nodeDir.getParent();
}
final Path indexDir = getIndexDir(logger, indexName, indexFile, dataDir);
assertNotNull(indexFormat.loadLatestState(logger, indexDir));
assertNotNull(indexFormat.loadLatestState(logger, xContentRegistry(), indexDir));
}
}

View File

@ -244,7 +244,7 @@ public class ClusterChangedEventTests extends ESTestCase {
event = new ClusterChangedEvent("_na_", originalState, nextState);
Set<String> changedCustomMetaDataTypeSet = event.changedCustomMetaDataSet();
assertTrue(changedCustomMetaDataTypeSet.size() == 1);
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData1.type()));
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData1.getWriteableName()));
// next state has same custom metadata
nextState = nextState(originalState, Collections.singletonList(customMetaData1));
@ -263,14 +263,14 @@ public class ClusterChangedEventTests extends ESTestCase {
event = new ClusterChangedEvent("_na_", stateWithCustomMetaData, nextState);
changedCustomMetaDataTypeSet = event.changedCustomMetaDataSet();
assertTrue(changedCustomMetaDataTypeSet.size() == 1);
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData1.type()));
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData1.getWriteableName()));
// next state updates custom metadata
nextState = nextState(stateWithCustomMetaData, Collections.singletonList(new CustomMetaData1("data1")));
event = new ClusterChangedEvent("_na_", stateWithCustomMetaData, nextState);
changedCustomMetaDataTypeSet = event.changedCustomMetaDataSet();
assertTrue(changedCustomMetaDataTypeSet.size() == 1);
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData1.type()));
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData1.getWriteableName()));
// next state adds new custom metadata type
CustomMetaData2 customMetaData2 = new CustomMetaData2("data2");
@ -278,15 +278,15 @@ public class ClusterChangedEventTests extends ESTestCase {
event = new ClusterChangedEvent("_na_", stateWithCustomMetaData, nextState);
changedCustomMetaDataTypeSet = event.changedCustomMetaDataSet();
assertTrue(changedCustomMetaDataTypeSet.size() == 1);
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData2.type()));
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData2.getWriteableName()));
// next state adds two custom metadata type
nextState = nextState(originalState, Arrays.asList(customMetaData1, customMetaData2));
event = new ClusterChangedEvent("_na_", originalState, nextState);
changedCustomMetaDataTypeSet = event.changedCustomMetaDataSet();
assertTrue(changedCustomMetaDataTypeSet.size() == 2);
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData2.type()));
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData1.type()));
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData2.getWriteableName()));
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData1.getWriteableName()));
// next state removes two custom metadata type
nextState = originalState;
@ -294,25 +294,17 @@ public class ClusterChangedEventTests extends ESTestCase {
nextState(originalState, Arrays.asList(customMetaData1, customMetaData2)), nextState);
changedCustomMetaDataTypeSet = event.changedCustomMetaDataSet();
assertTrue(changedCustomMetaDataTypeSet.size() == 2);
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData2.type()));
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData1.type()));
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData2.getWriteableName()));
assertTrue(changedCustomMetaDataTypeSet.contains(customMetaData1.getWriteableName()));
}
private static class CustomMetaData2 extends TestCustomMetaData {
static {
MetaData.registerPrototype("2", new CustomMetaData2(""));
}
protected CustomMetaData2(String data) {
super(data);
}
@Override
protected TestCustomMetaData newTestCustomMetaData(String data) {
return new CustomMetaData2(data);
}
@Override
public String type() {
public String getWriteableName() {
return "2";
}
@ -323,20 +315,12 @@ public class ClusterChangedEventTests extends ESTestCase {
}
private static class CustomMetaData1 extends TestCustomMetaData {
static {
MetaData.registerPrototype("1", new CustomMetaData1(""));
}
protected CustomMetaData1(String data) {
super(data);
}
@Override
protected TestCustomMetaData newTestCustomMetaData(String data) {
return new CustomMetaData1(data);
}
@Override
public String type() {
public String getWriteableName() {
return "1";
}
@ -378,7 +362,7 @@ public class ClusterChangedEventTests extends ESTestCase {
}
}
for (TestCustomMetaData testCustomMetaData : customMetaDataList) {
metaDataBuilder.putCustom(testCustomMetaData.type(), testCustomMetaData);
metaDataBuilder.putCustom(testCustomMetaData.getWriteableName(), testCustomMetaData);
}
builder.metaData(metaDataBuilder);
return builder.build();

View File

@ -31,6 +31,8 @@ import org.elasticsearch.cluster.metadata.IndexTemplateMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.RepositoriesMetaData;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
@ -75,13 +77,15 @@ import static org.hamcrest.Matchers.is;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 0, numClientNodes = 0)
public class ClusterStateDiffIT extends ESIntegTestCase {
public void testClusterStateDiffSerialization() throws Exception {
NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(ClusterModule.getNamedWriteables());
DiscoveryNode masterNode = new DiscoveryNode("master", buildNewFakeTransportAddress(),
emptyMap(), emptySet(), Version.CURRENT);
DiscoveryNode otherNode = new DiscoveryNode("other", buildNewFakeTransportAddress(),
emptyMap(), emptySet(), Version.CURRENT);
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(masterNode).add(otherNode).localNodeId(masterNode.getId()).build();
ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build();
ClusterState clusterStateFromDiffs = ClusterState.Builder.fromBytes(ClusterState.Builder.toBytes(clusterState), otherNode);
ClusterState clusterStateFromDiffs =
ClusterState.Builder.fromBytes(ClusterState.Builder.toBytes(clusterState), otherNode, namedWriteableRegistry);
int iterationCount = randomIntBetween(10, 300);
for (int iteration = 0; iteration < iterationCount; iteration++) {
@ -117,7 +121,8 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
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().getLocalNode());
clusterStateFromDiffs = ClusterState.Builder.fromBytes(ClusterState.Builder.toBytes(clusterState),
previousClusterStateFromDiffs.nodes().getLocalNode(), namedWriteableRegistry);
} else {
// Update cluster states using diffs
Diff<ClusterState> diffBeforeSerialization = clusterState.diff(previousClusterState);
@ -126,7 +131,8 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
byte[] diffBytes = BytesReference.toBytes(os.bytes());
Diff<ClusterState> diff;
try (StreamInput input = StreamInput.wrap(diffBytes)) {
diff = previousClusterStateFromDiffs.readDiffFrom(input);
StreamInput namedInput = new NamedWriteableAwareStreamInput(input, namedWriteableRegistry);
diff = ClusterState.readDiffFrom(namedInput, previousClusterStateFromDiffs.nodes().getLocalNode());
clusterStateFromDiffs = diff.apply(previousClusterStateFromDiffs);
}
}
@ -618,7 +624,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
@Override
public MetaData.Builder put(MetaData.Builder builder, MetaData.Custom part) {
return builder.putCustom(part.type(), part);
return builder.putCustom(part.getWriteableName(), part);
}
@Override
@ -660,7 +666,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
@Override
public ClusterState.Builder put(ClusterState.Builder builder, ClusterState.Custom part) {
return builder.putCustom(part.type(), part);
return builder.putCustom(part.getWriteableName(), part);
}
@Override

View File

@ -57,7 +57,7 @@ public class IndexGraveyardTests extends ESTestCase {
final IndexGraveyard graveyard = createRandom();
final BytesStreamOutput out = new BytesStreamOutput();
graveyard.writeTo(out);
assertThat(IndexGraveyard.fromStream(out.bytes().streamInput()), equalTo(graveyard));
assertThat(new IndexGraveyard(out.bytes().streamInput()), equalTo(graveyard));
}
public void testXContent() throws IOException {
@ -68,7 +68,7 @@ public class IndexGraveyardTests extends ESTestCase {
builder.endObject();
XContentParser parser = createParser(JsonXContent.jsonXContent, builder.bytes());
parser.nextToken(); // the beginning of the parser
assertThat(IndexGraveyard.PROTO.fromXContent(parser), equalTo(graveyard));
assertThat(IndexGraveyard.fromXContent(parser), equalTo(graveyard));
}
public void testAddTombstones() {

View File

@ -53,7 +53,7 @@ public class IndexMetaDataTests extends ESTestCase {
metaData.toXContent(builder, ToXContent.EMPTY_PARAMS);
builder.endObject();
XContentParser parser = createParser(JsonXContent.jsonXContent, builder.bytes());
final IndexMetaData fromXContentMeta = IndexMetaData.PROTO.fromXContent(parser, null);
final IndexMetaData fromXContentMeta = IndexMetaData.fromXContent(parser);
assertEquals(metaData, fromXContentMeta);
assertEquals(metaData.hashCode(), fromXContentMeta.hashCode());
@ -67,7 +67,7 @@ public class IndexMetaDataTests extends ESTestCase {
final BytesStreamOutput out = new BytesStreamOutput();
metaData.writeTo(out);
IndexMetaData deserialized = IndexMetaData.PROTO.readFrom(out.bytes().streamInput());
IndexMetaData deserialized = IndexMetaData.readFrom(out.bytes().streamInput());
assertEquals(metaData, deserialized);
assertEquals(metaData.hashCode(), deserialized.hashCode());

View File

@ -58,7 +58,7 @@ public class IndexTemplateMetaDataTests extends ESTestCase {
try (StreamInput in = bytes.streamInput()) {
in.setVersion(Version.V_5_0_0);
IndexTemplateMetaData readMetaData = IndexTemplateMetaData.Builder.readFrom(in);
IndexTemplateMetaData readMetaData = IndexTemplateMetaData.readFrom(in);
assertEquals(0, in.available());
assertEquals(metaData.getName(), readMetaData.getName());
assertEquals(metaData.getPatterns(), readMetaData.getPatterns());

View File

@ -20,9 +20,13 @@
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -173,7 +177,7 @@ public class MetaDataTests extends ESTestCase {
originalMeta.toXContent(builder, ToXContent.EMPTY_PARAMS);
builder.endObject();
XContentParser parser = createParser(JsonXContent.jsonXContent, builder.bytes());
final MetaData fromXContentMeta = MetaData.PROTO.fromXContent(parser, null);
final MetaData fromXContentMeta = MetaData.fromXContent(parser);
assertThat(fromXContentMeta.indexGraveyard(), equalTo(originalMeta.indexGraveyard()));
}
@ -182,7 +186,10 @@ public class MetaDataTests extends ESTestCase {
final MetaData originalMeta = MetaData.builder().indexGraveyard(graveyard).build();
final BytesStreamOutput out = new BytesStreamOutput();
originalMeta.writeTo(out);
final MetaData fromStreamMeta = MetaData.PROTO.readFrom(out.bytes().streamInput());
NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(ClusterModule.getNamedWriteables());
final MetaData fromStreamMeta = MetaData.readFrom(
new NamedWriteableAwareStreamInput(out.bytes().streamInput(), namedWriteableRegistry)
);
assertThat(fromStreamMeta.indexGraveyard(), equalTo(fromStreamMeta.indexGraveyard()));
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.cluster.serialization;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ESAllocationTestCase;
@ -29,9 +30,12 @@ import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import java.util.Collections;
import static org.hamcrest.Matchers.equalTo;
public class ClusterSerializationTests extends ESAllocationTestCase {
@ -52,7 +56,8 @@ public class ClusterSerializationTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService();
clusterState = ClusterState.builder(clusterState).routingTable(strategy.reroute(clusterState, "reroute").routingTable()).build();
ClusterState serializedClusterState = ClusterState.Builder.fromBytes(ClusterState.Builder.toBytes(clusterState), newNode("node1"));
ClusterState serializedClusterState = ClusterState.Builder.fromBytes(ClusterState.Builder.toBytes(clusterState), newNode("node1"),
new NamedWriteableRegistry(ClusterModule.getNamedWriteables()));
assertThat(serializedClusterState.getClusterName().value(), equalTo(clusterState.getClusterName().value()));
@ -79,7 +84,7 @@ public class ClusterSerializationTests extends ESAllocationTestCase {
BytesStreamOutput outStream = new BytesStreamOutput();
source.writeTo(outStream);
StreamInput inStream = outStream.bytes().streamInput();
RoutingTable target = RoutingTable.Builder.readFrom(inStream);
RoutingTable target = RoutingTable.readFrom(inStream);
assertThat(target.toString(), equalTo(source.toString()));
}

View File

@ -65,7 +65,7 @@ public class DiffableTests extends ESTestCase {
@Override
protected MapDiff readDiff(StreamInput in) throws IOException {
return useProtoForDiffableSerialization
? DiffableUtils.readJdkMapDiff(in, keySerializer, TestDiffable.PROTO)
? DiffableUtils.readJdkMapDiff(in, keySerializer, TestDiffable::readFrom, TestDiffable::readDiffFrom)
: DiffableUtils.readJdkMapDiff(in, keySerializer, diffableValueSerializer());
}
}.execute();
@ -113,7 +113,7 @@ public class DiffableTests extends ESTestCase {
@Override
protected MapDiff readDiff(StreamInput in) throws IOException {
return useProtoForDiffableSerialization
? DiffableUtils.readImmutableOpenMapDiff(in, keySerializer, TestDiffable.PROTO)
? DiffableUtils.readImmutableOpenMapDiff(in, keySerializer, TestDiffable::readFrom, TestDiffable::readDiffFrom)
: DiffableUtils.readImmutableOpenMapDiff(in, keySerializer, diffableValueSerializer());
}
}.execute();
@ -161,7 +161,7 @@ public class DiffableTests extends ESTestCase {
@Override
protected MapDiff readDiff(StreamInput in) throws IOException {
return useProtoForDiffableSerialization
? DiffableUtils.readImmutableOpenIntMapDiff(in, keySerializer, TestDiffable.PROTO)
? DiffableUtils.readImmutableOpenIntMapDiff(in, keySerializer, TestDiffable::readFrom, TestDiffable::readDiffFrom)
: DiffableUtils.readImmutableOpenIntMapDiff(in, keySerializer, diffableValueSerializer());
}
}.execute();
@ -398,7 +398,7 @@ public class DiffableTests extends ESTestCase {
@Override
public Diff<TestDiffable> readDiff(StreamInput in, K key) throws IOException {
return AbstractDiffable.readDiffFrom(TestDiffable.PROTO, in);
return AbstractDiffable.readDiffFrom(TestDiffable::readFrom, in);
}
};
}
@ -419,8 +419,6 @@ public class DiffableTests extends ESTestCase {
public static class TestDiffable extends AbstractDiffable<TestDiffable> {
public static final TestDiffable PROTO = new TestDiffable("");
private final String value;
public TestDiffable(String value) {
@ -431,11 +429,14 @@ public class DiffableTests extends ESTestCase {
return value;
}
@Override
public TestDiffable readFrom(StreamInput in) throws IOException {
public static TestDiffable readFrom(StreamInput in) throws IOException {
return new TestDiffable(in.readString());
}
public static Diff<TestDiffable> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(TestDiffable::readFrom, in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(value);

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.gateway.MetaDataStateFormat;
@ -223,7 +224,7 @@ public class IndexFolderUpgraderTests extends ESTestCase {
assertEquals(indexFolders.size(), 1);
// ensure index metadata is moved
IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger,
IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY,
nodeEnvironment.resolveIndexFolder(indexFolders.iterator().next()));
assertNotNull(indexMetaData);
Index index = indexMetaData.getIndex();
@ -265,7 +266,8 @@ public class IndexFolderUpgraderTests extends ESTestCase {
int numIdxFiles, int numTranslogFiles) throws IOException {
final Index index = indexSettings.getIndex();
// ensure index state can be loaded
IndexMetaData loadLatestState = IndexMetaData.FORMAT.loadLatestState(logger, nodeEnv.indexPaths(index));
IndexMetaData loadLatestState = IndexMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY,
nodeEnv.indexPaths(index));
assertNotNull(loadLatestState);
assertEquals(loadLatestState.getIndex(), index);
for (int shardId = 0; shardId < indexSettings.getNumberOfShards(); shardId++) {

View File

@ -31,6 +31,7 @@ import java.util.function.Supplier;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
@ -55,6 +56,7 @@ import static org.mockito.Mockito.when;
public class DiscoveryModuleTests extends ESTestCase {
private TransportService transportService;
private NamedWriteableRegistry namedWriteableRegistry;
private ClusterService clusterService;
private ThreadPool threadPool;
@ -71,6 +73,7 @@ public class DiscoveryModuleTests extends ESTestCase {
Map<String, Supplier<Discovery>> impl();
@Override
default Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) {
return impl();
}
@ -80,6 +83,7 @@ public class DiscoveryModuleTests extends ESTestCase {
public void setupDummyServices() {
transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, null, null);
clusterService = mock(ClusterService.class);
namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
when(clusterService.getClusterSettings()).thenReturn(clusterSettings);
threadPool = mock(ThreadPool.class);
@ -91,7 +95,7 @@ public class DiscoveryModuleTests extends ESTestCase {
}
private DiscoveryModule newModule(Settings settings, List<DiscoveryPlugin> plugins) {
return new DiscoveryModule(settings, threadPool, transportService, null, clusterService, plugins);
return new DiscoveryModule(settings, threadPool, transportService, namedWriteableRegistry, null, clusterService, plugins);
}
public void testDefaults() {

View File

@ -23,6 +23,7 @@ import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener;
@ -36,6 +37,7 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
@ -248,9 +250,11 @@ public class PublishClusterStateActionTests extends ESTestCase {
) {
DiscoverySettings discoverySettings =
new DiscoverySettings(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(ClusterModule.getNamedWriteables());
return new MockPublishAction(
settings,
transportService,
namedWriteableRegistry,
clusterStateSupplier,
listener,
discoverySettings,
@ -873,10 +877,10 @@ public class PublishClusterStateActionTests extends ESTestCase {
AtomicBoolean timeoutOnCommit = new AtomicBoolean();
AtomicBoolean errorOnCommit = new AtomicBoolean();
public MockPublishAction(Settings settings, TransportService transportService,
public MockPublishAction(Settings settings, TransportService transportService, NamedWriteableRegistry namedWriteableRegistry,
Supplier<ClusterState> clusterStateSupplier, NewPendingClusterStateListener listener,
DiscoverySettings discoverySettings, ClusterName clusterName) {
super(settings, transportService, clusterStateSupplier, listener, discoverySettings, clusterName);
super(settings, transportService, namedWriteableRegistry, clusterStateSupplier, listener, discoverySettings, clusterName);
}
@Override

View File

@ -236,12 +236,7 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
}
@Override
protected TestCustomMetaData newTestCustomMetaData(String data) {
return new CustomMetaData(data);
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}

View File

@ -36,6 +36,7 @@ import org.apache.lucene.util.IOUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.replication.ClusterStateCreationUtils;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -50,6 +51,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery;
@ -289,7 +291,8 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
}
private ZenDiscovery buildZenDiscovery(Settings settings, TransportService service, ClusterService clusterService, ThreadPool threadPool) {
ZenDiscovery zenDiscovery = new ZenDiscovery(settings, threadPool, service, clusterService, Collections::emptyList);
ZenDiscovery zenDiscovery = new ZenDiscovery(settings, threadPool, service, new NamedWriteableRegistry(ClusterModule.getNamedWriteables()),
clusterService, Collections::emptyList);
zenDiscovery.start();
return zenDiscovery;
}

View File

@ -47,7 +47,7 @@ public class DanglingIndicesStateTests extends ESTestCase {
public void testCleanupWhenEmpty() throws Exception {
try (NodeEnvironment env = newNodeEnvironment()) {
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env);
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env, xContentRegistry());
DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
assertTrue(danglingState.getDanglingIndices().isEmpty());
@ -58,7 +58,7 @@ public class DanglingIndicesStateTests extends ESTestCase {
}
public void testDanglingIndicesDiscovery() throws Exception {
try (NodeEnvironment env = newNodeEnvironment()) {
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env);
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env, xContentRegistry());
DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
assertTrue(danglingState.getDanglingIndices().isEmpty());
@ -76,7 +76,7 @@ public class DanglingIndicesStateTests extends ESTestCase {
public void testInvalidIndexFolder() throws Exception {
try (NodeEnvironment env = newNodeEnvironment()) {
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env);
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env, xContentRegistry());
DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
MetaData metaData = MetaData.builder().build();
@ -100,7 +100,7 @@ public class DanglingIndicesStateTests extends ESTestCase {
public void testDanglingProcessing() throws Exception {
try (NodeEnvironment env = newNodeEnvironment()) {
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env);
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env, xContentRegistry());
DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
MetaData metaData = MetaData.builder().build();
@ -144,7 +144,7 @@ public class DanglingIndicesStateTests extends ESTestCase {
public void testDanglingIndicesNotImportedWhenTombstonePresent() throws Exception {
try (NodeEnvironment env = newNodeEnvironment()) {
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env);
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env, xContentRegistry());
DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
final Settings.Builder settings = Settings.builder().put(indexSettings).put(IndexMetaData.SETTING_INDEX_UUID, "test1UUID");

View File

@ -61,12 +61,6 @@ import static org.hamcrest.Matchers.equalTo;
*/
public class GatewayMetaStateTests extends ESAllocationTestCase {
@Before
public void setup() {
MetaData.registerPrototype(CustomMetaData1.TYPE, new CustomMetaData1(""));
MetaData.registerPrototype(CustomMetaData2.TYPE, new CustomMetaData2(""));
}
ClusterChangedEvent generateEvent(boolean initializing, boolean versionChanged, boolean masterEligible) {
//ridiculous settings to make sure we don't run into uninitialized because fo default
AllocationService strategy = createAllocationService(Settings.builder()
@ -409,12 +403,7 @@ public class GatewayMetaStateTests extends ESAllocationTestCase {
}
@Override
protected TestCustomMetaData newTestCustomMetaData(String data) {
return new CustomMetaData1(data);
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@ -432,12 +421,7 @@ public class GatewayMetaStateTests extends ESAllocationTestCase {
}
@Override
protected TestCustomMetaData newTestCustomMetaData(String data) {
return new CustomMetaData2(data);
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@ -450,7 +434,7 @@ public class GatewayMetaStateTests extends ESAllocationTestCase {
private static MetaData randomMetaData(TestCustomMetaData... customMetaDatas) {
MetaData.Builder builder = MetaData.builder();
for (TestCustomMetaData customMetaData : customMetaDatas) {
builder.putCustom(customMetaData.type(), customMetaData);
builder.putCustom(customMetaData.getWriteableName(), customMetaData);
}
for (int i = 0; i < randomIntBetween(1, 5); i++) {
builder.put(

View File

@ -35,6 +35,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -89,7 +90,7 @@ public class MetaDataStateFormatTests extends ESTestCase {
assertThat(resource, notNullValue());
Path dst = tmp.resolve("global-3.st");
Files.copy(resource, dst);
MetaData read = format.read(dst);
MetaData read = format.read(xContentRegistry(), dst);
assertThat(read, notNullValue());
assertThat(read.clusterUUID(), equalTo("3O1tDF1IRB6fSJ-GrTMUtg"));
// indices are empty since they are serialized separately
@ -114,7 +115,7 @@ public class MetaDataStateFormatTests extends ESTestCase {
list = content("foo-*", stateDir);
assertEquals(list.length, 1);
assertThat(list[0].getFileName().toString(), equalTo("foo-" + id + ".st"));
DummyState read = format.read(list[0]);
DummyState read = format.read(NamedXContentRegistry.EMPTY, list[0]);
assertThat(read, equalTo(state));
}
final int version2 = between(version, Integer.MAX_VALUE);
@ -130,7 +131,7 @@ public class MetaDataStateFormatTests extends ESTestCase {
list = content("foo-*", stateDir);
assertEquals(list.length,1);
assertThat(list[0].getFileName().toString(), equalTo("foo-"+ (id+1) + ".st"));
DummyState read = format.read(list[0]);
DummyState read = format.read(NamedXContentRegistry.EMPTY, list[0]);
assertThat(read, equalTo(state2));
}
@ -156,7 +157,7 @@ public class MetaDataStateFormatTests extends ESTestCase {
list = content("foo-*", stateDir);
assertEquals(list.length, 1);
assertThat(list[0].getFileName().toString(), equalTo("foo-" + id + ".st"));
DummyState read = format.read(list[0]);
DummyState read = format.read(NamedXContentRegistry.EMPTY, list[0]);
assertThat(read, equalTo(state));
}
}
@ -180,12 +181,12 @@ public class MetaDataStateFormatTests extends ESTestCase {
list = content("foo-*", stateDir);
assertEquals(list.length, 1);
assertThat(list[0].getFileName().toString(), equalTo("foo-" + id + ".st"));
DummyState read = format.read(list[0]);
DummyState read = format.read(NamedXContentRegistry.EMPTY, list[0]);
assertThat(read, equalTo(state));
// now corrupt it
corruptFile(list[0], logger);
try {
format.read(list[0]);
format.read(NamedXContentRegistry.EMPTY, list[0]);
fail("corrupted file");
} catch (CorruptStateException ex) {
// expected
@ -272,7 +273,7 @@ public class MetaDataStateFormatTests extends ESTestCase {
}
List<Path> dirList = Arrays.asList(dirs);
Collections.shuffle(dirList, random());
MetaData loadedMetaData = format.loadLatestState(logger, dirList.toArray(new Path[0]));
MetaData loadedMetaData = format.loadLatestState(logger, xContentRegistry(), dirList.toArray(new Path[0]));
MetaData latestMetaData = meta.get(numStates-1);
assertThat(loadedMetaData.clusterUUID(), not(equalTo("_na_")));
assertThat(loadedMetaData.clusterUUID(), equalTo(latestMetaData.clusterUUID()));
@ -299,7 +300,7 @@ public class MetaDataStateFormatTests extends ESTestCase {
MetaDataStateFormatTests.corruptFile(file, logger);
}
try {
format.loadLatestState(logger, dirList.toArray(new Path[0]));
format.loadLatestState(logger, xContentRegistry(), dirList.toArray(new Path[0]));
fail("latest version can not be read");
} catch (ElasticsearchException ex) {
assertThat(ExceptionsHelper.unwrap(ex, CorruptStateException.class), notNullValue());

View File

@ -39,7 +39,7 @@ public class MetaStateServiceTests extends ESTestCase {
public void testWriteLoadIndex() throws Exception {
try (NodeEnvironment env = newNodeEnvironment()) {
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env);
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env, xContentRegistry());
IndexMetaData index = IndexMetaData.builder("test1").settings(indexSettings).build();
metaStateService.writeIndex("test_write", index);
@ -49,14 +49,14 @@ public class MetaStateServiceTests extends ESTestCase {
public void testLoadMissingIndex() throws Exception {
try (NodeEnvironment env = newNodeEnvironment()) {
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env);
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env, xContentRegistry());
assertThat(metaStateService.loadIndexState(new Index("test1", "test1UUID")), nullValue());
}
}
public void testWriteLoadGlobal() throws Exception {
try (NodeEnvironment env = newNodeEnvironment()) {
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env);
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env, xContentRegistry());
MetaData metaData = MetaData.builder()
.persistentSettings(Settings.builder().put("test1", "value1").build())
@ -68,7 +68,7 @@ public class MetaStateServiceTests extends ESTestCase {
public void testWriteGlobalStateWithIndexAndNoIndexIsLoaded() throws Exception {
try (NodeEnvironment env = newNodeEnvironment()) {
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env);
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env, xContentRegistry());
MetaData metaData = MetaData.builder()
.persistentSettings(Settings.builder().put("test1", "value1").build())
@ -84,7 +84,7 @@ public class MetaStateServiceTests extends ESTestCase {
public void testLoadGlobal() throws Exception {
try (NodeEnvironment env = newNodeEnvironment()) {
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env);
MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env, xContentRegistry());
IndexMetaData index = IndexMetaData.builder("test1").settings(indexSettings).build();
MetaData metaData = MetaData.builder()

View File

@ -2158,7 +2158,7 @@ public class InternalEngineTests extends ESTestCase {
final long size = Files.size(tlogFile);
logger.debug("upgrading index {} file: {} size: {}", indexName, tlogFiles[0].getFileName(), size);
Directory directory = newFSDirectory(src.resolve("0").resolve("index"));
final IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger, src);
final IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger, xContentRegistry(), src);
final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData);
final Store store = createStore(indexSettings, directory);
final int iters = randomIntBetween(0, 2);

View File

@ -62,6 +62,7 @@ import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.VersionType;
@ -139,7 +140,7 @@ import static org.hamcrest.Matchers.nullValue;
public class IndexShardTests extends IndexShardTestCase {
public static ShardStateMetaData load(Logger logger, Path... shardPaths) throws IOException {
return ShardStateMetaData.FORMAT.loadLatestState(logger, shardPaths);
return ShardStateMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY, shardPaths);
}
public static void write(ShardStateMetaData shardStateMetaData,

View File

@ -58,7 +58,7 @@ public class IngestMetadataTests extends ESTestCase {
builder.endObject();
XContentBuilder shuffled = shuffleXContent(builder);
final XContentParser parser = createParser(shuffled);
MetaData.Custom custom = ingestMetadata.fromXContent(parser);
MetaData.Custom custom = IngestMetadata.fromXContent(parser);
assertTrue(custom instanceof IngestMetadata);
IngestMetadata m = (IngestMetadata) custom;
assertEquals(2, m.getPipelines().size());

View File

@ -80,7 +80,7 @@ public class ScriptMetaDataTests extends ESTestCase {
XContentParser parser = createParser(xContentBuilder);
parser.nextToken();
ScriptMetaData result = ScriptMetaData.PROTO.fromXContent(parser);
ScriptMetaData result = ScriptMetaData.fromXContent(parser);
assertEquals(expected, result);
assertEquals(expected.hashCode(), result.hashCode());
}
@ -90,7 +90,7 @@ public class ScriptMetaDataTests extends ESTestCase {
ByteArrayOutputStream out = new ByteArrayOutputStream();
expected.writeTo(new OutputStreamStreamOutput(out));
ScriptMetaData result = ScriptMetaData.PROTO.readFrom(new InputStreamStreamInput(new ByteArrayInputStream(out.toByteArray())));
ScriptMetaData result = new ScriptMetaData(new InputStreamStreamInput(new ByteArrayInputStream(out.toByteArray())));
assertEquals(expected, result);
assertEquals(expected.hashCode(), result.hashCode());
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.snapshots;
import org.elasticsearch.ElasticsearchCorruptionException;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.blobstore.BlobContainer;
import org.elasticsearch.common.blobstore.BlobMetaData;
import org.elasticsearch.common.blobstore.BlobPath;
@ -32,7 +31,6 @@ import org.elasticsearch.common.io.Streams;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.FromXContentBuilder;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -56,12 +54,9 @@ import static org.hamcrest.Matchers.greaterThan;
public class BlobStoreFormatIT extends AbstractSnapshotIntegTestCase {
private static final ParseFieldMatcher parseFieldMatcher = new ParseFieldMatcher(Settings.EMPTY);
public static final String BLOB_CODEC = "blob";
private static class BlobObj implements ToXContent, FromXContentBuilder<BlobObj> {
public static final BlobObj PROTO = new BlobObj("");
private static class BlobObj implements ToXContent {
private final String text;
@ -73,8 +68,7 @@ public class BlobStoreFormatIT extends AbstractSnapshotIntegTestCase {
return text;
}
@Override
public BlobObj fromXContent(XContentParser parser, ParseFieldMatcher parseFieldMatcher) throws IOException {
public static BlobObj fromXContent(XContentParser parser) throws IOException {
String text = null;
XContentParser.Token token = parser.currentToken();
if (token == null) {
@ -114,9 +108,12 @@ public class BlobStoreFormatIT extends AbstractSnapshotIntegTestCase {
public void testBlobStoreOperations() throws IOException {
BlobStore blobStore = createTestBlobStore();
BlobContainer blobContainer = blobStore.blobContainer(BlobPath.cleanPath());
ChecksumBlobStoreFormat<BlobObj> checksumJSON = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj.PROTO, parseFieldMatcher, false, XContentType.JSON);
ChecksumBlobStoreFormat<BlobObj> checksumSMILE = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj.PROTO, parseFieldMatcher, false, XContentType.SMILE);
ChecksumBlobStoreFormat<BlobObj> checksumSMILECompressed = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj.PROTO, parseFieldMatcher, true, XContentType.SMILE);
ChecksumBlobStoreFormat<BlobObj> checksumJSON = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent,
xContentRegistry(), false, XContentType.JSON);
ChecksumBlobStoreFormat<BlobObj> checksumSMILE = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent,
xContentRegistry(), false, XContentType.SMILE);
ChecksumBlobStoreFormat<BlobObj> checksumSMILECompressed = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent,
xContentRegistry(), true, XContentType.SMILE);
// Write blobs in different formats
checksumJSON.write(new BlobObj("checksum json"), blobContainer, "check-json");
@ -139,8 +136,10 @@ public class BlobStoreFormatIT extends AbstractSnapshotIntegTestCase {
for (int i = 0; i < randomIntBetween(100, 300); i++) {
veryRedundantText.append("Blah ");
}
ChecksumBlobStoreFormat<BlobObj> checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj.PROTO, parseFieldMatcher, false, randomBoolean() ? XContentType.SMILE : XContentType.JSON);
ChecksumBlobStoreFormat<BlobObj> checksumFormatComp = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj.PROTO, parseFieldMatcher, true, randomBoolean() ? XContentType.SMILE : XContentType.JSON);
ChecksumBlobStoreFormat<BlobObj> checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent,
xContentRegistry(), false, randomBoolean() ? XContentType.SMILE : XContentType.JSON);
ChecksumBlobStoreFormat<BlobObj> checksumFormatComp = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent,
xContentRegistry(), true, randomBoolean() ? XContentType.SMILE : XContentType.JSON);
BlobObj blobObj = new BlobObj(veryRedundantText.toString());
checksumFormatComp.write(blobObj, blobContainer, "blob-comp");
checksumFormat.write(blobObj, blobContainer, "blob-not-comp");
@ -154,7 +153,8 @@ public class BlobStoreFormatIT extends AbstractSnapshotIntegTestCase {
BlobContainer blobContainer = blobStore.blobContainer(BlobPath.cleanPath());
String testString = randomAsciiOfLength(randomInt(10000));
BlobObj blobObj = new BlobObj(testString);
ChecksumBlobStoreFormat<BlobObj> checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj.PROTO, parseFieldMatcher, randomBoolean(), randomBoolean() ? XContentType.SMILE : XContentType.JSON);
ChecksumBlobStoreFormat<BlobObj> checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent,
xContentRegistry(), randomBoolean(), randomBoolean() ? XContentType.SMILE : XContentType.JSON);
checksumFormat.write(blobObj, blobContainer, "test-path");
assertEquals(checksumFormat.read(blobContainer, "test-path").getText(), testString);
randomCorruption(blobContainer, "test-path");
@ -188,7 +188,8 @@ public class BlobStoreFormatIT extends AbstractSnapshotIntegTestCase {
return builder;
}
};
final ChecksumBlobStoreFormat<BlobObj> checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj.PROTO, parseFieldMatcher, randomBoolean(), randomBoolean() ? XContentType.SMILE : XContentType.JSON);
final ChecksumBlobStoreFormat<BlobObj> checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent,
xContentRegistry(), randomBoolean(), randomBoolean() ? XContentType.SMILE : XContentType.JSON);
ExecutorService threadPool = Executors.newFixedThreadPool(1);
try {
Future<Void> future = threadPool.submit(new Callable<Void>() {

View File

@ -35,15 +35,24 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.discovery.zen.ElectMasterService;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.indices.recovery.RecoveryState;
@ -62,6 +71,7 @@ import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.test.TestCustomMetaData;
import org.elasticsearch.test.rest.FakeRestRequest;
import java.io.IOException;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
@ -88,9 +98,51 @@ import static org.hamcrest.Matchers.nullValue;
@ClusterScope(scope = Scope.TEST, numDataNodes = 0, transportClientRatio = 0)
public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCase {
public static class TestCustomMetaDataPlugin extends Plugin {
private final List<NamedWriteableRegistry.Entry> namedWritables = new ArrayList<>();
private final List<NamedXContentRegistry.Entry> namedXContents = new ArrayList<>();
public TestCustomMetaDataPlugin() {
registerBuiltinWritables();
}
private <T extends MetaData.Custom> void registerMetaDataCustom(String name, Writeable.Reader<T> reader,
Writeable.Reader<NamedDiff> diffReader,
NamedXContentRegistry.FromXContent<T> parser) {
namedWritables.add(new NamedWriteableRegistry.Entry(MetaData.Custom.class, name, reader));
namedWritables.add(new NamedWriteableRegistry.Entry(NamedDiff.class, name, diffReader));
namedXContents.add(new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField(name), parser));
}
private void registerBuiltinWritables() {
registerMetaDataCustom(SnapshottableMetadata.TYPE, SnapshottableMetadata::readFrom,
SnapshottableMetadata::readDiffFrom, SnapshottableMetadata::fromXContent);
registerMetaDataCustom(NonSnapshottableMetadata.TYPE, NonSnapshottableMetadata::readFrom,
NonSnapshottableMetadata::readDiffFrom, NonSnapshottableMetadata::fromXContent);
registerMetaDataCustom(SnapshottableGatewayMetadata.TYPE, SnapshottableGatewayMetadata::readFrom,
SnapshottableGatewayMetadata::readDiffFrom, SnapshottableGatewayMetadata::fromXContent);
registerMetaDataCustom(NonSnapshottableGatewayMetadata.TYPE, NonSnapshottableGatewayMetadata::readFrom,
NonSnapshottableGatewayMetadata::readDiffFrom, NonSnapshottableGatewayMetadata::fromXContent);
registerMetaDataCustom(SnapshotableGatewayNoApiMetadata.TYPE, SnapshotableGatewayNoApiMetadata::readFrom,
NonSnapshottableGatewayMetadata::readDiffFrom, SnapshotableGatewayNoApiMetadata::fromXContent);
}
@Override
public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
return namedWritables;
}
@Override
public List<NamedXContentRegistry.Entry> getNamedXContent() {
return namedXContents;
}
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return Arrays.asList(MockRepository.Plugin.class);
return Arrays.asList(MockRepository.Plugin.class, TestCustomMetaDataPlugin.class);
}
public void testRestorePersistentSettings() throws Exception {
@ -780,33 +832,31 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
}
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);
}
public static class SnapshottableMetadata extends TestCustomMetaData {
public static final String TYPE = "test_snapshottable";
public static final SnapshottableMetadata PROTO = new SnapshottableMetadata("");
public SnapshottableMetadata(String data) {
super(data);
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@Override
protected TestCustomMetaData newTestCustomMetaData(String data) {
return new SnapshottableMetadata(data);
public static SnapshottableMetadata readFrom(StreamInput in) throws IOException {
return readFrom(SnapshottableMetadata::new, in);
}
public static NamedDiff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(TYPE, in);
}
public static SnapshottableMetadata fromXContent(XContentParser parser) throws IOException {
return fromXContent(SnapshottableMetadata::new, parser);
}
@Override
public EnumSet<MetaData.XContentContext> context() {
return MetaData.API_AND_SNAPSHOT;
@ -816,20 +866,25 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
public static class NonSnapshottableMetadata extends TestCustomMetaData {
public static final String TYPE = "test_non_snapshottable";
public static final NonSnapshottableMetadata PROTO = new NonSnapshottableMetadata("");
public NonSnapshottableMetadata(String data) {
super(data);
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@Override
protected NonSnapshottableMetadata newTestCustomMetaData(String data) {
return new NonSnapshottableMetadata(data);
public static NonSnapshottableMetadata readFrom(StreamInput in) throws IOException {
return readFrom(NonSnapshottableMetadata::new, in);
}
public static NamedDiff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(TYPE, in);
}
public static NonSnapshottableMetadata fromXContent(XContentParser parser) throws IOException {
return fromXContent(NonSnapshottableMetadata::new, parser);
}
@Override
@ -841,20 +896,25 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
public static class SnapshottableGatewayMetadata extends TestCustomMetaData {
public static final String TYPE = "test_snapshottable_gateway";
public static final SnapshottableGatewayMetadata PROTO = new SnapshottableGatewayMetadata("");
public SnapshottableGatewayMetadata(String data) {
super(data);
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@Override
protected TestCustomMetaData newTestCustomMetaData(String data) {
return new SnapshottableGatewayMetadata(data);
public static SnapshottableGatewayMetadata readFrom(StreamInput in) throws IOException {
return readFrom(SnapshottableGatewayMetadata::new, in);
}
public static NamedDiff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(TYPE, in);
}
public static SnapshottableGatewayMetadata fromXContent(XContentParser parser) throws IOException {
return fromXContent(SnapshottableGatewayMetadata::new, parser);
}
@Override
@ -866,20 +926,25 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
public static class NonSnapshottableGatewayMetadata extends TestCustomMetaData {
public static final String TYPE = "test_non_snapshottable_gateway";
public static final NonSnapshottableGatewayMetadata PROTO = new NonSnapshottableGatewayMetadata("");
public NonSnapshottableGatewayMetadata(String data) {
super(data);
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@Override
protected NonSnapshottableGatewayMetadata newTestCustomMetaData(String data) {
return new NonSnapshottableGatewayMetadata(data);
public static NonSnapshottableGatewayMetadata readFrom(StreamInput in) throws IOException {
return readFrom(NonSnapshottableGatewayMetadata::new, in);
}
public static NamedDiff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(TYPE, in);
}
public static NonSnapshottableGatewayMetadata fromXContent(XContentParser parser) throws IOException {
return fromXContent(NonSnapshottableGatewayMetadata::new, parser);
}
@Override
@ -892,20 +957,21 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
public static class SnapshotableGatewayNoApiMetadata extends TestCustomMetaData {
public static final String TYPE = "test_snapshottable_gateway_no_api";
public static final SnapshotableGatewayNoApiMetadata PROTO = new SnapshotableGatewayNoApiMetadata("");
public SnapshotableGatewayNoApiMetadata(String data) {
super(data);
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
@Override
protected SnapshotableGatewayNoApiMetadata newTestCustomMetaData(String data) {
return new SnapshotableGatewayNoApiMetadata(data);
public static SnapshotableGatewayNoApiMetadata readFrom(StreamInput in) throws IOException {
return readFrom(SnapshotableGatewayNoApiMetadata::new, in);
}
public static SnapshotableGatewayNoApiMetadata fromXContent(XContentParser parser) throws IOException {
return fromXContent(SnapshotableGatewayNoApiMetadata::new, parser);
}
@Override

View File

@ -46,6 +46,7 @@ import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.plugins.RepositoryPlugin;
import org.elasticsearch.repositories.Repository;
@ -63,8 +64,8 @@ public class MockRepository extends FsRepository {
@Override
public Map<String, Repository.Factory> getRepositories(Environment env) {
return Collections.singletonMap("mock", (metadata) -> new MockRepository(metadata, env));
public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) {
return Collections.singletonMap("mock", (metadata) -> new MockRepository(metadata, env, namedXContentRegistry));
}
@Override
@ -101,8 +102,9 @@ public class MockRepository extends FsRepository {
private volatile boolean blocked = false;
public MockRepository(RepositoryMetaData metadata, Environment environment) throws IOException {
super(overrideSettings(metadata, environment), environment);
public MockRepository(RepositoryMetaData metadata, Environment environment,
NamedXContentRegistry namedXContentRegistry) throws IOException {
super(overrideSettings(metadata, environment), environment, namedXContentRegistry);
randomControlIOExceptionRate = metadata.settings().getAsDouble("random_control_io_exception_rate", 0.0);
randomDataFileIOExceptionRate = metadata.settings().getAsDouble("random_data_file_io_exception_rate", 0.0);
useLuceneCorruptionException = metadata.settings().getAsBoolean("use_lucene_corruption", false);

View File

@ -24,8 +24,8 @@ import org.elasticsearch.action.support.DestructiveOperations;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.LocalClusterUpdateTask;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.MetaData;
@ -33,6 +33,8 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.settings.Settings;
@ -53,6 +55,7 @@ import org.junit.AfterClass;
import org.junit.Before;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
@ -117,9 +120,37 @@ public class TribeIT extends ESIntegTestCase {
.build();
}
public static class TestCustomMetaDataPlugin extends Plugin {
private final List<NamedWriteableRegistry.Entry> namedWritables = new ArrayList<>();
public TestCustomMetaDataPlugin() {
registerBuiltinWritables();
}
private <T extends MetaData.Custom> void registerMetaDataCustom(String name, Writeable.Reader<? extends T> reader,
Writeable.Reader<NamedDiff> diffReader) {
namedWritables.add(new NamedWriteableRegistry.Entry(MetaData.Custom.class, name, reader));
namedWritables.add(new NamedWriteableRegistry.Entry(NamedDiff.class, name, diffReader));
}
private void registerBuiltinWritables() {
registerMetaDataCustom(MergableCustomMetaData1.TYPE, MergableCustomMetaData1::readFrom, MergableCustomMetaData1::readDiffFrom);
registerMetaDataCustom(MergableCustomMetaData2.TYPE, MergableCustomMetaData2::readFrom, MergableCustomMetaData2::readDiffFrom);
}
@Override
public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
return namedWritables;
}
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return getMockPlugins();
ArrayList<Class<? extends Plugin>> plugins = new ArrayList<>();
plugins.addAll(getMockPlugins());
plugins.add(TestCustomMetaDataPlugin.class);
return plugins;
}
@Before
@ -456,7 +487,6 @@ public class TribeIT extends ESIntegTestCase {
}
public void testMergingRemovedCustomMetaData() throws Exception {
MetaData.registerPrototype(MergableCustomMetaData1.TYPE, new MergableCustomMetaData1(""));
removeCustomMetaData(cluster1, MergableCustomMetaData1.TYPE);
removeCustomMetaData(cluster2, MergableCustomMetaData1.TYPE);
MergableCustomMetaData1 customMetaData1 = new MergableCustomMetaData1("a");
@ -466,13 +496,12 @@ public class TribeIT extends ESIntegTestCase {
putCustomMetaData(cluster1, customMetaData1);
putCustomMetaData(cluster2, customMetaData2);
assertCustomMetaDataUpdated(internalCluster(), customMetaData2);
removeCustomMetaData(cluster2, customMetaData2.type());
removeCustomMetaData(cluster2, customMetaData2.getWriteableName());
assertCustomMetaDataUpdated(internalCluster(), customMetaData1);
}
}
public void testMergingCustomMetaData() throws Exception {
MetaData.registerPrototype(MergableCustomMetaData1.TYPE, new MergableCustomMetaData1(""));
removeCustomMetaData(cluster1, MergableCustomMetaData1.TYPE);
removeCustomMetaData(cluster2, MergableCustomMetaData1.TYPE);
MergableCustomMetaData1 customMetaData1 = new MergableCustomMetaData1(randomAsciiOfLength(10));
@ -490,8 +519,6 @@ public class TribeIT extends ESIntegTestCase {
}
public void testMergingMultipleCustomMetaData() throws Exception {
MetaData.registerPrototype(MergableCustomMetaData1.TYPE, new MergableCustomMetaData1(""));
MetaData.registerPrototype(MergableCustomMetaData2.TYPE, new MergableCustomMetaData2(""));
removeCustomMetaData(cluster1, MergableCustomMetaData1.TYPE);
removeCustomMetaData(cluster2, MergableCustomMetaData1.TYPE);
MergableCustomMetaData1 firstCustomMetaDataType1 = new MergableCustomMetaData1(randomAsciiOfLength(10));
@ -521,10 +548,10 @@ public class TribeIT extends ESIntegTestCase {
assertCustomMetaDataUpdated(internalCluster(), mergedCustomMetaDataType2.get(0));
// test removing custom md is propagates to tribe
removeCustomMetaData(cluster2, secondCustomMetaDataType1.type());
removeCustomMetaData(cluster2, secondCustomMetaDataType1.getWriteableName());
assertCustomMetaDataUpdated(internalCluster(), firstCustomMetaDataType1);
assertCustomMetaDataUpdated(internalCluster(), mergedCustomMetaDataType2.get(0));
removeCustomMetaData(cluster2, secondCustomMetaDataType2.type());
removeCustomMetaData(cluster2, secondCustomMetaDataType2.getWriteableName());
assertCustomMetaDataUpdated(internalCluster(), firstCustomMetaDataType1);
assertCustomMetaDataUpdated(internalCluster(), firstCustomMetaDataType2);
}
@ -534,7 +561,7 @@ public class TribeIT extends ESIntegTestCase {
TestCustomMetaData expectedCustomMetaData) throws Exception {
assertBusy(() -> {
ClusterState tribeState = cluster.getInstance(ClusterService.class, cluster.getNodeNames()[0]).state();
MetaData.Custom custom = tribeState.metaData().custom(expectedCustomMetaData.type());
MetaData.Custom custom = tribeState.metaData().custom(expectedCustomMetaData.getWriteableName());
assertNotNull(custom);
assertThat(custom, equalTo(expectedCustomMetaData));
});
@ -546,9 +573,9 @@ public class TribeIT extends ESIntegTestCase {
}
private void putCustomMetaData(InternalTestCluster cluster, final TestCustomMetaData customMetaData) {
logger.info("putting custom_md type [{}] with data[{}] from [{}]", customMetaData.type(),
logger.info("putting custom_md type [{}] with data[{}] from [{}]", customMetaData.getWriteableName(),
customMetaData.getData(), cluster.getClusterName());
updateMetaData(cluster, builder -> builder.putCustom(customMetaData.type(), customMetaData));
updateMetaData(cluster, builder -> builder.putCustom(customMetaData.getWriteableName(), customMetaData));
}
private static void updateMetaData(InternalTestCluster cluster, UnaryOperator<MetaData.Builder> addCustoms) {

View File

@ -19,13 +19,16 @@
package org.elasticsearch.tribe;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.TestCustomMetaData;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@ -185,13 +188,16 @@ public class TribeServiceTests extends ESTestCase {
}
@Override
protected TestCustomMetaData newTestCustomMetaData(String data) {
return new MergableCustomMetaData1(data);
public String getWriteableName() {
return TYPE;
}
@Override
public String type() {
return TYPE;
public static MergableCustomMetaData1 readFrom(StreamInput in) throws IOException {
return readFrom(MergableCustomMetaData1::new, in);
}
public static NamedDiff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(TYPE, in);
}
@Override
@ -214,15 +220,19 @@ public class TribeServiceTests extends ESTestCase {
}
@Override
protected TestCustomMetaData newTestCustomMetaData(String data) {
return new MergableCustomMetaData2(data);
}
@Override
public String type() {
public String getWriteableName() {
return TYPE;
}
public static MergableCustomMetaData2 readFrom(StreamInput in) throws IOException {
return readFrom(MergableCustomMetaData2::new, in);
}
public static NamedDiff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(TYPE, in);
}
@Override
public EnumSet<MetaData.XContentContext> context() {
return EnumSet.of(MetaData.XContentContext.GATEWAY);

View File

@ -31,6 +31,7 @@ import org.elasticsearch.cloud.azure.classic.management.AzureComputeService;
import org.elasticsearch.cloud.azure.classic.management.AzureComputeServiceImpl;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkService;
@ -74,10 +75,11 @@ public class AzureDiscoveryPlugin extends Plugin implements DiscoveryPlugin {
@Override
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) {
// this is for backcompat with pre 5.1, where users would set discovery.type to use ec2 hosts provider
return Collections.singletonMap(AZURE, () ->
new ZenDiscovery(settings, threadPool, transportService, clusterService, hostsProvider));
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider));
}
@Override

View File

@ -45,6 +45,7 @@ import org.elasticsearch.cloud.aws.AwsEc2Service;
import org.elasticsearch.cloud.aws.AwsEc2ServiceImpl;
import org.elasticsearch.cloud.aws.network.Ec2NameResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkService;
@ -101,10 +102,11 @@ public class Ec2DiscoveryPlugin extends Plugin implements DiscoveryPlugin, Close
@Override
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) {
// this is for backcompat with pre 5.1, where users would set discovery.type to use ec2 hosts provider
return Collections.singletonMap(EC2, () ->
new ZenDiscovery(settings, threadPool, transportService, clusterService, hostsProvider));
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider));
}
@Override

View File

@ -33,6 +33,7 @@ import org.elasticsearch.cloud.gce.network.GceNameResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkService;
@ -99,10 +100,11 @@ public class GceDiscoveryPlugin extends Plugin implements DiscoveryPlugin, Close
@Override
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) {
// this is for backcompat with pre 5.1, where users would set discovery.type to use ec2 hosts provider
return Collections.singletonMap(GCE, () ->
new ZenDiscovery(settings, threadPool, transportService, clusterService, hostsProvider));
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider));
}
@Override

View File

@ -23,6 +23,7 @@ import org.elasticsearch.cloud.azure.storage.AzureStorageService;
import org.elasticsearch.cloud.azure.storage.AzureStorageServiceImpl;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.RepositoryPlugin;
@ -45,9 +46,9 @@ public class AzureRepositoryPlugin extends Plugin implements RepositoryPlugin {
}
@Override
public Map<String, Repository.Factory> getRepositories(Environment env) {
public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) {
return Collections.singletonMap(AzureRepository.TYPE,
(metadata) -> new AzureRepository(metadata, env, createStorageService(env.settings())));
(metadata) -> new AzureRepository(metadata, env, namedXContentRegistry, createStorageService(env.settings())));
}
@Override

View File

@ -32,6 +32,7 @@ import org.elasticsearch.cloud.azure.storage.AzureStorageService;
import org.elasticsearch.cloud.azure.storage.AzureStorageService.Storage;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.RepositoryMetaData;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.common.Strings;
@ -84,9 +85,10 @@ public class AzureRepository extends BlobStoreRepository {
private final boolean compress;
private final boolean readonly;
public AzureRepository(RepositoryMetaData metadata, Environment environment, AzureStorageService storageService)
public AzureRepository(RepositoryMetaData metadata, Environment environment,
NamedXContentRegistry namedXContentRegistry, AzureStorageService storageService)
throws IOException, URISyntaxException, StorageException {
super(metadata, environment.settings());
super(metadata, environment.settings(), namedXContentRegistry);
blobStore = new AzureBlobStore(metadata, environment.settings(), storageService);
String container = getValue(metadata.settings(), settings, Repository.CONTAINER_SETTING, Storage.CONTAINER_SETTING);

View File

@ -27,6 +27,7 @@ import org.elasticsearch.cloud.azure.storage.AzureStorageServiceImpl;
import org.elasticsearch.cloud.azure.storage.AzureStorageSettings;
import org.elasticsearch.cluster.metadata.RepositoryMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.test.ESTestCase;
@ -47,7 +48,8 @@ public class AzureRepositorySettingsTests extends ESTestCase {
.putArray(Environment.PATH_DATA_SETTING.getKey(), tmpPaths())
.put(settings)
.build();
return new AzureRepository(new RepositoryMetaData("foo", "azure", internalSettings), new Environment(internalSettings), null);
return new AzureRepository(new RepositoryMetaData("foo", "azure", internalSettings), new Environment(internalSettings),
NamedXContentRegistry.EMPTY, null);
}

View File

@ -42,6 +42,7 @@ import com.google.api.services.storage.model.StorageObject;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.RepositoryPlugin;
@ -120,8 +121,8 @@ public class GoogleCloudStoragePlugin extends Plugin implements RepositoryPlugin
}
@Override
public Map<String, Repository.Factory> getRepositories(Environment env) {
public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) {
return Collections.singletonMap(GoogleCloudStorageRepository.TYPE,
(metadata) -> new GoogleCloudStorageRepository(metadata, env, createStorageService(env)));
(metadata) -> new GoogleCloudStorageRepository(metadata, env, namedXContentRegistry, createStorageService(env)));
}
}

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.plugin.repository.gcs.GoogleCloudStoragePlugin;
import org.elasticsearch.repositories.RepositoryException;
@ -72,8 +73,9 @@ public class GoogleCloudStorageRepository extends BlobStoreRepository {
private final GoogleCloudStorageBlobStore blobStore;
public GoogleCloudStorageRepository(RepositoryMetaData metadata, Environment environment,
NamedXContentRegistry namedXContentRegistry,
GoogleCloudStorageService storageService) throws Exception {
super(metadata, environment.settings());
super(metadata, environment.settings(), namedXContentRegistry);
String bucket = get(BUCKET, metadata);
String application = get(APPLICATION_NAME, metadata);

View File

@ -28,6 +28,7 @@ import java.util.Map;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.RepositoryPlugin;
@ -88,7 +89,7 @@ public final class HdfsPlugin extends Plugin implements RepositoryPlugin {
}
@Override
public Map<String, Repository.Factory> getRepositories(Environment env) {
return Collections.singletonMap("hdfs", (metadata) -> new HdfsRepository(metadata, env));
public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) {
return Collections.singletonMap("hdfs", (metadata) -> new HdfsRepository(metadata, env, namedXContentRegistry));
}
}

View File

@ -45,6 +45,7 @@ import org.elasticsearch.common.blobstore.BlobStore;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
@ -60,8 +61,9 @@ public final class HdfsRepository extends BlobStoreRepository {
// TODO: why 100KB?
private static final ByteSizeValue DEFAULT_BUFFER_SIZE = new ByteSizeValue(100, ByteSizeUnit.KB);
public HdfsRepository(RepositoryMetaData metadata, Environment environment) throws IOException {
super(metadata, environment.settings());
public HdfsRepository(RepositoryMetaData metadata, Environment environment,
NamedXContentRegistry namedXContentRegistry) throws IOException {
super(metadata, environment.settings(), namedXContentRegistry);
this.chunkSize = metadata.settings().getAsBytesSize("chunk_size", null);
this.compress = metadata.settings().getAsBoolean("compress", false);
@ -88,7 +90,7 @@ public final class HdfsRepository extends BlobStoreRepository {
if (pathSetting == null) {
throw new IllegalArgumentException("No 'path' defined for hdfs snapshot/restore");
}
int bufferSize = getMetadata().settings().getAsBytesSize("buffer_size", DEFAULT_BUFFER_SIZE).bytesAsInt();
try {
@ -110,7 +112,7 @@ public final class HdfsRepository extends BlobStoreRepository {
}
super.doStart();
}
// create hadoop filecontext
@SuppressForbidden(reason = "lesser of two evils (the other being a bunch of JNI/classloader nightmares)")
private static FileContext createContext(URI uri, Settings repositorySettings) {

Some files were not shown because too many files have changed in this diff Show More