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:
parent
da5bfda5f3
commit
eda4964f64
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,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));
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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?");
|
||||
|
|
|
@ -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<>();
|
||||
|
||||
|
|
|
@ -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()),
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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() {
|
||||
|
|
Loading…
Reference in New Issue