Browse Source

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.
Tal Levy 7 years ago
parent
commit
eda4964f64

+ 9 - 0
server/src/main/java/org/elasticsearch/action/admin/indices/rollover/MaxAgeCondition.java

@@ -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());
+        }
+    }
 }

+ 9 - 0
server/src/main/java/org/elasticsearch/action/admin/indices/rollover/MaxDocsCondition.java

@@ -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());
+        }
+    }
 }

+ 9 - 0
server/src/main/java/org/elasticsearch/action/admin/indices/rollover/MaxSizeCondition.java

@@ -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());
+        }
+    }
 }

+ 134 - 0
server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverInfo.java

@@ -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);
+    }
+}

+ 31 - 8
server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java

@@ -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));

+ 3 - 0
server/src/main/java/org/elasticsearch/client/transport/TransportClient.java

@@ -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())

+ 67 - 2
server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java

@@ -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?");

+ 19 - 4
server/src/main/java/org/elasticsearch/indices/IndicesModule.java

@@ -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<>();
 

+ 1 - 0
server/src/main/java/org/elasticsearch/node/Node.java

@@ -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()),

+ 24 - 1
server/src/test/java/org/elasticsearch/action/admin/indices/rollover/RolloverIT.java

@@ -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));
         }
     }
 

+ 50 - 12
server/src/test/java/org/elasticsearch/cluster/metadata/IndexMetaDataTests.java

@@ -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());
-
-        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));
+        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.getRolloverInfos(), deserialized.getRolloverInfos());
+        }
     }
 
     public void testGetRoutingFactor() {