Add rollover-creation-date setting to rolled over index (#31144)

This commit introduces a new property to IndexMetaData called
RolloverInfo. This object contains a map containing the aliases
that were used to rollover the related index, which conditions
were met, and at what time the rollover took place.

much like the `index.creation_date`, it captures the approximate time
that the index was rolled over to a new one.
This commit is contained in:
Tal Levy 2018-06-15 08:44:29 -07:00 committed by GitHub
parent da5bfda5f3
commit eda4964f64
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 355 additions and 26 deletions

View File

@ -23,6 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@ -64,4 +65,12 @@ public class MaxAgeCondition extends Condition<TimeValue> {
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
return builder.field(NAME, value.getStringRep());
}
public static MaxAgeCondition fromXContent(XContentParser parser) throws IOException {
if (parser.nextToken() == XContentParser.Token.VALUE_STRING) {
return new MaxAgeCondition(TimeValue.parseTimeValue(parser.text(), NAME));
} else {
throw new IllegalArgumentException("invalid token: " + parser.currentToken());
}
}
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.action.admin.indices.rollover;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@ -61,4 +62,12 @@ public class MaxDocsCondition extends Condition<Long> {
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
return builder.field(NAME, value);
}
public static MaxDocsCondition fromXContent(XContentParser parser) throws IOException {
if (parser.nextToken() == XContentParser.Token.VALUE_NUMBER) {
return new MaxDocsCondition(parser.longValue());
} else {
throw new IllegalArgumentException("invalid token: " + parser.currentToken());
}
}
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@ -70,4 +71,12 @@ public class MaxSizeCondition extends Condition<ByteSizeValue> {
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
return builder.field(NAME, value.getStringRep());
}
public static MaxSizeCondition fromXContent(XContentParser parser) throws IOException {
if (parser.nextToken() == XContentParser.Token.VALUE_STRING) {
return new MaxSizeCondition(ByteSizeValue.parseBytesSizeValue(parser.text(), NAME));
} else {
throw new IllegalArgumentException("invalid token: " + parser.currentToken());
}
}
}

View File

@ -0,0 +1,134 @@
/*
* 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.action.admin.indices.rollover;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ToXContentFragment;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
import java.util.List;
import java.util.Objects;
/**
* Class for holding Rollover related information within an index
*/
public class RolloverInfo extends AbstractDiffable<RolloverInfo> implements Writeable, ToXContentFragment {
public static final ParseField CONDITION_FIELD = new ParseField("met_conditions");
public static final ParseField TIME_FIELD = new ParseField("time");
@SuppressWarnings("unchecked")
public static ConstructingObjectParser<RolloverInfo, String> PARSER = new ConstructingObjectParser<>("rollover_info", false,
(a, alias) -> new RolloverInfo(alias, (List<Condition>) a[0], (Long) a[1]));
static {
PARSER.declareNamedObjects(ConstructingObjectParser.constructorArg(),
(p, c, n) -> p.namedObject(Condition.class, n, c), CONDITION_FIELD);
PARSER.declareLong(ConstructingObjectParser.constructorArg(), TIME_FIELD);
}
private final String alias;
private final List<Condition> metConditions;
private final long time;
public RolloverInfo(String alias, List<Condition> metConditions, long time) {
this.alias = alias;
this.metConditions = metConditions;
this.time = time;
}
public RolloverInfo(StreamInput in) throws IOException {
this.alias = in.readString();
this.time = in.readVLong();
this.metConditions = in.readNamedWriteableList(Condition.class);
}
public static RolloverInfo parse(XContentParser parser, String alias) {
return PARSER.apply(parser, alias);
}
public String getAlias() {
return alias;
}
public List<Condition> getMetConditions() {
return metConditions;
}
public long getTime() {
return time;
}
public static Diff<RolloverInfo> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(RolloverInfo::new, in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(alias);
out.writeVLong(time);
out.writeNamedWriteableList(metConditions);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(alias);
builder.startObject(CONDITION_FIELD.getPreferredName());
for (Condition condition : metConditions) {
condition.toXContent(builder, params);
}
builder.endObject();
builder.field(TIME_FIELD.getPreferredName(), time);
builder.endObject();
return builder;
}
@Override
public int hashCode() {
return Objects.hash(alias, metConditions, time);
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (obj.getClass() != getClass()) {
return false;
}
RolloverInfo other = (RolloverInfo) obj;
return Objects.equals(alias, other.alias) &&
Objects.equals(metConditions, other.metConditions) &&
Objects.equals(time, other.time);
}
@Override
public String toString() {
return Strings.toString(this);
}
}

View File

@ -31,6 +31,7 @@ import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.AliasAction;
@ -131,7 +132,9 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
new RolloverResponse(sourceIndexName, rolloverIndexName, conditionResults, true, false, false, false));
return;
}
if (conditionResults.size() == 0 || conditionResults.values().stream().anyMatch(result -> result)) {
List<Condition> metConditions = rolloverRequest.getConditions().values().stream()
.filter(condition -> conditionResults.get(condition.toString())).collect(Collectors.toList());
if (conditionResults.size() == 0 || metConditions.size() > 0) {
CreateIndexClusterStateUpdateRequest updateRequest = prepareCreateIndexRequest(unresolvedName, rolloverIndexName,
rolloverRequest);
createIndexService.createIndex(updateRequest, ActionListener.wrap(createIndexClusterStateUpdateResponse -> {
@ -141,13 +144,33 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
rolloverRequest),
ActionListener.wrap(aliasClusterStateUpdateResponse -> {
if (aliasClusterStateUpdateResponse.isAcknowledged()) {
activeShardsObserver.waitForActiveShards(new String[]{rolloverIndexName},
rolloverRequest.getCreateIndexRequest().waitForActiveShards(),
rolloverRequest.masterNodeTimeout(),
isShardsAcknowledged -> listener.onResponse(new RolloverResponse(
sourceIndexName, rolloverIndexName, conditionResults, false, true, true,
isShardsAcknowledged)),
listener::onFailure);
clusterService.submitStateUpdateTask("update_rollover_info", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
RolloverInfo rolloverInfo = new RolloverInfo(rolloverRequest.getAlias(), metConditions,
threadPool.absoluteTimeInMillis());
return ClusterState.builder(currentState)
.metaData(MetaData.builder(currentState.metaData())
.put(IndexMetaData.builder(currentState.metaData().index(sourceIndexName))
.putRolloverInfo(rolloverInfo))).build();
}
@Override
public void onFailure(String source, Exception e) {
listener.onFailure(e);
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
activeShardsObserver.waitForActiveShards(new String[]{rolloverIndexName},
rolloverRequest.getCreateIndexRequest().waitForActiveShards(),
rolloverRequest.masterNodeTimeout(),
isShardsAcknowledged -> listener.onResponse(new RolloverResponse(
sourceIndexName, rolloverIndexName, conditionResults, false, true, true,
isShardsAcknowledged)),
listener::onFailure);
}
});
} else {
listener.onResponse(new RolloverResponse(sourceIndexName, rolloverIndexName, conditionResults,
false, true, false, false));

View File

@ -46,6 +46,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.PageCacheRecycler;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.node.InternalSettingsPreparer;
import org.elasticsearch.node.Node;
@ -150,9 +151,11 @@ public abstract class TransportClient extends AbstractClient {
SettingsModule settingsModule = new SettingsModule(settings, additionalSettings, additionalSettingsFilter);
SearchModule searchModule = new SearchModule(settings, true, pluginsService.filterPlugins(SearchPlugin.class));
IndicesModule indicesModule = new IndicesModule(Collections.emptyList());
List<NamedWriteableRegistry.Entry> entries = new ArrayList<>();
entries.addAll(NetworkModule.getNamedWriteables());
entries.addAll(searchModule.getNamedWriteables());
entries.addAll(indicesModule.getNamedWriteables());
entries.addAll(ClusterModule.getNamedWriteables());
entries.addAll(pluginsService.filterPlugins(Plugin.class).stream()
.flatMap(p -> p.getNamedWriteables().stream())

View File

@ -25,6 +25,7 @@ import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.rollover.RolloverInfo;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.Diffable;
@ -294,6 +295,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
static final String KEY_STATE = "state";
static final String KEY_MAPPINGS = "mappings";
static final String KEY_ALIASES = "aliases";
static final String KEY_ROLLOVER_INFOS = "rollover_info";
public static final String KEY_PRIMARY_TERMS = "primary_terms";
public static final String INDEX_STATE_FILE_PREFIX = "state-";
@ -331,13 +333,14 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
private final Version indexUpgradedVersion;
private final ActiveShardCount waitForActiveShards;
private final ImmutableOpenMap<String, RolloverInfo> rolloverInfos;
private IndexMetaData(Index index, long version, long[] primaryTerms, State state, int numberOfShards, int numberOfReplicas, Settings settings,
ImmutableOpenMap<String, MappingMetaData> mappings, ImmutableOpenMap<String, AliasMetaData> aliases,
ImmutableOpenMap<String, Custom> customs, ImmutableOpenIntMap<Set<String>> inSyncAllocationIds,
DiscoveryNodeFilters requireFilters, DiscoveryNodeFilters initialRecoveryFilters, DiscoveryNodeFilters includeFilters, DiscoveryNodeFilters excludeFilters,
Version indexCreatedVersion, Version indexUpgradedVersion,
int routingNumShards, int routingPartitionSize, ActiveShardCount waitForActiveShards) {
int routingNumShards, int routingPartitionSize, ActiveShardCount waitForActiveShards, ImmutableOpenMap<String, RolloverInfo> rolloverInfos) {
this.index = index;
this.version = version;
@ -362,6 +365,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
this.routingFactor = routingNumShards / numberOfShards;
this.routingPartitionSize = routingPartitionSize;
this.waitForActiveShards = waitForActiveShards;
this.rolloverInfos = rolloverInfos;
assert numberOfShards * routingFactor == routingNumShards : routingNumShards + " must be a multiple of " + numberOfShards;
}
@ -517,6 +521,10 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
return inSyncAllocationIds;
}
public ImmutableOpenMap<String, RolloverInfo> getRolloverInfos() {
return rolloverInfos;
}
public Set<String> inSyncAllocationIds(int shardId) {
assert shardId >= 0 && shardId < numberOfShards;
return inSyncAllocationIds.get(shardId);
@ -587,6 +595,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
if (!inSyncAllocationIds.equals(that.inSyncAllocationIds)) {
return false;
}
if (rolloverInfos.equals(that.rolloverInfos) == false) {
return false;
}
return true;
}
@ -603,6 +614,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
result = 31 * result + Long.hashCode(routingNumShards);
result = 31 * result + Arrays.hashCode(primaryTerms);
result = 31 * result + inSyncAllocationIds.hashCode();
result = 31 * result + rolloverInfos.hashCode();
return result;
}
@ -638,6 +650,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
private final Diff<ImmutableOpenMap<String, AliasMetaData>> aliases;
private final Diff<ImmutableOpenMap<String, Custom>> customs;
private final Diff<ImmutableOpenIntMap<Set<String>>> inSyncAllocationIds;
private final Diff<ImmutableOpenMap<String, RolloverInfo>> rolloverInfos;
IndexMetaDataDiff(IndexMetaData before, IndexMetaData after) {
index = after.index.getName();
@ -651,6 +664,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
customs = DiffableUtils.diff(before.customs, after.customs, DiffableUtils.getStringKeySerializer());
inSyncAllocationIds = DiffableUtils.diff(before.inSyncAllocationIds, after.inSyncAllocationIds,
DiffableUtils.getVIntKeySerializer(), DiffableUtils.StringSetValueSerializer.getInstance());
rolloverInfos = DiffableUtils.diff(before.rolloverInfos, after.rolloverInfos, DiffableUtils.getStringKeySerializer());
}
IndexMetaDataDiff(StreamInput in) throws IOException {
@ -679,6 +693,13 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
});
inSyncAllocationIds = DiffableUtils.readImmutableOpenIntMapDiff(in, DiffableUtils.getVIntKeySerializer(),
DiffableUtils.StringSetValueSerializer.getInstance());
if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
rolloverInfos = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), RolloverInfo::new,
RolloverInfo::readDiffFrom);
} else {
ImmutableOpenMap<String, RolloverInfo> emptyMap = ImmutableOpenMap.of();
rolloverInfos = DiffableUtils.diff(emptyMap, emptyMap, DiffableUtils.getStringKeySerializer());
}
}
@Override
@ -693,6 +714,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
aliases.writeTo(out);
customs.writeTo(out);
inSyncAllocationIds.writeTo(out);
if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
rolloverInfos.writeTo(out);
}
}
@Override
@ -707,6 +731,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
builder.aliases.putAll(aliases.apply(part.aliases));
builder.customs.putAll(customs.apply(part.customs));
builder.inSyncAllocationIds.putAll(inSyncAllocationIds.apply(part.inSyncAllocationIds));
builder.rolloverInfos.putAll(rolloverInfos.apply(part.rolloverInfos));
return builder.build();
}
}
@ -740,6 +765,12 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
Set<String> allocationIds = DiffableUtils.StringSetValueSerializer.getInstance().read(in, key);
builder.putInSyncAllocationIds(key, allocationIds);
}
if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
int rolloverAliasesSize = in.readVInt();
for (int i = 0; i < rolloverAliasesSize; i++) {
builder.putRolloverInfo(new RolloverInfo(in));
}
}
return builder.build();
}
@ -769,6 +800,12 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
out.writeVInt(cursor.key);
DiffableUtils.StringSetValueSerializer.getInstance().write(cursor.value, out);
}
if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
out.writeVInt(rolloverInfos.size());
for (ObjectCursor<RolloverInfo> cursor : rolloverInfos.values()) {
cursor.value.writeTo(out);
}
}
}
public static Builder builder(String index) {
@ -790,6 +827,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
private final ImmutableOpenMap.Builder<String, AliasMetaData> aliases;
private final ImmutableOpenMap.Builder<String, Custom> customs;
private final ImmutableOpenIntMap.Builder<Set<String>> inSyncAllocationIds;
private final ImmutableOpenMap.Builder<String, RolloverInfo> rolloverInfos;
private Integer routingNumShards;
public Builder(String index) {
@ -798,6 +836,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
this.aliases = ImmutableOpenMap.builder();
this.customs = ImmutableOpenMap.builder();
this.inSyncAllocationIds = ImmutableOpenIntMap.builder();
this.rolloverInfos = ImmutableOpenMap.builder();
}
public Builder(IndexMetaData indexMetaData) {
@ -811,6 +850,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
this.customs = ImmutableOpenMap.builder(indexMetaData.customs);
this.routingNumShards = indexMetaData.routingNumShards;
this.inSyncAllocationIds = ImmutableOpenIntMap.builder(indexMetaData.inSyncAllocationIds);
this.rolloverInfos = ImmutableOpenMap.builder(indexMetaData.rolloverInfos);
}
public String index() {
@ -951,6 +991,15 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
return this;
}
public RolloverInfo getRolloverInfo(String alias) {
return rolloverInfos.get(alias);
}
public Builder putRolloverInfo(RolloverInfo rolloverInfo) {
rolloverInfos.put(rolloverInfo.getAlias(), rolloverInfo);
return this;
}
public long version() {
return this.version;
}
@ -1089,7 +1138,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
return new IndexMetaData(new Index(index, uuid), version, primaryTerms, state, numberOfShards, numberOfReplicas, tmpSettings, mappings.build(),
tmpAliases.build(), customs.build(), filledInSyncAllocationIds.build(), requireFilters, initialRecoveryFilters, includeFilters, excludeFilters,
indexCreatedVersion, indexUpgradedVersion, getRoutingNumShards(), routingPartitionSize, waitForActiveShards);
indexCreatedVersion, indexUpgradedVersion, getRoutingNumShards(), routingPartitionSize, waitForActiveShards, rolloverInfos.build());
}
public static void toXContent(IndexMetaData indexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException {
@ -1143,6 +1192,12 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
}
builder.endObject();
builder.startObject(KEY_ROLLOVER_INFOS);
for (ObjectCursor<RolloverInfo> cursor : indexMetaData.getRolloverInfos().values()) {
cursor.value.toXContent(builder, params);
}
builder.endObject();
builder.endObject();
}
@ -1202,6 +1257,16 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
throw new IllegalArgumentException("Unexpected token: " + token);
}
}
} else if (KEY_ROLLOVER_INFOS.equals(currentFieldName)) {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.START_OBJECT) {
builder.putRolloverInfo(RolloverInfo.parse(parser, currentFieldName));
} else {
throw new IllegalArgumentException("Unexpected token: " + token);
}
}
} else if ("warmers".equals(currentFieldName)) {
// TODO: do this in 6.0:
// throw new IllegalArgumentException("Warmers are not supported anymore - are you upgrading from 1.x?");

View File

@ -24,9 +24,12 @@ import org.elasticsearch.action.admin.indices.rollover.MaxAgeCondition;
import org.elasticsearch.action.admin.indices.rollover.MaxDocsCondition;
import org.elasticsearch.action.admin.indices.rollover.MaxSizeCondition;
import org.elasticsearch.action.resync.TransportResyncReplicationAction;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.geo.ShapesAvailability;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry.Entry;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.mapper.BinaryFieldMapper;
@ -62,6 +65,7 @@ import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData;
import org.elasticsearch.plugins.MapperPlugin;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedHashMap;
@ -86,15 +90,26 @@ public class IndicesModule extends AbstractModule {
}
private void registerBuiltinWritables() {
namedWritables.add(new Entry(Condition.class, MaxAgeCondition.NAME, MaxAgeCondition::new));
namedWritables.add(new Entry(Condition.class, MaxDocsCondition.NAME, MaxDocsCondition::new));
namedWritables.add(new Entry(Condition.class, MaxSizeCondition.NAME, MaxSizeCondition::new));
namedWritables.add(new NamedWriteableRegistry.Entry(Condition.class, MaxAgeCondition.NAME, MaxAgeCondition::new));
namedWritables.add(new NamedWriteableRegistry.Entry(Condition.class, MaxDocsCondition.NAME, MaxDocsCondition::new));
namedWritables.add(new NamedWriteableRegistry.Entry(Condition.class, MaxSizeCondition.NAME, MaxSizeCondition::new));
}
public List<Entry> getNamedWriteables() {
public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
return namedWritables;
}
public List<NamedXContentRegistry.Entry> getNamedXContents() {
return Arrays.asList(
new NamedXContentRegistry.Entry(Condition.class, new ParseField(MaxAgeCondition.NAME), (p, c) ->
MaxAgeCondition.fromXContent(p)),
new NamedXContentRegistry.Entry(Condition.class, new ParseField(MaxDocsCondition.NAME), (p, c) ->
MaxDocsCondition.fromXContent(p)),
new NamedXContentRegistry.Entry(Condition.class, new ParseField(MaxSizeCondition.NAME), (p, c) ->
MaxSizeCondition.fromXContent(p))
);
}
private Map<String, Mapper.TypeParser> getMappers(List<MapperPlugin> mapperPlugins) {
Map<String, Mapper.TypeParser> mappers = new LinkedHashMap<>();

View File

@ -389,6 +389,7 @@ public class Node implements Closeable {
final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(namedWriteables);
NamedXContentRegistry xContentRegistry = new NamedXContentRegistry(Stream.of(
NetworkModule.getNamedXContents().stream(),
indicesModule.getNamedXContents().stream(),
searchModule.getNamedXContents().stream(),
pluginsService.filterPlugins(Plugin.class).stream()
.flatMap(p -> p.getNamedXContent().stream()),

View File

@ -37,6 +37,7 @@ import org.joda.time.format.DateTimeFormat;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
@ -44,6 +45,10 @@ import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.everyItem;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.collection.IsEmptyCollection.empty;
import static org.hamcrest.core.CombinableMatcher.both;
import static org.hamcrest.number.OrderingComparison.greaterThanOrEqualTo;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST)
public class RolloverIT extends ESIntegTestCase {
@ -70,6 +75,7 @@ public class RolloverIT extends ESIntegTestCase {
}
public void testRollover() throws Exception {
long beforeTime = client().threadPool().absoluteTimeInMillis() - 1000L;
assertAcked(prepareCreate("test_index-2").addAlias(new Alias("test_alias")).get());
index("test_index-2", "type1", "1", "field", "value");
flush("test_index-2");
@ -84,6 +90,11 @@ public class RolloverIT extends ESIntegTestCase {
assertFalse(oldIndex.getAliases().containsKey("test_alias"));
final IndexMetaData newIndex = state.metaData().index("test_index-000003");
assertTrue(newIndex.getAliases().containsKey("test_alias"));
assertThat(oldIndex.getRolloverInfos().size(), equalTo(1));
assertThat(oldIndex.getRolloverInfos().get("test_alias").getAlias(), equalTo("test_alias"));
assertThat(oldIndex.getRolloverInfos().get("test_alias").getMetConditions(), is(empty()));
assertThat(oldIndex.getRolloverInfos().get("test_alias").getTime(),
is(both(greaterThanOrEqualTo(beforeTime)).and(lessThanOrEqualTo(client().threadPool().absoluteTimeInMillis() + 1000L))));
}
public void testRolloverWithIndexSettings() throws Exception {
@ -246,17 +257,27 @@ public class RolloverIT extends ESIntegTestCase {
assertThat(response.getOldIndex(), equalTo("test-1"));
assertThat(response.getNewIndex(), equalTo("test-000002"));
assertThat("No rollover with a large max_size condition", response.isRolledOver(), equalTo(false));
final IndexMetaData oldIndex = client().admin().cluster().prepareState().get().getState().metaData().index("test-1");
assertThat(oldIndex.getRolloverInfos().size(), equalTo(0));
}
// A small max_size
{
ByteSizeValue maxSizeValue = new ByteSizeValue(randomIntBetween(1, 20), ByteSizeUnit.BYTES);
long beforeTime = client().threadPool().absoluteTimeInMillis() - 1000L;
final RolloverResponse response = client().admin().indices()
.prepareRolloverIndex("test_alias")
.addMaxIndexSizeCondition(new ByteSizeValue(randomIntBetween(1, 20), ByteSizeUnit.BYTES))
.addMaxIndexSizeCondition(maxSizeValue)
.get();
assertThat(response.getOldIndex(), equalTo("test-1"));
assertThat(response.getNewIndex(), equalTo("test-000002"));
assertThat("Should rollover with a small max_size condition", response.isRolledOver(), equalTo(true));
final IndexMetaData oldIndex = client().admin().cluster().prepareState().get().getState().metaData().index("test-1");
List<Condition> metConditions = oldIndex.getRolloverInfos().get("test_alias").getMetConditions();
assertThat(metConditions.size(), equalTo(1));
assertThat(metConditions.get(0).toString(), equalTo(new MaxSizeCondition(maxSizeValue).toString()));
assertThat(oldIndex.getRolloverInfos().get("test_alias").getTime(),
is(both(greaterThanOrEqualTo(beforeTime)).and(lessThanOrEqualTo(client().threadPool().absoluteTimeInMillis() + 1000L))));
}
// An empty index
@ -268,6 +289,8 @@ public class RolloverIT extends ESIntegTestCase {
assertThat(response.getOldIndex(), equalTo("test-000002"));
assertThat(response.getNewIndex(), equalTo("test-000003"));
assertThat("No rollover with an empty index", response.isRolledOver(), equalTo(false));
final IndexMetaData oldIndex = client().admin().cluster().prepareState().get().getState().metaData().index("test-000002");
assertThat(oldIndex.getRolloverInfos().size(), equalTo(0));
}
}

View File

@ -19,18 +19,31 @@
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.action.admin.indices.rollover.MaxAgeCondition;
import org.elasticsearch.action.admin.indices.rollover.MaxDocsCondition;
import org.elasticsearch.action.admin.indices.rollover.MaxSizeCondition;
import org.elasticsearch.action.admin.indices.rollover.RolloverInfo;
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.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.test.ESTestCase;
import org.junit.Before;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.Set;
@ -38,6 +51,23 @@ import static org.hamcrest.Matchers.is;
public class IndexMetaDataTests extends ESTestCase {
private IndicesModule INDICES_MODULE = new IndicesModule(Collections.emptyList());
@Before
public void setUp() throws Exception {
super.setUp();
}
@Override
protected NamedWriteableRegistry writableRegistry() {
return new NamedWriteableRegistry(INDICES_MODULE.getNamedWriteables());
}
@Override
protected NamedXContentRegistry xContentRegistry() {
return new NamedXContentRegistry(INDICES_MODULE.getNamedXContents());
}
public void testIndexMetaDataSerialization() throws IOException {
Integer numShard = randomFrom(1, 2, 4, 8, 16);
int numberOfReplicas = randomIntBetween(0, 10);
@ -50,7 +80,12 @@ public class IndexMetaDataTests extends ESTestCase {
.creationDate(randomLong())
.primaryTerm(0, 2)
.setRoutingNumShards(32)
.build();
.putRolloverInfo(
new RolloverInfo(randomAlphaOfLength(5),
Arrays.asList(new MaxAgeCondition(TimeValue.timeValueMillis(randomNonNegativeLong())),
new MaxSizeCondition(new ByteSizeValue(randomNonNegativeLong())),
new MaxDocsCondition(randomNonNegativeLong())),
randomNonNegativeLong())).build();
final XContentBuilder builder = JsonXContent.contentBuilder();
builder.startObject();
@ -71,17 +106,20 @@ public class IndexMetaDataTests extends ESTestCase {
final BytesStreamOutput out = new BytesStreamOutput();
metaData.writeTo(out);
IndexMetaData deserialized = IndexMetaData.readFrom(out.bytes().streamInput());
assertEquals(metaData, deserialized);
assertEquals(metaData.hashCode(), deserialized.hashCode());
try (StreamInput in = new NamedWriteableAwareStreamInput(out.bytes().streamInput(), writableRegistry())) {
IndexMetaData deserialized = IndexMetaData.readFrom(in);
assertEquals(metaData, deserialized);
assertEquals(metaData.hashCode(), deserialized.hashCode());
assertEquals(metaData.getNumberOfReplicas(), deserialized.getNumberOfReplicas());
assertEquals(metaData.getNumberOfShards(), deserialized.getNumberOfShards());
assertEquals(metaData.getCreationVersion(), deserialized.getCreationVersion());
assertEquals(metaData.getRoutingNumShards(), deserialized.getRoutingNumShards());
assertEquals(metaData.getCreationDate(), deserialized.getCreationDate());
assertEquals(metaData.getRoutingFactor(), deserialized.getRoutingFactor());
assertEquals(metaData.primaryTerm(0), deserialized.primaryTerm(0));
assertEquals(metaData.getNumberOfReplicas(), deserialized.getNumberOfReplicas());
assertEquals(metaData.getNumberOfShards(), deserialized.getNumberOfShards());
assertEquals(metaData.getCreationVersion(), deserialized.getCreationVersion());
assertEquals(metaData.getRoutingNumShards(), deserialized.getRoutingNumShards());
assertEquals(metaData.getCreationDate(), deserialized.getCreationDate());
assertEquals(metaData.getRoutingFactor(), deserialized.getRoutingFactor());
assertEquals(metaData.primaryTerm(0), deserialized.primaryTerm(0));
assertEquals(metaData.getRolloverInfos(), deserialized.getRolloverInfos());
}
}
public void testGetRoutingFactor() {