Move data stream transport and rest action to xpack (#59593)

Backport of #59525 to 7.x branch.

* Actions are moved to xpack core.
* Transport and rest actions are moved the data-streams module.
* Removed data streams methods from Client interface.
* Adjusted tests to use client.execute(...) instead of data stream specific methods.
* only attempt to delete all data streams if xpack is installed in rest tests
* Now that ds apis are in xpack and ESIntegTestCase
no longers deletes all ds, do that in the MlNativeIntegTestCase
class for ml tests.
This commit is contained in:
Martijn van Groningen 2020-07-15 16:50:44 +02:00 committed by GitHub
parent 8cac702171
commit 2a89e13e43
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
53 changed files with 1867 additions and 1793 deletions

View File

@ -20,7 +20,7 @@
package org.elasticsearch.client.indices;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.admin.indices.datastream.DataStreamsStatsAction;
import org.elasticsearch.xpack.core.action.DataStreamsStatsAction;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.client.AbstractResponseTestCase;
import org.elasticsearch.common.unit.ByteSizeValue;

View File

@ -19,8 +19,8 @@
package org.elasticsearch.client.indices;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction.Response.DataStreamInfo;
import org.elasticsearch.xpack.core.action.GetDataStreamAction;
import org.elasticsearch.xpack.core.action.GetDataStreamAction.Response.DataStreamInfo;
import org.elasticsearch.client.AbstractResponseTestCase;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.DataStream;

View File

@ -104,6 +104,7 @@ import org.elasticsearch.action.admin.indices.cache.clear.TransportClearIndicesC
import org.elasticsearch.action.admin.indices.close.CloseIndexAction;
import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction;
import org.elasticsearch.action.admin.indices.create.AutoCreateAction;
import org.elasticsearch.action.admin.indices.create.AutoCreateAction;
import org.elasticsearch.action.admin.indices.create.CreateIndexAction;
import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction;
import org.elasticsearch.action.admin.indices.dangling.delete.DeleteDanglingIndexAction;
@ -114,10 +115,6 @@ import org.elasticsearch.action.admin.indices.dangling.import_index.ImportDangli
import org.elasticsearch.action.admin.indices.dangling.import_index.TransportImportDanglingIndexAction;
import org.elasticsearch.action.admin.indices.dangling.list.ListDanglingIndicesAction;
import org.elasticsearch.action.admin.indices.dangling.list.TransportListDanglingIndicesAction;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.DataStreamsStatsAction;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction;
import org.elasticsearch.action.admin.indices.delete.TransportDeleteIndexAction;
import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsAction;
@ -308,12 +305,10 @@ import org.elasticsearch.rest.action.admin.indices.RestAddIndexBlockAction;
import org.elasticsearch.rest.action.admin.indices.RestAnalyzeAction;
import org.elasticsearch.rest.action.admin.indices.RestClearIndicesCacheAction;
import org.elasticsearch.rest.action.admin.indices.RestCloseIndexAction;
import org.elasticsearch.rest.action.admin.indices.RestCreateDataStreamAction;
import org.elasticsearch.rest.action.admin.indices.RestCreateIndexAction;
import org.elasticsearch.rest.action.admin.indices.RestDataStreamsStatsAction;
import org.elasticsearch.rest.action.admin.indices.RestDeleteComponentTemplateAction;
import org.elasticsearch.rest.action.admin.indices.RestDeleteComposableIndexTemplateAction;
import org.elasticsearch.rest.action.admin.indices.RestDeleteDataStreamAction;
import org.elasticsearch.rest.action.admin.indices.RestDeleteComposableIndexTemplateAction;
import org.elasticsearch.rest.action.admin.indices.RestDeleteIndexAction;
import org.elasticsearch.rest.action.admin.indices.RestDeleteIndexTemplateAction;
import org.elasticsearch.rest.action.admin.indices.RestFlushAction;
@ -321,7 +316,7 @@ import org.elasticsearch.rest.action.admin.indices.RestForceMergeAction;
import org.elasticsearch.rest.action.admin.indices.RestGetAliasesAction;
import org.elasticsearch.rest.action.admin.indices.RestGetComponentTemplateAction;
import org.elasticsearch.rest.action.admin.indices.RestGetComposableIndexTemplateAction;
import org.elasticsearch.rest.action.admin.indices.RestGetDataStreamsAction;
import org.elasticsearch.rest.action.admin.indices.RestGetComposableIndexTemplateAction;
import org.elasticsearch.rest.action.admin.indices.RestGetFieldMappingAction;
import org.elasticsearch.rest.action.admin.indices.RestGetIndexTemplateAction;
import org.elasticsearch.rest.action.admin.indices.RestGetIndicesAction;
@ -592,6 +587,7 @@ public class ActionModule extends AbstractModule {
actions.register(RecoveryAction.INSTANCE, TransportRecoveryAction.class);
actions.register(NodesReloadSecureSettingsAction.INSTANCE, TransportNodesReloadSecureSettingsAction.class);
actions.register(AutoCreateAction.INSTANCE, AutoCreateAction.TransportAction.class);
actions.register(ResolveIndexAction.INSTANCE, ResolveIndexAction.TransportAction.class);
//Indexed scripts
actions.register(PutStoredScriptAction.INSTANCE, TransportPutStoredScriptAction.class);
@ -610,13 +606,6 @@ public class ActionModule extends AbstractModule {
actionPlugins.stream().flatMap(p -> p.getActions().stream()).forEach(actions::register);
// Data streams:
actions.register(CreateDataStreamAction.INSTANCE, CreateDataStreamAction.TransportAction.class);
actions.register(DeleteDataStreamAction.INSTANCE, DeleteDataStreamAction.TransportAction.class);
actions.register(GetDataStreamAction.INSTANCE, GetDataStreamAction.TransportAction.class);
actions.register(ResolveIndexAction.INSTANCE, ResolveIndexAction.TransportAction.class);
actions.register(DataStreamsStatsAction.INSTANCE, DataStreamsStatsAction.TransportAction.class);
// Persistent tasks:
actions.register(StartPersistentTaskAction.INSTANCE, StartPersistentTaskAction.TransportAction.class);
actions.register(UpdatePersistentTaskStatusAction.INSTANCE, UpdatePersistentTaskStatusAction.TransportAction.class);
@ -722,6 +711,7 @@ public class ActionModule extends AbstractModule {
registerHandler.accept(new RestUpgradeAction());
registerHandler.accept(new RestUpgradeStatusAction());
registerHandler.accept(new RestClearIndicesCacheAction());
registerHandler.accept(new RestResolveIndexAction());
registerHandler.accept(new RestIndexAction());
registerHandler.accept(new CreateHandler());
@ -774,13 +764,6 @@ public class ActionModule extends AbstractModule {
registerHandler.accept(new RestImportDanglingIndexAction());
registerHandler.accept(new RestDeleteDanglingIndexAction());
// Data Stream API
registerHandler.accept(new RestCreateDataStreamAction());
registerHandler.accept(new RestDeleteDataStreamAction());
registerHandler.accept(new RestGetDataStreamsAction());
registerHandler.accept(new RestResolveIndexAction());
registerHandler.accept(new RestDataStreamsStatsAction());
// CAT API
registerHandler.accept(new RestAllocationAction());
registerHandler.accept(new RestShardsAction());

View File

@ -1,148 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.datastream;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.ValidateActions;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.AcknowledgedRequest;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetadataCreateDataStreamService;
import org.elasticsearch.cluster.metadata.MetadataCreateDataStreamService.CreateDataStreamClusterStateUpdateRequest;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.Objects;
public class CreateDataStreamAction extends ActionType<AcknowledgedResponse> {
public static final CreateDataStreamAction INSTANCE = new CreateDataStreamAction();
public static final String NAME = "indices:admin/data_stream/create";
private CreateDataStreamAction() {
super(NAME, AcknowledgedResponse::new);
}
public static class Request extends AcknowledgedRequest<Request> implements IndicesRequest {
private final String name;
public Request(String name) {
this.name = name;
}
@Override
public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = null;
if (Strings.hasText(name) == false) {
validationException = ValidateActions.addValidationError("name is missing", validationException);
}
return validationException;
}
public Request(StreamInput in) throws IOException {
super(in);
this.name = in.readString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(name);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Request request = (Request) o;
return name.equals(request.name);
}
@Override
public int hashCode() {
return Objects.hash(name);
}
@Override
public String[] indices() {
return new String[]{name};
}
@Override
public IndicesOptions indicesOptions() {
return IndicesOptions.strictSingleIndexNoExpandForbidClosed();
}
}
public static class TransportAction extends TransportMasterNodeAction<Request, AcknowledgedResponse> {
private final MetadataCreateDataStreamService metadataCreateDataStreamService;
@Inject
public TransportAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
MetadataCreateDataStreamService metadataCreateDataStreamService) {
super(NAME, transportService, clusterService, threadPool, actionFilters, Request::new, indexNameExpressionResolver);
this.metadataCreateDataStreamService = metadataCreateDataStreamService;
}
@Override
protected String executor() {
return ThreadPool.Names.SAME;
}
@Override
protected AcknowledgedResponse read(StreamInput in) throws IOException {
return new AcknowledgedResponse(in);
}
@Override
protected void masterOperation(Request request, ClusterState state,
ActionListener<AcknowledgedResponse> listener) throws Exception {
CreateDataStreamClusterStateUpdateRequest updateRequest = new CreateDataStreamClusterStateUpdateRequest(
request.name,
request.masterNodeTimeout(),
request.timeout()
);
metadataCreateDataStreamService.createDataStream(updateRequest, listener);
}
@Override
protected ClusterBlockException checkBlock(Request request, ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
}
}
}

View File

@ -1,441 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.datastream;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.PointValues;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexAbstractionResolver;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
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.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.index.store.StoreStats;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.SortedMap;
import java.util.stream.Stream;
public class DataStreamsStatsAction extends ActionType<DataStreamsStatsAction.Response> {
public static final DataStreamsStatsAction INSTANCE = new DataStreamsStatsAction();
public static final String NAME = "indices:monitor/data_stream/stats";
public DataStreamsStatsAction() {
super(NAME, DataStreamsStatsAction.Response::new);
}
public static class Request extends BroadcastRequest<Request> {
public Request() {
super((String[]) null);
}
public Request(StreamInput in) throws IOException {
super(in);
}
}
public static class Response extends BroadcastResponse {
private final int dataStreamCount;
private final int backingIndices;
private final ByteSizeValue totalStoreSize;
private final DataStreamStats[] dataStreams;
public Response(int totalShards, int successfulShards, int failedShards, List<DefaultShardOperationFailedException> shardFailures,
int dataStreamCount, int backingIndices, ByteSizeValue totalStoreSize, DataStreamStats[] dataStreams) {
super(totalShards, successfulShards, failedShards, shardFailures);
this.dataStreamCount = dataStreamCount;
this.backingIndices = backingIndices;
this.totalStoreSize = totalStoreSize;
this.dataStreams = dataStreams;
}
public Response(StreamInput in) throws IOException {
super(in);
this.dataStreamCount = in.readVInt();
this.backingIndices = in.readVInt();
this.totalStoreSize = new ByteSizeValue(in);
this.dataStreams = in.readArray(DataStreamStats::new, DataStreamStats[]::new);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(dataStreamCount);
out.writeVInt(backingIndices);
totalStoreSize.writeTo(out);
out.writeArray(dataStreams);
}
@Override
protected void addCustomXContentFields(XContentBuilder builder, Params params) throws IOException {
builder.field("data_stream_count", dataStreamCount);
builder.field("backing_indices", backingIndices);
builder.humanReadableField("total_store_size_bytes", "total_store_size", totalStoreSize);
builder.array("data_streams", (Object[]) dataStreams);
}
public int getDataStreamCount() {
return dataStreamCount;
}
public int getBackingIndices() {
return backingIndices;
}
public ByteSizeValue getTotalStoreSize() {
return totalStoreSize;
}
public DataStreamStats[] getDataStreams() {
return dataStreams;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
Response response = (Response) obj;
return dataStreamCount == response.dataStreamCount &&
backingIndices == response.backingIndices &&
Objects.equals(totalStoreSize, response.totalStoreSize) &&
Arrays.equals(dataStreams, response.dataStreams);
}
@Override
public int hashCode() {
int result = Objects.hash(dataStreamCount, backingIndices, totalStoreSize);
result = 31 * result + Arrays.hashCode(dataStreams);
return result;
}
@Override
public String toString() {
return "Response{" +
"dataStreamCount=" + dataStreamCount +
", backingIndices=" + backingIndices +
", totalStoreSize=" + totalStoreSize +
", dataStreams=" + Arrays.toString(dataStreams) +
'}';
}
}
public static class DataStreamStats implements ToXContentObject, Writeable {
private final String dataStream;
private final int backingIndices;
private final ByteSizeValue storeSize;
private final long maximumTimestamp;
public DataStreamStats(String dataStream, int backingIndices, ByteSizeValue storeSize, long maximumTimestamp) {
this.dataStream = dataStream;
this.backingIndices = backingIndices;
this.storeSize = storeSize;
this.maximumTimestamp = maximumTimestamp;
}
public DataStreamStats(StreamInput in) throws IOException {
this.dataStream = in.readString();
this.backingIndices = in.readVInt();
this.storeSize = new ByteSizeValue(in);
this.maximumTimestamp = in.readVLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(dataStream);
out.writeVInt(backingIndices);
storeSize.writeTo(out);
out.writeVLong(maximumTimestamp);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field("data_stream", dataStream);
builder.field("backing_indices", backingIndices);
builder.humanReadableField("store_size_bytes", "store_size", storeSize);
builder.field("maximum_timestamp", maximumTimestamp);
builder.endObject();
return builder;
}
public String getDataStream() {
return dataStream;
}
public int getBackingIndices() {
return backingIndices;
}
public ByteSizeValue getStoreSize() {
return storeSize;
}
public long getMaximumTimestamp() {
return maximumTimestamp;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
DataStreamStats that = (DataStreamStats) obj;
return backingIndices == that.backingIndices &&
maximumTimestamp == that.maximumTimestamp &&
Objects.equals(dataStream, that.dataStream) &&
Objects.equals(storeSize, that.storeSize);
}
@Override
public int hashCode() {
return Objects.hash(dataStream, backingIndices, storeSize, maximumTimestamp);
}
@Override
public String toString() {
return "DataStreamStats{" +
"dataStream='" + dataStream + '\'' +
", backingIndices=" + backingIndices +
", storeSize=" + storeSize +
", maximumTimestamp=" + maximumTimestamp +
'}';
}
}
public static class DataStreamShardStats implements Writeable {
private final ShardRouting shardRouting;
private final StoreStats storeStats;
private final long maxTimestamp;
public DataStreamShardStats(ShardRouting shardRouting, StoreStats storeStats, long maxTimestamp) {
this.shardRouting = shardRouting;
this.storeStats = storeStats;
this.maxTimestamp = maxTimestamp;
}
public DataStreamShardStats(StreamInput in) throws IOException {
this.shardRouting = new ShardRouting(in);
this.storeStats = new StoreStats(in);
this.maxTimestamp = in.readVLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
shardRouting.writeTo(out);
storeStats.writeTo(out);
out.writeVLong(maxTimestamp);
}
public ShardRouting getShardRouting() {
return shardRouting;
}
public StoreStats getStoreStats() {
return storeStats;
}
public long getMaxTimestamp() {
return maxTimestamp;
}
}
private static class AggregatedStats {
Set<String> backingIndices = new HashSet<>();
long storageBytes = 0L;
long maxTimestamp = 0L;
}
public static class TransportAction extends TransportBroadcastByNodeAction<Request, Response, DataStreamShardStats> {
private final ClusterService clusterService;
private final IndicesService indicesService;
private final IndexAbstractionResolver indexAbstractionResolver;
@Inject
public TransportAction(ClusterService clusterService, TransportService transportService, IndicesService indicesService,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
super(DataStreamsStatsAction.NAME, clusterService, transportService, actionFilters, indexNameExpressionResolver,
Request::new, ThreadPool.Names.MANAGEMENT);
this.clusterService = clusterService;
this.indicesService = indicesService;
this.indexAbstractionResolver = new IndexAbstractionResolver(indexNameExpressionResolver);
}
@Override
protected Request readRequestFrom(StreamInput in) throws IOException {
return new Request(in);
}
@Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, Request request) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ);
}
@Override
protected ClusterBlockException checkRequestBlock(ClusterState state, Request request, String[] concreteIndices) {
return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, concreteIndices);
}
@Override
protected ShardsIterator shards(ClusterState clusterState, Request request, String[] concreteIndices) {
String[] requestIndices = request.indices();
if (requestIndices == null || requestIndices.length == 0) {
requestIndices = new String[]{"*"};
}
List<String> abstractionNames = indexAbstractionResolver.resolveIndexAbstractions(requestIndices, request.indicesOptions(),
clusterState.getMetadata(), true); // Always include data streams for data streams stats api
SortedMap<String, IndexAbstraction> indicesLookup = clusterState.getMetadata().getIndicesLookup();
String[] concreteDatastreamIndices = abstractionNames.stream().flatMap(abstractionName -> {
IndexAbstraction indexAbstraction = indicesLookup.get(abstractionName);
assert indexAbstraction != null;
if (indexAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM) {
IndexAbstraction.DataStream dataStream = (IndexAbstraction.DataStream) indexAbstraction;
List<IndexMetadata> indices = dataStream.getIndices();
return indices.stream().map(idx -> idx.getIndex().getName());
} else {
return Stream.empty();
}
}).toArray(String[]::new);
return clusterState.getRoutingTable().allShards(concreteDatastreamIndices);
}
@Override
protected DataStreamShardStats shardOperation(Request request, ShardRouting shardRouting) throws IOException {
IndexService indexService = indicesService.indexServiceSafe(shardRouting.shardId().getIndex());
IndexShard indexShard = indexService.getShard(shardRouting.shardId().id());
// if we don't have the routing entry yet, we need it stats wise, we treat it as if the shard is not ready yet
if (indexShard.routingEntry() == null) {
throw new ShardNotFoundException(indexShard.shardId());
}
StoreStats storeStats = indexShard.storeStats();
IndexAbstraction indexAbstraction = clusterService.state().getMetadata().getIndicesLookup().get(shardRouting.getIndexName());
assert indexAbstraction != null;
IndexAbstraction.DataStream dataStream = indexAbstraction.getParentDataStream();
assert dataStream != null;
long maxTimestamp = 0L;
try (Engine.Searcher searcher = indexShard.acquireSearcher("data_stream_stats")) {
IndexReader indexReader = searcher.getIndexReader();
String fieldName = dataStream.getDataStream().getTimeStampField().getName();
byte[] maxPackedValue = PointValues.getMaxPackedValue(indexReader, fieldName);
if (maxPackedValue != null) {
maxTimestamp = LongPoint.decodeDimension(maxPackedValue, 0);
}
}
return new DataStreamShardStats(
indexShard.routingEntry(),
storeStats,
maxTimestamp
);
}
@Override
protected DataStreamShardStats readShardResult(StreamInput in) throws IOException {
return new DataStreamShardStats(in);
}
@Override
protected Response newResponse(Request request, int totalShards, int successfulShards,
int failedShards, List<DataStreamShardStats> dataStreamShardStats,
List<DefaultShardOperationFailedException> shardFailures,
ClusterState clusterState) {
Map<String, AggregatedStats> aggregatedDataStreamsStats = new HashMap<>();
Set<String> allBackingIndices = new HashSet<>();
long totalStoreSizeBytes = 0L;
SortedMap<String, IndexAbstraction> indicesLookup = clusterState.getMetadata().getIndicesLookup();
for (DataStreamShardStats shardStat : dataStreamShardStats) {
String indexName = shardStat.getShardRouting().getIndexName();
IndexAbstraction indexAbstraction = indicesLookup.get(indexName);
IndexAbstraction.DataStream dataStream = indexAbstraction.getParentDataStream();
assert dataStream != null;
// Aggregate global stats
totalStoreSizeBytes += shardStat.getStoreStats().sizeInBytes();
allBackingIndices.add(indexName);
// Aggregate data stream stats
AggregatedStats stats = aggregatedDataStreamsStats.computeIfAbsent(dataStream.getName(), s -> new AggregatedStats());
stats.storageBytes += shardStat.getStoreStats().sizeInBytes();
stats.maxTimestamp = Math.max(stats.maxTimestamp, shardStat.getMaxTimestamp());
stats.backingIndices.add(indexName);
}
DataStreamStats[] dataStreamStats = aggregatedDataStreamsStats.entrySet().stream()
.map(entry -> new DataStreamStats(
entry.getKey(),
entry.getValue().backingIndices.size(),
new ByteSizeValue(entry.getValue().storageBytes),
entry.getValue().maxTimestamp))
.toArray(DataStreamStats[]::new);
return new Response(
totalShards,
successfulShards,
failedShards,
shardFailures,
aggregatedDataStreamsStats.size(),
allBackingIndices.size(),
new ByteSizeValue(totalStoreSizeBytes),
dataStreamStats
);
}
}
}

View File

@ -1,233 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.datastream;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.support.master.MasterNodeRequest;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
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.DataStream;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.MetadataDeleteIndexService;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.index.Index;
import org.elasticsearch.snapshots.SnapshotInProgressException;
import org.elasticsearch.snapshots.SnapshotsService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Objects;
import java.util.Set;
import static org.elasticsearch.action.ValidateActions.addValidationError;
public class DeleteDataStreamAction extends ActionType<AcknowledgedResponse> {
private static final Logger logger = LogManager.getLogger(DeleteDataStreamAction.class);
public static final DeleteDataStreamAction INSTANCE = new DeleteDataStreamAction();
public static final String NAME = "indices:admin/data_stream/delete";
private DeleteDataStreamAction() {
super(NAME, AcknowledgedResponse::new);
}
public static class Request extends MasterNodeRequest<Request> implements IndicesRequest.Replaceable {
private String[] names;
public Request(String[] names) {
this.names = Objects.requireNonNull(names);
}
@Override
public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = null;
if (CollectionUtils.isEmpty(names)) {
validationException = addValidationError("no data stream(s) specified", validationException);
}
return validationException;
}
public Request(StreamInput in) throws IOException {
super(in);
this.names = in.readStringArray();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeStringArray(names);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Request request = (Request) o;
return Arrays.equals(names, request.names);
}
@Override
public int hashCode() {
return Arrays.hashCode(names);
}
@Override
public String[] indices() {
return names;
}
@Override
public IndicesOptions indicesOptions() {
// this doesn't really matter since data stream name resolution isn't affected by IndicesOptions and
// a data stream's backing indices are retrieved from its metadata
return IndicesOptions.fromOptions(false, true, true, true, false, false, true, false);
}
@Override
public boolean includeDataStreams() {
return true;
}
@Override
public IndicesRequest indices(String... indices) {
this.names = indices;
return this;
}
}
public static class TransportAction extends TransportMasterNodeAction<Request, AcknowledgedResponse> {
private final MetadataDeleteIndexService deleteIndexService;
@Inject
public TransportAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
MetadataDeleteIndexService deleteIndexService) {
super(NAME, transportService, clusterService, threadPool, actionFilters, Request::new, indexNameExpressionResolver);
this.deleteIndexService = deleteIndexService;
}
@Override
protected String executor() {
return ThreadPool.Names.SAME;
}
@Override
protected AcknowledgedResponse read(StreamInput in) throws IOException {
return new AcknowledgedResponse(in);
}
@Override
protected void masterOperation(Request request, ClusterState state,
ActionListener<AcknowledgedResponse> listener) throws Exception {
clusterService.submitStateUpdateTask("remove-data-stream [" + Strings.arrayToCommaDelimitedString(request.names) + "]",
new ClusterStateUpdateTask(Priority.HIGH) {
@Override
public TimeValue timeout() {
return request.masterNodeTimeout();
}
@Override
public void onFailure(String source, Exception e) {
listener.onFailure(e);
}
@Override
public ClusterState execute(ClusterState currentState) {
return removeDataStream(deleteIndexService, currentState, request);
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
listener.onResponse(new AcknowledgedResponse(true));
}
});
}
static ClusterState removeDataStream(MetadataDeleteIndexService deleteIndexService, ClusterState currentState, Request request) {
Set<String> dataStreams = new HashSet<>();
Set<String> snapshottingDataStreams = new HashSet<>();
for (String name : request.names) {
for (String dataStreamName : currentState.metadata().dataStreams().keySet()) {
if (Regex.simpleMatch(name, dataStreamName)) {
dataStreams.add(dataStreamName);
}
}
snapshottingDataStreams.addAll(SnapshotsService.snapshottingDataStreams(currentState, dataStreams));
}
if (snapshottingDataStreams.isEmpty() == false) {
throw new SnapshotInProgressException("Cannot delete data streams that are being snapshotted: " + snapshottingDataStreams +
". Try again after snapshot finishes or cancel the currently running snapshot.");
}
Set<Index> backingIndicesToRemove = new HashSet<>();
for (String dataStreamName : dataStreams) {
DataStream dataStream = currentState.metadata().dataStreams().get(dataStreamName);
assert dataStream != null;
backingIndicesToRemove.addAll(dataStream.getIndices());
}
// first delete the data streams and then the indices:
// (this to avoid data stream validation from failing when deleting an index that is part of a data stream
// without updating the data stream)
// TODO: change order when delete index api also updates the data stream the index to be removed is member of
Metadata.Builder metadata = Metadata.builder(currentState.metadata());
for (String ds : dataStreams) {
logger.info("removing data stream [{}]", ds);
metadata.removeDataStream(ds);
}
currentState = ClusterState.builder(currentState).metadata(metadata).build();
return deleteIndexService.deleteIndices(currentState, backingIndicesToRemove);
}
@Override
protected ClusterBlockException checkBlock(Request request, ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
}
}
}

View File

@ -39,9 +39,6 @@ import org.elasticsearch.action.admin.indices.close.CloseIndexResponse;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequest;
@ -844,36 +841,6 @@ public interface IndicesAdminClient extends ElasticsearchClient {
*/
void rolloverIndex(RolloverRequest request, ActionListener<RolloverResponse> listener);
/**
* Store a data stream
*/
void createDataStream(CreateDataStreamAction.Request request, ActionListener<AcknowledgedResponse> listener);
/**
* Store a data stream
*/
ActionFuture<AcknowledgedResponse> createDataStream(CreateDataStreamAction.Request request);
/**
* Delete a data stream
*/
void deleteDataStream(DeleteDataStreamAction.Request request, ActionListener<AcknowledgedResponse> listener);
/**
* Delete a data stream
*/
ActionFuture<AcknowledgedResponse> deleteDataStream(DeleteDataStreamAction.Request request);
/**
* Get data streams
*/
void getDataStreams(GetDataStreamAction.Request request, ActionListener<GetDataStreamAction.Response> listener);
/**
* Get data streams
*/
ActionFuture<GetDataStreamAction.Response> getDataStreams(GetDataStreamAction.Request request);
/**
* Resolves names and wildcard expressions to indices, aliases, and data streams
*/

View File

@ -166,9 +166,6 @@ import org.elasticsearch.action.admin.indices.dangling.import_index.ImportDangli
import org.elasticsearch.action.admin.indices.dangling.list.ListDanglingIndicesAction;
import org.elasticsearch.action.admin.indices.dangling.list.ListDanglingIndicesRequest;
import org.elasticsearch.action.admin.indices.dangling.list.ListDanglingIndicesResponse;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequestBuilder;
@ -1799,36 +1796,6 @@ public abstract class AbstractClient implements Client {
execute(GetSettingsAction.INSTANCE, request, listener);
}
@Override
public void createDataStream(CreateDataStreamAction.Request request, ActionListener<AcknowledgedResponse> listener) {
execute(CreateDataStreamAction.INSTANCE, request, listener);
}
@Override
public ActionFuture<AcknowledgedResponse> createDataStream(CreateDataStreamAction.Request request) {
return execute(CreateDataStreamAction.INSTANCE, request);
}
@Override
public void deleteDataStream(DeleteDataStreamAction.Request request, ActionListener<AcknowledgedResponse> listener) {
execute(DeleteDataStreamAction.INSTANCE, request, listener);
}
@Override
public ActionFuture<AcknowledgedResponse> deleteDataStream(DeleteDataStreamAction.Request request) {
return execute(DeleteDataStreamAction.INSTANCE, request);
}
@Override
public void getDataStreams(GetDataStreamAction.Request request, ActionListener<GetDataStreamAction.Response> listener) {
execute(GetDataStreamAction.INSTANCE, request, listener);
}
@Override
public ActionFuture<GetDataStreamAction.Response> getDataStreams(GetDataStreamAction.Request request) {
return execute(GetDataStreamAction.INSTANCE, request);
}
@Override
public void resolveIndex(ResolveIndexAction.Request request,
ActionListener<ResolveIndexAction.Response> listener) {

View File

@ -1,50 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.rest.action.admin.indices;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
public class RestCreateDataStreamAction extends BaseRestHandler {
@Override
public String getName() {
return "create_data_stream_action";
}
@Override
public List<Route> routes() {
return Collections.singletonList(
new Route(RestRequest.Method.PUT, "/_data_stream/{name}")
);
}
@Override
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
CreateDataStreamAction.Request putDataStreamRequest = new CreateDataStreamAction.Request(request.param("name"));
return channel -> client.admin().indices().createDataStream(putDataStreamRequest, new RestToXContentListener<>(channel));
}
}

View File

@ -1,50 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.rest.action.admin.indices;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.common.Strings;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
public class RestDeleteDataStreamAction extends BaseRestHandler {
@Override
public String getName() {
return "delete_data_stream_action";
}
@Override
public List<Route> routes() {
return Collections.singletonList(new Route(RestRequest.Method.DELETE, "/_data_stream/{name}"));
}
@Override
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
DeleteDataStreamAction.Request deleteDataStreamRequest = new DeleteDataStreamAction.Request(
Strings.splitStringByCommaToArray(request.param("name")));
return channel -> client.admin().indices().deleteDataStream(deleteDataStreamRequest, new RestToXContentListener<>(channel));
}
}

View File

@ -1,52 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.rest.action.admin.indices;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.common.Strings;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
import java.io.IOException;
import java.util.List;
public class RestGetDataStreamsAction extends BaseRestHandler {
@Override
public String getName() {
return "get_data_streams_action";
}
@Override
public List<Route> routes() {
return org.elasticsearch.common.collect.List.of(
new Route(RestRequest.Method.GET, "/_data_stream"),
new Route(RestRequest.Method.GET, "/_data_stream/{name}")
);
}
@Override
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
GetDataStreamAction.Request getDataStreamsRequest = new GetDataStreamAction.Request(
Strings.splitStringByCommaToArray(request.param("name")));
return channel -> client.admin().indices().getDataStreams(getDataStreamsRequest, new RestToXContentListener<>(channel));
}
}

View File

@ -1,222 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.datastream;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction.Request;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.MetadataDeleteIndexService;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.snapshots.SnapshotInProgressException;
import org.elasticsearch.test.AbstractWireSerializingTestCase;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import static org.elasticsearch.cluster.DataStreamTestHelper.createTimestampField;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class DeleteDataStreamRequestTests extends AbstractWireSerializingTestCase<Request> {
@Override
protected Writeable.Reader<Request> instanceReader() {
return Request::new;
}
@Override
protected Request createTestInstance() {
return new Request(randomArray(1, 3, String[]::new, () -> randomAlphaOfLength(6)));
}
public void testValidateRequest() {
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[]{"my-data-stream"});
ActionRequestValidationException e = req.validate();
assertNull(e);
}
public void testValidateRequestWithoutName() {
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[0]);
ActionRequestValidationException e = req.validate();
assertNotNull(e);
assertThat(e.validationErrors().size(), equalTo(1));
assertThat(e.validationErrors().get(0), containsString("no data stream(s) specified"));
}
public void testDeleteDataStream() {
final String dataStreamName = "my-data-stream";
final List<String> otherIndices = randomSubsetOf(org.elasticsearch.common.collect.List.of("foo", "bar", "baz"));
ClusterState cs = getClusterStateWithDataStreams(
org.elasticsearch.common.collect.List.of(new Tuple<>(dataStreamName, 2)), otherIndices);
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[]{dataStreamName});
ClusterState newState = DeleteDataStreamAction.TransportAction.removeDataStream(getMetadataDeleteIndexService(), cs, req);
assertThat(newState.metadata().dataStreams().size(), equalTo(0));
assertThat(newState.metadata().indices().size(), equalTo(otherIndices.size()));
for (String indexName : otherIndices) {
assertThat(newState.metadata().indices().get(indexName).getIndex().getName(), equalTo(indexName));
}
}
public void testDeleteMultipleDataStreams() {
String[] dataStreamNames = {"foo", "bar", "baz", "eggplant"};
ClusterState cs = getClusterStateWithDataStreams(org.elasticsearch.common.collect.List.of(
new Tuple<>(dataStreamNames[0], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[1], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[2], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[3], randomIntBetween(1, 3))
), org.elasticsearch.common.collect.List.of());
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[]{"ba*", "eggplant"});
ClusterState newState = DeleteDataStreamAction.TransportAction.removeDataStream(getMetadataDeleteIndexService(), cs, req);
assertThat(newState.metadata().dataStreams().size(), equalTo(1));
DataStream remainingDataStream = newState.metadata().dataStreams().get(dataStreamNames[0]);
assertNotNull(remainingDataStream);
assertThat(newState.metadata().indices().size(), equalTo(remainingDataStream.getIndices().size()));
for (Index i : remainingDataStream.getIndices()) {
assertThat(newState.metadata().indices().get(i.getName()).getIndex(), equalTo(i));
}
}
public void testDeleteSnapshottingDataStream() {
final String dataStreamName = "my-data-stream1";
final String dataStreamName2 = "my-data-stream2";
final List<String> otherIndices = randomSubsetOf(Arrays.asList("foo", "bar", "baz"));
ClusterState cs = getClusterStateWithDataStreams(Arrays.asList(new Tuple<>(dataStreamName, 2), new Tuple<>(dataStreamName2, 2)),
otherIndices);
SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.of(Arrays.asList(
createEntry(dataStreamName, "repo1", false),
createEntry(dataStreamName2, "repo2", true)));
ClusterState snapshotCs = ClusterState.builder(cs).putCustom(SnapshotsInProgress.TYPE, snapshotsInProgress).build();
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[]{dataStreamName});
SnapshotInProgressException e = expectThrows(SnapshotInProgressException.class,
() -> DeleteDataStreamAction.TransportAction.removeDataStream(getMetadataDeleteIndexService(), snapshotCs, req));
assertThat(e.getMessage(), equalTo("Cannot delete data streams that are being snapshotted: [my-data-stream1]. Try again after " +
"snapshot finishes or cancel the currently running snapshot."));
}
private SnapshotsInProgress.Entry createEntry(String dataStreamName, String repo, boolean partial) {
return new SnapshotsInProgress.Entry(new Snapshot(repo, new SnapshotId("", "")), false, partial,
SnapshotsInProgress.State.STARTED, Collections.emptyList(), Collections.singletonList(dataStreamName), 0, 1,
ImmutableOpenMap.of(), null, null, null);
}
public void testDeleteNonexistentDataStream() {
final String dataStreamName = "my-data-stream";
String[] dataStreamNames = {"foo", "bar", "baz", "eggplant"};
ClusterState cs = getClusterStateWithDataStreams(org.elasticsearch.common.collect.List.of(
new Tuple<>(dataStreamNames[0], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[1], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[2], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[3], randomIntBetween(1, 3))
), org.elasticsearch.common.collect.List.of());
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[]{dataStreamName});
ClusterState newState = DeleteDataStreamAction.TransportAction.removeDataStream(getMetadataDeleteIndexService(), cs, req);
assertThat(newState.metadata().dataStreams().size(), equalTo(cs.metadata().dataStreams().size()));
assertThat(newState.metadata().dataStreams().keySet(),
containsInAnyOrder(cs.metadata().dataStreams().keySet().toArray(Strings.EMPTY_ARRAY)));
}
@SuppressWarnings("unchecked")
private static MetadataDeleteIndexService getMetadataDeleteIndexService() {
MetadataDeleteIndexService s = mock(MetadataDeleteIndexService.class);
when(s.deleteIndices(any(ClusterState.class), any(Set.class)))
.thenAnswer(mockInvocation -> {
ClusterState currentState = (ClusterState) mockInvocation.getArguments()[0];
Set<Index> indices = (Set<Index>) mockInvocation.getArguments()[1];
final Metadata.Builder b = Metadata.builder(currentState.metadata());
for (Index index : indices) {
b.remove(index.getName());
}
return ClusterState.builder(currentState).metadata(b.build()).build();
});
return s;
}
/**
* Constructs {@code ClusterState} with the specified data streams and indices.
*
* @param dataStreams The names of the data streams to create with their respective number of backing indices
* @param indexNames The names of indices to create that do not back any data streams
*/
public static ClusterState getClusterStateWithDataStreams(List<Tuple<String, Integer>> dataStreams, List<String> indexNames) {
Metadata.Builder builder = Metadata.builder();
List<IndexMetadata> allIndices = new ArrayList<>();
for (Tuple<String, Integer> dsTuple : dataStreams) {
List<IndexMetadata> backingIndices = new ArrayList<>();
for (int backingIndexNumber = 1; backingIndexNumber <= dsTuple.v2(); backingIndexNumber++) {
backingIndices.add(createIndexMetadata(DataStream.getDefaultBackingIndexName(dsTuple.v1(), backingIndexNumber), true));
}
allIndices.addAll(backingIndices);
DataStream ds = new DataStream(dsTuple.v1(), createTimestampField("@timestamp"),
backingIndices.stream().map(IndexMetadata::getIndex).collect(Collectors.toList()), dsTuple.v2());
builder.put(ds);
}
for (String indexName : indexNames) {
allIndices.add(createIndexMetadata(indexName, false));
}
for (IndexMetadata index : allIndices) {
builder.put(index, false);
}
return ClusterState.builder(new ClusterName("_name")).metadata(builder).build();
}
private static IndexMetadata createIndexMetadata(String name, boolean hidden) {
Settings.Builder b = Settings.builder()
.put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT)
.put("index.hidden", hidden);
return IndexMetadata.builder(name)
.settings(b)
.numberOfShards(1)
.numberOfReplicas(1)
.build();
}
}

View File

@ -1,143 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.datastream;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction.Request;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.test.AbstractWireSerializingTestCase;
import java.util.List;
import static org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamRequestTests.getClusterStateWithDataStreams;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
public class GetDataStreamsRequestTests extends AbstractWireSerializingTestCase<Request> {
@Override
protected Writeable.Reader<Request> instanceReader() {
return Request::new;
}
@Override
protected Request createTestInstance() {
final String[] searchParameter;
switch (randomIntBetween(1, 4)) {
case 1:
searchParameter = generateRandomStringArray(3, 8, false, false);
break;
case 2:
String[] parameters = generateRandomStringArray(3, 8, false, false);
for (int k = 0; k < parameters.length; k++) {
parameters[k] = parameters[k] + "*";
}
searchParameter = parameters;
break;
case 3:
searchParameter = new String[]{"*"};
break;
default:
searchParameter = null;
break;
}
return new Request(searchParameter);
}
public void testGetDataStream() {
final String dataStreamName = "my-data-stream";
ClusterState cs = getClusterStateWithDataStreams(
org.elasticsearch.common.collect.List.of(new Tuple<>(dataStreamName, 1)), org.elasticsearch.common.collect.List.of());
GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[]{dataStreamName});
List<DataStream> dataStreams = GetDataStreamAction.TransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(1));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamName));
}
public void testGetDataStreamsWithWildcards() {
final String[] dataStreamNames = {"my-data-stream", "another-data-stream"};
ClusterState cs = getClusterStateWithDataStreams(
org.elasticsearch.common.collect.List.of(new Tuple<>(dataStreamNames[0], 1), new Tuple<>(dataStreamNames[1], 1)),
org.elasticsearch.common.collect.List.of());
GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[]{dataStreamNames[1].substring(0, 5) + "*"});
List<DataStream> dataStreams = GetDataStreamAction.TransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(1));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1]));
req = new GetDataStreamAction.Request(new String[]{"*"});
dataStreams = GetDataStreamAction.TransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(2));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1]));
assertThat(dataStreams.get(1).getName(), equalTo(dataStreamNames[0]));
req = new GetDataStreamAction.Request((String[]) null);
dataStreams = GetDataStreamAction.TransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(2));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1]));
assertThat(dataStreams.get(1).getName(), equalTo(dataStreamNames[0]));
req = new GetDataStreamAction.Request(new String[]{"matches-none*"});
dataStreams = GetDataStreamAction.TransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(0));
}
public void testGetDataStreamsWithoutWildcards() {
final String[] dataStreamNames = {"my-data-stream", "another-data-stream"};
ClusterState cs = getClusterStateWithDataStreams(
org.elasticsearch.common.collect.List.of(new Tuple<>(dataStreamNames[0], 1), new Tuple<>(dataStreamNames[1], 1)),
org.elasticsearch.common.collect.List.of());
GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[]{dataStreamNames[0], dataStreamNames[1]});
List<DataStream> dataStreams = GetDataStreamAction.TransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(2));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1]));
assertThat(dataStreams.get(1).getName(), equalTo(dataStreamNames[0]));
req = new GetDataStreamAction.Request(new String[]{dataStreamNames[1]});
dataStreams = GetDataStreamAction.TransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(1));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1]));
req = new GetDataStreamAction.Request(new String[]{dataStreamNames[0]});
dataStreams = GetDataStreamAction.TransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(1));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[0]));
GetDataStreamAction.Request req2 = new GetDataStreamAction.Request(new String[]{"foo"});
IndexNotFoundException e = expectThrows(IndexNotFoundException.class,
() -> GetDataStreamAction.TransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req2));
assertThat(e.getMessage(), containsString("no such index [foo]"));
}
public void testGetNonexistentDataStream() {
final String dataStreamName = "my-data-stream";
ClusterState cs = ClusterState.builder(new ClusterName("_name")).build();
GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[]{dataStreamName});
IndexNotFoundException e = expectThrows(IndexNotFoundException.class,
() -> GetDataStreamAction.TransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req));
assertThat(e.getMessage(), containsString("no such index [" + dataStreamName + "]"));
}
}

View File

@ -1,47 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.datastream;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction.Response;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.DataStreamTests;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.test.AbstractWireSerializingTestCase;
import java.util.ArrayList;
import java.util.List;
public class GetDataStreamsResponseTests extends AbstractWireSerializingTestCase<Response> {
@Override
protected Writeable.Reader<Response> instanceReader() {
return Response::new;
}
@Override
protected Response createTestInstance() {
int numDataStreams = randomIntBetween(0, 8);
List<Response.DataStreamInfo> dataStreams = new ArrayList<>();
for (int i = 0; i < numDataStreams; i++) {
dataStreams.add(new Response.DataStreamInfo(DataStreamTests.randomInstance(), ClusterHealthStatus.GREEN,
randomAlphaOfLengthBetween(2, 10), randomAlphaOfLengthBetween(2, 10)));
}
return new Response(dataStreams);
}
}

View File

@ -20,8 +20,8 @@
package org.elasticsearch.action.admin.indices.mapping.put;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamRequestTests;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.DataStreamTestHelper;
import org.elasticsearch.cluster.metadata.AliasMetadata;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexMetadata;
@ -161,7 +161,7 @@ public class PutMappingRequestTests extends ESTestCase {
tuple(dataStreamNames[1], randomIntBetween(1, 3)),
tuple(dataStreamNames[2], randomIntBetween(1, 3)));
ClusterState cs = DeleteDataStreamRequestTests.getClusterStateWithDataStreams(dsMetadata,
ClusterState cs = DataStreamTestHelper.getClusterStateWithDataStreams(dsMetadata,
org.elasticsearch.common.collect.List.of("index1", "index2", "index3"));
cs = addAliases(cs, org.elasticsearch.common.collect.List.of(
tuple("alias1", org.elasticsearch.common.collect.List.of(tuple("index1", false), tuple("index2", true))),
@ -182,7 +182,7 @@ public class PutMappingRequestTests extends ESTestCase {
tuple(dataStreamNames[1], randomIntBetween(1, 3)),
tuple(dataStreamNames[2], randomIntBetween(1, 3)));
ClusterState cs = DeleteDataStreamRequestTests.getClusterStateWithDataStreams(dsMetadata,
ClusterState cs = DataStreamTestHelper.getClusterStateWithDataStreams(dsMetadata,
org.elasticsearch.common.collect.List.of("index1", "index2", "index3"));
cs = addAliases(cs, org.elasticsearch.common.collect.List.of(
tuple("alias1", org.elasticsearch.common.collect.List.of(tuple("index1", false), tuple("index2", true))),
@ -205,7 +205,7 @@ public class PutMappingRequestTests extends ESTestCase {
tuple(dataStreamNames[1], randomIntBetween(1, 3)),
tuple(dataStreamNames[2], randomIntBetween(1, 3)));
ClusterState cs = DeleteDataStreamRequestTests.getClusterStateWithDataStreams(dsMetadata,
ClusterState cs = DataStreamTestHelper.getClusterStateWithDataStreams(dsMetadata,
org.elasticsearch.common.collect.List.of("index1", "index2", "index3"));
cs = addAliases(cs, org.elasticsearch.common.collect.List.of(
tuple("alias1", org.elasticsearch.common.collect.List.of(tuple("index1", false), tuple("index2", true))),
@ -228,7 +228,7 @@ public class PutMappingRequestTests extends ESTestCase {
tuple(dataStreamNames[1], randomIntBetween(1, 3)),
tuple(dataStreamNames[2], randomIntBetween(1, 3)));
ClusterState cs = DeleteDataStreamRequestTests.getClusterStateWithDataStreams(dsMetadata,
ClusterState cs = DataStreamTestHelper.getClusterStateWithDataStreams(dsMetadata,
org.elasticsearch.common.collect.List.of("index1", "index2", "index3"));
final ClusterState cs2 = addAliases(cs, org.elasticsearch.common.collect.List.of(
tuple("alias1", org.elasticsearch.common.collect.List.of(tuple("index1", false), tuple("index2", true))),
@ -247,7 +247,7 @@ public class PutMappingRequestTests extends ESTestCase {
tuple(dataStreamNames[1], randomIntBetween(1, 3)),
tuple(dataStreamNames[2], randomIntBetween(1, 3)));
ClusterState cs = DeleteDataStreamRequestTests.getClusterStateWithDataStreams(dsMetadata,
ClusterState cs = DataStreamTestHelper.getClusterStateWithDataStreams(dsMetadata,
org.elasticsearch.common.collect.List.of("index1", "index2", "index3"));
final ClusterState cs2 = addAliases(cs, org.elasticsearch.common.collect.List.of(
tuple("alias1", org.elasticsearch.common.collect.List.of(tuple("index1", false), tuple("index2", false))),

View File

@ -34,7 +34,6 @@ import org.elasticsearch.cluster.metadata.AliasValidator;
import org.elasticsearch.cluster.metadata.ComponentTemplate;
import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.DataStreamTests;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
@ -258,7 +257,7 @@ public class MetadataRolloverServiceTests extends ESTestCase {
public void testDataStreamValidation() throws IOException {
Metadata.Builder md = Metadata.builder();
DataStream randomDataStream = DataStreamTests.randomInstance();
DataStream randomDataStream = DataStreamTestHelper.randomInstance();
for (Index index : randomDataStream.getIndices()) {
md.put(DataStreamTestHelper.getIndexMetadataBuilderForIndex(index));
}
@ -332,7 +331,7 @@ public class MetadataRolloverServiceTests extends ESTestCase {
}
public void testCreateIndexRequestForDataStream() {
DataStream dataStream = DataStreamTests.randomInstance();
DataStream dataStream = DataStreamTestHelper.randomInstance();
final String newWriteIndexName = DataStream.getDefaultBackingIndexName(dataStream.getName(), dataStream.getGeneration() + 1);
final RolloverRequest rolloverRequest = new RolloverRequest(dataStream.getName(), randomAlphaOfLength(10));
final ActiveShardCount activeShardCount = randomBoolean() ? ActiveShardCount.ALL : ActiveShardCount.ONE;
@ -537,7 +536,7 @@ public class MetadataRolloverServiceTests extends ESTestCase {
}
public void testRolloverClusterStateForDataStream() throws Exception {
final DataStream dataStream = DataStreamTests.randomInstance();
final DataStream dataStream = DataStreamTestHelper.randomInstance();
ComposableIndexTemplate template = new ComposableIndexTemplate(Collections.singletonList(dataStream.getName() + "*"),
null, null, null, null, null, new ComposableIndexTemplate.DataStreamTemplate());
Metadata.Builder builder = Metadata.builder();
@ -633,7 +632,7 @@ public class MetadataRolloverServiceTests extends ESTestCase {
final boolean useDataStream = randomBoolean();
final Metadata.Builder builder = Metadata.builder();
if (useDataStream) {
DataStream dataStream = DataStreamTests.randomInstance();
DataStream dataStream = DataStreamTestHelper.randomInstance();
rolloverTarget = dataStream.getName();
sourceIndexName = dataStream.getIndices().get(dataStream.getIndices().size() - 1).getName();
defaultRolloverIndexName = DataStream.getDefaultBackingIndexName(dataStream.getName(), dataStream.getGeneration() + 1);
@ -690,7 +689,7 @@ public class MetadataRolloverServiceTests extends ESTestCase {
}
public void testRolloverClusterStateForDataStreamNoTemplate() throws Exception {
final DataStream dataStream = DataStreamTests.randomInstance();
final DataStream dataStream = DataStreamTestHelper.randomInstance();
Metadata.Builder builder = Metadata.builder();
for (Index index : dataStream.getIndices()) {
builder.put(DataStreamTestHelper.getIndexMetadataBuilderForIndex(index));

View File

@ -19,6 +19,7 @@
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.cluster.DataStreamTestHelper;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.test.AbstractNamedWriteableTestCase;
@ -36,7 +37,7 @@ public class DataStreamMetadataTests extends AbstractNamedWriteableTestCase<Data
}
Map<String, DataStream> dataStreams = new HashMap<>();
for (int i = 0; i < randomIntBetween(1, 5); i++) {
dataStreams.put(randomAlphaOfLength(5), DataStreamTests.randomInstance());
dataStreams.put(randomAlphaOfLength(5), DataStreamTestHelper.randomInstance());
}
return new DataStreamMetadata(dataStreams);
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.cluster.DataStreamTestHelper;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.XContentParser;
@ -35,23 +36,6 @@ import static org.hamcrest.Matchers.equalTo;
public class DataStreamTests extends AbstractSerializingTestCase<DataStream> {
public static List<Index> randomIndexInstances() {
int numIndices = randomIntBetween(0, 128);
List<Index> indices = new ArrayList<>(numIndices);
for (int i = 0; i < numIndices; i++) {
indices.add(new Index(randomAlphaOfLength(10).toLowerCase(Locale.ROOT), UUIDs.randomBase64UUID(random())));
}
return indices;
}
public static DataStream randomInstance() {
List<Index> indices = randomIndexInstances();
long generation = indices.size() + randomLongBetween(1, 128);
String dataStreamName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
indices.add(new Index(getDefaultBackingIndexName(dataStreamName, generation), UUIDs.randomBase64UUID(random())));
return new DataStream(dataStreamName, createTimestampField("@timestamp"), indices, generation);
}
@Override
protected DataStream doParseInstance(XContentParser parser) throws IOException {
return DataStream.fromXContent(parser);
@ -64,11 +48,11 @@ public class DataStreamTests extends AbstractSerializingTestCase<DataStream> {
@Override
protected DataStream createTestInstance() {
return randomInstance();
return DataStreamTestHelper.randomInstance();
}
public void testRollover() {
DataStream ds = randomInstance();
DataStream ds = DataStreamTestHelper.randomInstance();
Index newWriteIndex = new Index(getDefaultBackingIndexName(ds.getName(), ds.getGeneration() + 1), UUIDs.randomBase64UUID(random()));
DataStream rolledDs = ds.rollover(newWriteIndex);

View File

@ -16,12 +16,11 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamRequestTests;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.DataStreamTestHelper;
import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.routing.RoutingTable;
@ -115,7 +114,7 @@ public class MetadataDeleteIndexServiceTests extends ESTestCase {
public void testDeleteBackingIndexForDataStream() {
int numBackingIndices = randomIntBetween(2, 5);
String dataStreamName = randomAlphaOfLength(6).toLowerCase(Locale.ROOT);
ClusterState before = DeleteDataStreamRequestTests.getClusterStateWithDataStreams(
ClusterState before = DataStreamTestHelper.getClusterStateWithDataStreams(
List.of(new Tuple<>(dataStreamName, numBackingIndices)), List.of());
int numIndexToDelete = randomIntBetween(1, numBackingIndices - 1);
@ -132,7 +131,7 @@ public class MetadataDeleteIndexServiceTests extends ESTestCase {
public void testDeleteCurrentWriteIndexForDataStream() {
int numBackingIndices = randomIntBetween(1, 5);
String dataStreamName = randomAlphaOfLength(6).toLowerCase(Locale.ROOT);
ClusterState before = DeleteDataStreamRequestTests.getClusterStateWithDataStreams(
ClusterState before = DataStreamTestHelper.getClusterStateWithDataStreams(
List.of(new Tuple<>(dataStreamName, numBackingIndices)), List.of());
Index indexToDelete = before.metadata().index(DataStream.getDefaultBackingIndexName(dataStreamName, numBackingIndices)).getIndex();

View File

@ -23,9 +23,9 @@ import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest;
import org.elasticsearch.action.admin.indices.close.CloseIndexResponse;
import org.elasticsearch.action.admin.indices.close.CloseIndexResponse.IndexResult;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamRequestTests;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.DataStreamTestHelper;
import org.elasticsearch.cluster.RestoreInProgress;
import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.block.ClusterBlock;
@ -401,7 +401,7 @@ public class MetadataIndexStateServiceTests extends ESTestCase {
dataStreamsToCreate.add(new Tuple<>(dataStreamName, numBackingIndices));
writeIndices.add(DataStream.getDefaultBackingIndexName(dataStreamName, numBackingIndices));
}
ClusterState cs = DeleteDataStreamRequestTests.getClusterStateWithDataStreams(dataStreamsToCreate,
ClusterState cs = DataStreamTestHelper.getClusterStateWithDataStreams(dataStreamsToCreate,
org.elasticsearch.common.collect.List.of());
ClusterService clusterService = mock(ClusterService.class);

View File

@ -1275,7 +1275,7 @@ public class MetadataTests extends ESTestCase {
.put("component_template_" + randomAlphaOfLength(3), ComponentTemplateTests.randomInstance())
.put("index_template_v2_" + randomAlphaOfLength(3), ComposableIndexTemplateTests.randomInstance());
DataStream randomDataStream = DataStreamTests.randomInstance();
DataStream randomDataStream = DataStreamTestHelper.randomInstance();
for (Index index : randomDataStream.getIndices()) {
md.put(DataStreamTestHelper.getIndexMetadataBuilderForIndex(index));
}

View File

@ -16,17 +16,27 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.test.ESTestCase;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
import java.util.stream.Collectors;
import static org.elasticsearch.cluster.metadata.DataStream.getDefaultBackingIndexName;
import static org.elasticsearch.cluster.metadata.IndexMetadata.SETTING_INDEX_UUID;
import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength;
public final class DataStreamTestHelper {
@ -78,4 +88,64 @@ public final class DataStreamTestHelper {
" }\n" +
" }";
}
public static List<Index> randomIndexInstances() {
int numIndices = ESTestCase.randomIntBetween(0, 128);
List<Index> indices = new ArrayList<>(numIndices);
for (int i = 0; i < numIndices; i++) {
indices.add(new Index(randomAlphaOfLength(10).toLowerCase(Locale.ROOT), UUIDs.randomBase64UUID(LuceneTestCase.random())));
}
return indices;
}
public static DataStream randomInstance() {
List<Index> indices = randomIndexInstances();
long generation = indices.size() + ESTestCase.randomLongBetween(1, 128);
String dataStreamName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
indices.add(new Index(getDefaultBackingIndexName(dataStreamName, generation), UUIDs.randomBase64UUID(LuceneTestCase.random())));
return new DataStream(dataStreamName, createTimestampField("@timestamp"), indices, generation);
}
/**
* Constructs {@code ClusterState} with the specified data streams and indices.
*
* @param dataStreams The names of the data streams to create with their respective number of backing indices
* @param indexNames The names of indices to create that do not back any data streams
*/
public static ClusterState getClusterStateWithDataStreams(List<Tuple<String, Integer>> dataStreams, List<String> indexNames) {
Metadata.Builder builder = Metadata.builder();
List<IndexMetadata> allIndices = new ArrayList<>();
for (Tuple<String, Integer> dsTuple : dataStreams) {
List<IndexMetadata> backingIndices = new ArrayList<>();
for (int backingIndexNumber = 1; backingIndexNumber <= dsTuple.v2(); backingIndexNumber++) {
backingIndices.add(createIndexMetadata(getDefaultBackingIndexName(dsTuple.v1(), backingIndexNumber), true));
}
allIndices.addAll(backingIndices);
DataStream ds = new DataStream(
dsTuple.v1(),
createTimestampField("@timestamp"),
backingIndices.stream().map(IndexMetadata::getIndex).collect(Collectors.toList()),
dsTuple.v2()
);
builder.put(ds);
}
for (String indexName : indexNames) {
allIndices.add(createIndexMetadata(indexName, false));
}
for (IndexMetadata index : allIndices) {
builder.put(index, false);
}
return ClusterState.builder(new ClusterName("_name")).metadata(builder).build();
}
private static IndexMetadata createIndexMetadata(String name, boolean hidden) {
Settings.Builder b = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).put("index.hidden", hidden);
return IndexMetadata.builder(name).settings(b).numberOfShards(1).numberOfReplicas(1).build();
}
}

View File

@ -20,14 +20,11 @@
package org.elasticsearch.test;
import com.carrotsearch.hppc.ObjectArrayList;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexTemplateMetadata;
@ -79,7 +76,6 @@ public abstract class TestCluster implements Closeable {
* Wipes any data that a test can leave behind: indices, templates (except exclude templates) and repositories
*/
public void wipe(Set<String> excludeTemplates) {
wipeAllDataStreams();
wipeIndices("_all");
wipeAllTemplates(excludeTemplates);
wipeRepositories();
@ -135,18 +131,6 @@ public abstract class TestCluster implements Closeable {
@Override
public abstract void close() throws IOException;
/**
* Deletes all data streams from the test cluster.
*/
public void wipeAllDataStreams() {
// Feature flag may not be enabled in all gradle modules that use ESIntegTestCase
if (size() > 0) {
AcknowledgedResponse response =
client().admin().indices().deleteDataStream(new DeleteDataStreamAction.Request(new String[]{"*"})).actionGet();
assertAcked(response);
}
}
/**
* Deletes the given indices from the tests cluster. If no index name is passed to this method
* all indices are removed.

View File

@ -687,7 +687,9 @@ public abstract class ESRestTestCase extends ESTestCase {
protected static void wipeDataStreams() throws IOException {
try {
adminClient().performRequest(new Request("DELETE", "_data_stream/*"));
if (hasXPack()) {
adminClient().performRequest(new Request("DELETE", "_data_stream/*"));
}
} catch (ResponseException e) {
// We hit a version of ES that doesn't have data streams enabled so it's safe to ignore
if (e.getResponse().getStatusLine().getStatusCode() != 405 && e.getResponse().getStatusLine().getStatusCode() != 500) {

View File

@ -0,0 +1,87 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.core.action;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.ValidateActions;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.AcknowledgedRequest;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
import java.util.Objects;
public class CreateDataStreamAction extends ActionType<AcknowledgedResponse> {
public static final CreateDataStreamAction INSTANCE = new CreateDataStreamAction();
public static final String NAME = "indices:admin/data_stream/create";
private CreateDataStreamAction() {
super(NAME, AcknowledgedResponse::new);
}
public static class Request extends AcknowledgedRequest<Request> implements IndicesRequest {
private final String name;
public Request(String name) {
this.name = name;
}
public String getName() {
return name;
}
@Override
public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = null;
if (Strings.hasText(name) == false) {
validationException = ValidateActions.addValidationError("name is missing", validationException);
}
return validationException;
}
public Request(StreamInput in) throws IOException {
super(in);
this.name = in.readString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(name);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Request request = (Request) o;
return name.equals(request.name);
}
@Override
public int hashCode() {
return Objects.hash(name);
}
@Override
public String[] indices() {
return new String[]{name};
}
@Override
public IndicesOptions indicesOptions() {
return IndicesOptions.strictSingleIndexNoExpandForbidClosed();
}
}
}

View File

@ -0,0 +1,257 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.core.action;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.routing.ShardRouting;
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.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.store.StoreStats;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
public class DataStreamsStatsAction extends ActionType<DataStreamsStatsAction.Response> {
public static final DataStreamsStatsAction INSTANCE = new DataStreamsStatsAction();
public static final String NAME = "indices:monitor/data_stream/stats";
public DataStreamsStatsAction() {
super(NAME, DataStreamsStatsAction.Response::new);
}
public static class Request extends BroadcastRequest<Request> {
public Request() {
super((String[]) null);
}
public Request(StreamInput in) throws IOException {
super(in);
}
}
public static class Response extends BroadcastResponse {
private final int dataStreamCount;
private final int backingIndices;
private final ByteSizeValue totalStoreSize;
private final DataStreamStats[] dataStreams;
public Response(int totalShards, int successfulShards, int failedShards, List<DefaultShardOperationFailedException> shardFailures,
int dataStreamCount, int backingIndices, ByteSizeValue totalStoreSize, DataStreamStats[] dataStreams) {
super(totalShards, successfulShards, failedShards, shardFailures);
this.dataStreamCount = dataStreamCount;
this.backingIndices = backingIndices;
this.totalStoreSize = totalStoreSize;
this.dataStreams = dataStreams;
}
public Response(StreamInput in) throws IOException {
super(in);
this.dataStreamCount = in.readVInt();
this.backingIndices = in.readVInt();
this.totalStoreSize = new ByteSizeValue(in);
this.dataStreams = in.readArray(DataStreamStats::new, DataStreamStats[]::new);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(dataStreamCount);
out.writeVInt(backingIndices);
totalStoreSize.writeTo(out);
out.writeArray(dataStreams);
}
@Override
protected void addCustomXContentFields(XContentBuilder builder, Params params) throws IOException {
builder.field("data_stream_count", dataStreamCount);
builder.field("backing_indices", backingIndices);
builder.humanReadableField("total_store_size_bytes", "total_store_size", totalStoreSize);
builder.array("data_streams", (Object[]) dataStreams);
}
public int getDataStreamCount() {
return dataStreamCount;
}
public int getBackingIndices() {
return backingIndices;
}
public ByteSizeValue getTotalStoreSize() {
return totalStoreSize;
}
public DataStreamStats[] getDataStreams() {
return dataStreams;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
Response response = (Response) obj;
return dataStreamCount == response.dataStreamCount &&
backingIndices == response.backingIndices &&
Objects.equals(totalStoreSize, response.totalStoreSize) &&
Arrays.equals(dataStreams, response.dataStreams);
}
@Override
public int hashCode() {
int result = Objects.hash(dataStreamCount, backingIndices, totalStoreSize);
result = 31 * result + Arrays.hashCode(dataStreams);
return result;
}
@Override
public String toString() {
return "Response{" +
"dataStreamCount=" + dataStreamCount +
", backingIndices=" + backingIndices +
", totalStoreSize=" + totalStoreSize +
", dataStreams=" + Arrays.toString(dataStreams) +
'}';
}
}
public static class DataStreamStats implements ToXContentObject, Writeable {
private final String dataStream;
private final int backingIndices;
private final ByteSizeValue storeSize;
private final long maximumTimestamp;
public DataStreamStats(String dataStream, int backingIndices, ByteSizeValue storeSize, long maximumTimestamp) {
this.dataStream = dataStream;
this.backingIndices = backingIndices;
this.storeSize = storeSize;
this.maximumTimestamp = maximumTimestamp;
}
public DataStreamStats(StreamInput in) throws IOException {
this.dataStream = in.readString();
this.backingIndices = in.readVInt();
this.storeSize = new ByteSizeValue(in);
this.maximumTimestamp = in.readVLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(dataStream);
out.writeVInt(backingIndices);
storeSize.writeTo(out);
out.writeVLong(maximumTimestamp);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field("data_stream", dataStream);
builder.field("backing_indices", backingIndices);
builder.humanReadableField("store_size_bytes", "store_size", storeSize);
builder.field("maximum_timestamp", maximumTimestamp);
builder.endObject();
return builder;
}
public String getDataStream() {
return dataStream;
}
public int getBackingIndices() {
return backingIndices;
}
public ByteSizeValue getStoreSize() {
return storeSize;
}
public long getMaximumTimestamp() {
return maximumTimestamp;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
DataStreamStats that = (DataStreamStats) obj;
return backingIndices == that.backingIndices &&
maximumTimestamp == that.maximumTimestamp &&
Objects.equals(dataStream, that.dataStream) &&
Objects.equals(storeSize, that.storeSize);
}
@Override
public int hashCode() {
return Objects.hash(dataStream, backingIndices, storeSize, maximumTimestamp);
}
@Override
public String toString() {
return "DataStreamStats{" +
"dataStream='" + dataStream + '\'' +
", backingIndices=" + backingIndices +
", storeSize=" + storeSize +
", maximumTimestamp=" + maximumTimestamp +
'}';
}
}
public static class DataStreamShardStats implements Writeable {
private final ShardRouting shardRouting;
private final StoreStats storeStats;
private final long maxTimestamp;
public DataStreamShardStats(ShardRouting shardRouting, StoreStats storeStats, long maxTimestamp) {
this.shardRouting = shardRouting;
this.storeStats = storeStats;
this.maxTimestamp = maxTimestamp;
}
public DataStreamShardStats(StreamInput in) throws IOException {
this.shardRouting = new ShardRouting(in);
this.storeStats = new StoreStats(in);
this.maxTimestamp = in.readVLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
shardRouting.writeTo(out);
storeStats.writeTo(out);
out.writeVLong(maxTimestamp);
}
public ShardRouting getShardRouting() {
return shardRouting;
}
public StoreStats getStoreStats() {
return storeStats;
}
public long getMaxTimestamp() {
return maxTimestamp;
}
}
}

View File

@ -0,0 +1,102 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.core.action;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.support.master.MasterNodeRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.util.CollectionUtils;
import java.io.IOException;
import java.util.Arrays;
import java.util.Objects;
import static org.elasticsearch.action.ValidateActions.addValidationError;
public class DeleteDataStreamAction extends ActionType<AcknowledgedResponse> {
public static final DeleteDataStreamAction INSTANCE = new DeleteDataStreamAction();
public static final String NAME = "indices:admin/data_stream/delete";
private DeleteDataStreamAction() {
super(NAME, AcknowledgedResponse::new);
}
public static class Request extends MasterNodeRequest<Request> implements IndicesRequest.Replaceable {
private String[] names;
public Request(String[] names) {
this.names = Objects.requireNonNull(names);
}
public String[] getNames() {
return names;
}
@Override
public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = null;
if (CollectionUtils.isEmpty(names)) {
validationException = addValidationError("no data stream(s) specified", validationException);
}
return validationException;
}
public Request(StreamInput in) throws IOException {
super(in);
this.names = in.readStringArray();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeStringArray(names);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Request request = (Request) o;
return Arrays.equals(names, request.names);
}
@Override
public int hashCode() {
return Arrays.hashCode(names);
}
@Override
public String[] indices() {
return names;
}
@Override
public IndicesOptions indicesOptions() {
// this doesn't really matter since data stream name resolution isn't affected by IndicesOptions and
// a data stream's backing indices are retrieved from its metadata
return IndicesOptions.fromOptions(false, true, true, true, false, false, true, false);
}
@Override
public boolean includeDataStreams() {
return true;
}
@Override
public IndicesRequest indices(String... indices) {
this.names = indices;
return this;
}
}
}

View File

@ -1,64 +1,30 @@
/*
* 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.
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.action.admin.indices.datastream;
package org.elasticsearch.xpack.core.action;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.MasterNodeReadRequest;
import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.health.ClusterStateHealth;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.Index;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;
public class GetDataStreamAction extends ActionType<GetDataStreamAction.Response> {
@ -77,6 +43,10 @@ public class GetDataStreamAction extends ActionType<GetDataStreamAction.Response
this.names = names;
}
public String[] getNames() {
return names;
}
@Override
public ActionRequestValidationException validate() {
return null;
@ -262,62 +232,4 @@ public class GetDataStreamAction extends ActionType<GetDataStreamAction.Response
}
}
public static class TransportAction extends TransportMasterNodeReadAction<Request, Response> {
private static final Logger logger = LogManager.getLogger(TransportAction.class);
@Inject
public TransportAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
super(NAME, transportService, clusterService, threadPool, actionFilters, Request::new, indexNameExpressionResolver);
}
@Override
protected String executor() {
return ThreadPool.Names.SAME;
}
@Override
protected Response read(StreamInput in) throws IOException {
return new Response(in);
}
@Override
protected void masterOperation(Request request, ClusterState state,
ActionListener<Response> listener) throws Exception {
List<DataStream> dataStreams = getDataStreams(state, indexNameExpressionResolver, request);
List<Response.DataStreamInfo> dataStreamInfos = new ArrayList<>(dataStreams.size());
for (DataStream dataStream : dataStreams) {
String indexTemplate = MetadataIndexTemplateService.findV2Template(state.metadata(), dataStream.getName(), false);
String ilmPolicyName = null;
if (indexTemplate != null) {
Settings settings = MetadataIndexTemplateService.resolveSettings(state.metadata(), indexTemplate);
ilmPolicyName = settings.get("index.lifecycle.name");
} else {
logger.warn("couldn't find any matching template for data stream [{}]. has it been restored (and possibly renamed)" +
"from a snapshot?", dataStream.getName());
}
ClusterStateHealth streamHealth = new ClusterStateHealth(state,
dataStream.getIndices().stream().map(Index::getName).toArray(String[]::new));
dataStreamInfos.add(new Response.DataStreamInfo(dataStream, streamHealth.getStatus(), indexTemplate, ilmPolicyName));
}
listener.onResponse(new Response(dataStreamInfos));
}
static List<DataStream> getDataStreams(ClusterState clusterState, IndexNameExpressionResolver iner, Request request) {
List<String> results = iner.dataStreamNames(clusterState, request.indicesOptions(), request.names);
Map<String, DataStream> dataStreams = clusterState.metadata().dataStreams();
return results.stream()
.map(dataStreams::get)
.sorted(Comparator.comparing(DataStream::getName))
.collect(Collectors.toList());
}
@Override
protected ClusterBlockException checkBlock(Request request, ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
}
}
}

View File

@ -14,9 +14,9 @@ import org.elasticsearch.action.admin.indices.alias.get.GetAliasesAction;
import org.elasticsearch.action.admin.indices.close.CloseIndexAction;
import org.elasticsearch.action.admin.indices.create.AutoCreateAction;
import org.elasticsearch.action.admin.indices.create.CreateIndexAction;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction;
import org.elasticsearch.xpack.core.action.CreateDataStreamAction;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction;
import org.elasticsearch.xpack.core.action.GetDataStreamAction;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction;
import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsAction;
import org.elasticsearch.action.admin.indices.exists.types.TypesExistsAction;

View File

@ -9,9 +9,6 @@ import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction;
import org.elasticsearch.action.admin.indices.get.GetIndexRequest;
import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
@ -34,6 +31,7 @@ import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
@ -54,6 +52,9 @@ import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xpack.core.action.CreateDataStreamAction;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction;
import org.elasticsearch.xpack.core.action.GetDataStreamAction;
import org.elasticsearch.xpack.datastreams.DataStreamsPlugin;
import org.junit.After;
@ -70,6 +71,7 @@ import static org.elasticsearch.action.DocWriteRequest.OpType.CREATE;
import static org.elasticsearch.cluster.DataStreamTestHelper.generateMapping;
import static org.elasticsearch.cluster.metadata.MetadataIndexTemplateService.DEFAULT_TIMESTAMP_FIELD;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.arrayWithSize;
import static org.hamcrest.Matchers.containsString;
@ -82,6 +84,7 @@ import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.startsWith;
@ESIntegTestCase.ClusterScope(transportClientRatio = 0)
public class DataStreamIT extends ESIntegTestCase {
@Override
@ -90,7 +93,13 @@ public class DataStreamIT extends ESIntegTestCase {
}
@After
public void deleteAllComposableTemplates() {
public void cleanup() {
AcknowledgedResponse response = client().execute(
DeleteDataStreamAction.INSTANCE,
new DeleteDataStreamAction.Request(new String[] { "*" })
).actionGet();
assertAcked(response);
DeleteDataStreamAction.Request deleteDSRequest = new DeleteDataStreamAction.Request(new String[] { "*" });
client().execute(DeleteDataStreamAction.INSTANCE, deleteDSRequest).actionGet();
DeleteComposableIndexTemplateAction.Request deleteTemplateRequest = new DeleteComposableIndexTemplateAction.Request("*");
@ -100,14 +109,15 @@ public class DataStreamIT extends ESIntegTestCase {
public void testBasicScenario() throws Exception {
putComposableIndexTemplate("id1", List.of("metrics-foo*"));
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request("metrics-foo");
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
putComposableIndexTemplate("id2", List.of("metrics-bar*"));
createDataStreamRequest = new CreateDataStreamAction.Request("metrics-bar");
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
GetDataStreamAction.Request getDataStreamRequest = new GetDataStreamAction.Request(new String[] { "*" });
GetDataStreamAction.Response getDataStreamResponse = client().admin().indices().getDataStreams(getDataStreamRequest).actionGet();
GetDataStreamAction.Response getDataStreamResponse = client().execute(GetDataStreamAction.INSTANCE, getDataStreamRequest)
.actionGet();
getDataStreamResponse.getDataStreams().sort(Comparator.comparing(dataStreamInfo -> dataStreamInfo.getDataStream().getName()));
assertThat(getDataStreamResponse.getDataStreams().size(), equalTo(2));
DataStream firstDataStream = getDataStreamResponse.getDataStreams().get(0).getDataStream();
@ -174,8 +184,8 @@ public class DataStreamIT extends ESIntegTestCase {
verifyDocs("metrics-foo", numDocsFoo + numDocsFoo2, 1, 2);
DeleteDataStreamAction.Request deleteDataStreamRequest = new DeleteDataStreamAction.Request(new String[] { "metrics-*" });
client().admin().indices().deleteDataStream(deleteDataStreamRequest).actionGet();
getDataStreamResponse = client().admin().indices().getDataStreams(getDataStreamRequest).actionGet();
client().execute(DeleteDataStreamAction.INSTANCE, deleteDataStreamRequest).actionGet();
getDataStreamResponse = client().execute(GetDataStreamAction.INSTANCE, getDataStreamRequest).actionGet();
assertThat(getDataStreamResponse.getDataStreams().size(), equalTo(0));
expectThrows(
@ -212,7 +222,7 @@ public class DataStreamIT extends ESIntegTestCase {
putComposableIndexTemplate("id", List.of("metrics-foobar*"));
String dataStreamName = "metrics-foobar";
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
{
BulkRequest bulkRequest = new BulkRequest().add(
@ -305,7 +315,8 @@ public class DataStreamIT extends ESIntegTestCase {
String backingIndex = DataStream.getDefaultBackingIndexName(dataStreamName, 1);
GetDataStreamAction.Request getDataStreamRequest = new GetDataStreamAction.Request(new String[] { "*" });
GetDataStreamAction.Response getDataStreamResponse = client().admin().indices().getDataStreams(getDataStreamRequest).actionGet();
GetDataStreamAction.Response getDataStreamResponse = client().execute(GetDataStreamAction.INSTANCE, getDataStreamRequest)
.actionGet();
assertThat(getDataStreamResponse.getDataStreams().size(), equalTo(1));
assertThat(getDataStreamResponse.getDataStreams().get(0).getDataStream().getName(), equalTo(dataStreamName));
assertThat(getDataStreamResponse.getDataStreams().get(0).getDataStream().getTimeStampField().getName(), equalTo("@timestamp"));
@ -338,8 +349,8 @@ public class DataStreamIT extends ESIntegTestCase {
verifyDocs(dataStreamName, numDocs + numDocs2, 1, 2);
DeleteDataStreamAction.Request deleteDataStreamRequest = new DeleteDataStreamAction.Request(new String[] { dataStreamName });
client().admin().indices().deleteDataStream(deleteDataStreamRequest).actionGet();
getDataStreamResponse = client().admin().indices().getDataStreams(getDataStreamRequest).actionGet();
client().execute(DeleteDataStreamAction.INSTANCE, deleteDataStreamRequest).actionGet();
getDataStreamResponse = client().execute(GetDataStreamAction.INSTANCE, getDataStreamRequest).actionGet();
assertThat(getDataStreamResponse.getDataStreams().size(), equalTo(0));
expectThrows(
@ -394,7 +405,7 @@ public class DataStreamIT extends ESIntegTestCase {
putComposableIndexTemplate("id", List.of("logs-*"));
String dataStreamName = "logs-foobar";
CreateDataStreamAction.Request request = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(request).actionGet();
client().execute(CreateDataStreamAction.INSTANCE, request).actionGet();
verifyResolvability(
dataStreamName,
@ -449,7 +460,7 @@ public class DataStreamIT extends ESIntegTestCase {
verifyResolvability(dataStreamName, client().admin().indices().prepareShardStores(dataStreamName), false);
request = new CreateDataStreamAction.Request("logs-barbaz");
client().admin().indices().createDataStream(request).actionGet();
client().execute(CreateDataStreamAction.INSTANCE, request).actionGet();
verifyResolvability(
"logs-barbaz",
client().prepareIndex("logs-barbaz", "_doc")
@ -509,9 +520,9 @@ public class DataStreamIT extends ESIntegTestCase {
putComposableIndexTemplate("id", List.of("metrics-foobar*"));
String dataStreamName = "metrics-foobar-baz";
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName + "-eggplant");
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
DeleteComposableIndexTemplateAction.Request req = new DeleteComposableIndexTemplateAction.Request("id");
Exception e = expectThrows(Exception.class, () -> client().execute(DeleteComposableIndexTemplateAction.INSTANCE, req).get());
@ -542,7 +553,7 @@ public class DataStreamIT extends ESIntegTestCase {
putComposableIndexTemplate("id1", List.of("metrics-foo*"));
String dataStreamName = "metrics-foo";
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
IndicesAliasesRequest.AliasActions addAction = new IndicesAliasesRequest.AliasActions(IndicesAliasesRequest.AliasActions.Type.ADD)
.index(dataStreamName)
@ -557,7 +568,7 @@ public class DataStreamIT extends ESIntegTestCase {
putComposableIndexTemplate("id1", List.of("metrics-foo*"));
String dataStreamName = "metrics-foo";
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
String backingIndex = DataStream.getDefaultBackingIndexName(dataStreamName, 1);
IndicesAliasesRequest.AliasActions addAction = new IndicesAliasesRequest.AliasActions(IndicesAliasesRequest.AliasActions.Type.ADD)
@ -594,9 +605,10 @@ public class DataStreamIT extends ESIntegTestCase {
putComposableIndexTemplate("id1", mapping, List.of("logs-foo*"), null);
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request("logs-foobar");
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
GetDataStreamAction.Request getDataStreamRequest = new GetDataStreamAction.Request(new String[] { "logs-foobar" });
GetDataStreamAction.Response getDataStreamResponse = client().admin().indices().getDataStreams(getDataStreamRequest).actionGet();
GetDataStreamAction.Response getDataStreamResponse = client().execute(GetDataStreamAction.INSTANCE, getDataStreamRequest)
.actionGet();
assertThat(getDataStreamResponse.getDataStreams().size(), equalTo(1));
assertThat(getDataStreamResponse.getDataStreams().get(0).getDataStream().getName(), equalTo("logs-foobar"));
assertThat(getDataStreamResponse.getDataStreams().get(0).getDataStream().getTimeStampField().getName(), equalTo("@timestamp"));
@ -607,7 +619,7 @@ public class DataStreamIT extends ESIntegTestCase {
public void testUpdateMappingViaDataStream() throws Exception {
putComposableIndexTemplate("id1", List.of("logs-*"));
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request("logs-foobar");
client().admin().indices().createDataStream(createDataStreamRequest).actionGet();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).actionGet();
String backingIndex1 = DataStream.getDefaultBackingIndexName("logs-foobar", 1);
String backingIndex2 = DataStream.getDefaultBackingIndexName("logs-foobar", 2);
@ -649,7 +661,7 @@ public class DataStreamIT extends ESIntegTestCase {
public void testUpdateIndexSettingsViaDataStream() throws Exception {
putComposableIndexTemplate("id1", List.of("logs-*"));
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request("logs-foobar");
client().admin().indices().createDataStream(createDataStreamRequest).actionGet();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).actionGet();
String backingIndex1 = DataStream.getDefaultBackingIndexName("logs-foobar", 1);
String backingIndex2 = DataStream.getDefaultBackingIndexName("logs-foobar", 2);
@ -750,11 +762,11 @@ public class DataStreamIT extends ESIntegTestCase {
public void testSearchAllResolvesDataStreams() throws Exception {
putComposableIndexTemplate("id1", List.of("metrics-foo*"));
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request("metrics-foo");
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
putComposableIndexTemplate("id2", List.of("metrics-bar*"));
createDataStreamRequest = new CreateDataStreamAction.Request("metrics-bar");
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
int numDocsBar = randomIntBetween(2, 16);
indexDocs("metrics-bar", numDocsBar);
@ -780,10 +792,10 @@ public class DataStreamIT extends ESIntegTestCase {
int numDocsFoo = randomIntBetween(2, 16);
indexDocs("metrics-foo", numDocsFoo);
GetDataStreamAction.Response response = client().admin()
.indices()
.getDataStreams(new GetDataStreamAction.Request(new String[] { "metrics-foo" }))
.actionGet();
GetDataStreamAction.Response response = client().execute(
GetDataStreamAction.INSTANCE,
new GetDataStreamAction.Request(new String[] { "metrics-foo" })
).actionGet();
assertThat(response.getDataStreams().size(), is(1));
GetDataStreamAction.Response.DataStreamInfo metricsFooDataStream = response.getDataStreams().get(0);
assertThat(metricsFooDataStream.getDataStream().getName(), is("metrics-foo"));
@ -808,7 +820,7 @@ public class DataStreamIT extends ESIntegTestCase {
putComposableIndexTemplate("id", List.of("logs-foobar*"));
String dataStreamName = "logs-foobar";
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
IndexRequest indexRequest = new IndexRequest(dataStreamName).opType("create").source("{}", XContentType.JSON);
Exception e = expectThrows(MapperParsingException.class, () -> client().index(indexRequest).actionGet());
@ -819,7 +831,7 @@ public class DataStreamIT extends ESIntegTestCase {
putComposableIndexTemplate("id", List.of("logs-foobar*"));
String dataStreamName = "logs-foobar";
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(createDataStreamRequest).get();
client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get();
IndexRequest indexRequest = new IndexRequest(dataStreamName).opType("create")
.source("{\"@timestamp\": [\"2020-12-12\",\"2022-12-12\"]}", XContentType.JSON);
@ -869,7 +881,8 @@ public class DataStreamIT extends ESIntegTestCase {
assertThat("bulk failures: " + Strings.toString(bulkResponse), bulkResponse.hasFailures(), is(false));
GetDataStreamAction.Request getDataStreamRequest = new GetDataStreamAction.Request(new String[] { "*" });
GetDataStreamAction.Response getDataStreamsResponse = client().admin().indices().getDataStreams(getDataStreamRequest).actionGet();
GetDataStreamAction.Response getDataStreamsResponse = client().execute(GetDataStreamAction.INSTANCE, getDataStreamRequest)
.actionGet();
assertThat(getDataStreamsResponse.getDataStreams(), hasSize(4));
getDataStreamsResponse.getDataStreams().sort(Comparator.comparing(dataStreamInfo -> dataStreamInfo.getDataStream().getName()));
assertThat(getDataStreamsResponse.getDataStreams().get(0).getDataStream().getName(), equalTo("logs-foobar"));
@ -905,7 +918,8 @@ public class DataStreamIT extends ESIntegTestCase {
assertThat("bulk failures: " + Strings.toString(bulkResponse), bulkResponse.hasFailures(), is(false));
GetDataStreamAction.Request getDataStreamRequest = new GetDataStreamAction.Request(new String[] { "*" });
GetDataStreamAction.Response getDataStreamsResponse = client().admin().indices().getDataStreams(getDataStreamRequest).actionGet();
GetDataStreamAction.Response getDataStreamsResponse = client().execute(GetDataStreamAction.INSTANCE, getDataStreamRequest)
.actionGet();
assertThat(getDataStreamsResponse.getDataStreams(), hasSize(0));
GetIndexResponse getIndexResponse = client().admin().indices().getIndex(new GetIndexRequest().indices("logs-foobar")).actionGet();

View File

@ -11,9 +11,10 @@ import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse;
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xpack.core.action.CreateDataStreamAction;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction;
import org.elasticsearch.xpack.core.action.GetDataStreamAction;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
@ -30,6 +31,7 @@ import org.elasticsearch.snapshots.SnapshotState;
import org.elasticsearch.snapshots.mockstore.MockRepository;
import org.elasticsearch.xpack.datastreams.DataStreamsPlugin;
import org.hamcrest.Matchers;
import org.junit.After;
import org.junit.Before;
import java.nio.file.Path;
@ -44,6 +46,7 @@ import static org.hamcrest.Matchers.nullValue;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
@ESIntegTestCase.ClusterScope(transportClientRatio = 0)
public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
private static final String DS_BACKING_INDEX_NAME = DataStream.getDefaultBackingIndexName("ds", 1);
@ -69,11 +72,11 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
DataStreamIT.putComposableIndexTemplate("t1", List.of("ds", "other-ds"));
CreateDataStreamAction.Request request = new CreateDataStreamAction.Request("ds");
AcknowledgedResponse response = client.admin().indices().createDataStream(request).get();
AcknowledgedResponse response = client.execute(CreateDataStreamAction.INSTANCE, request).get();
assertTrue(response.isAcknowledged());
request = new CreateDataStreamAction.Request("other-ds");
response = client.admin().indices().createDataStream(request).get();
response = client.execute(CreateDataStreamAction.INSTANCE, request).get();
assertTrue(response.isAcknowledged());
IndexResponse indexResponse = client.prepareIndex("ds", "_doc")
@ -84,6 +87,15 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
id = indexResponse.getId();
}
@After
public void cleanup() {
AcknowledgedResponse response = client().execute(
DeleteDataStreamAction.INSTANCE,
new DeleteDataStreamAction.Request(new String[] { "*" })
).actionGet();
assertAcked(response);
}
public void testSnapshotAndRestore() throws Exception {
CreateSnapshotResponse createSnapshotResponse = client.admin()
.cluster()
@ -102,7 +114,9 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
assertEquals(Collections.singletonList(DS_BACKING_INDEX_NAME), snap.get(0).indices());
assertTrue(
client.admin().indices().deleteDataStream(new DeleteDataStreamAction.Request(new String[] { "ds" })).get().isAcknowledged()
client.execute(DeleteDataStreamAction.INSTANCE, new DeleteDataStreamAction.Request(new String[] { "ds" }))
.get()
.isAcknowledged()
);
RestoreSnapshotResponse restoreSnapshotResponse = client.admin()
@ -119,10 +133,10 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
assertEquals(1, hits.length);
assertEquals(DOCUMENT_SOURCE, hits[0].getSourceAsMap());
GetDataStreamAction.Response ds = client.admin()
.indices()
.getDataStreams(new GetDataStreamAction.Request(new String[] { "ds" }))
.get();
GetDataStreamAction.Response ds = client.execute(
GetDataStreamAction.INSTANCE,
new GetDataStreamAction.Request(new String[] { "ds" })
).get();
assertEquals(1, ds.getDataStreams().size());
assertEquals(1, ds.getDataStreams().get(0).getDataStream().getIndices().size());
assertEquals(DS_BACKING_INDEX_NAME, ds.getDataStreams().get(0).getDataStream().getIndices().get(0).getName());
@ -145,7 +159,7 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
assertEquals(1, snap.size());
assertEquals(Collections.singletonList(DS_BACKING_INDEX_NAME), snap.get(0).indices());
assertAcked(client.admin().indices().deleteDataStream(new DeleteDataStreamAction.Request(new String[] { "*" })).get());
assertAcked(client.execute(DeleteDataStreamAction.INSTANCE, new DeleteDataStreamAction.Request(new String[] { "*" })).get());
assertAcked(client.admin().indices().prepareDelete("*").setIndicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN_CLOSED_HIDDEN));
RestoreSnapshotResponse restoreSnapshotResponse = client.admin()
@ -162,15 +176,15 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
assertEquals(1, hits.length);
assertEquals(DOCUMENT_SOURCE, hits[0].getSourceAsMap());
GetDataStreamAction.Response ds = client.admin()
.indices()
.getDataStreams(new GetDataStreamAction.Request(new String[] { "ds" }))
.get();
GetDataStreamAction.Response ds = client.execute(
GetDataStreamAction.INSTANCE,
new GetDataStreamAction.Request(new String[] { "ds" })
).get();
assertEquals(1, ds.getDataStreams().size());
assertEquals(1, ds.getDataStreams().get(0).getDataStream().getIndices().size());
assertEquals(DS_BACKING_INDEX_NAME, ds.getDataStreams().get(0).getDataStream().getIndices().get(0).getName());
assertAcked(client().admin().indices().deleteDataStream(new DeleteDataStreamAction.Request(new String[] { "ds" })).get());
assertAcked(client().execute(DeleteDataStreamAction.INSTANCE, new DeleteDataStreamAction.Request(new String[] { "ds" })).get());
}
public void testRename() throws Exception {
@ -199,10 +213,10 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
.setRenameReplacement("ds2")
.get();
GetDataStreamAction.Response ds = client.admin()
.indices()
.getDataStreams(new GetDataStreamAction.Request(new String[] { "ds2" }))
.get();
GetDataStreamAction.Response ds = client.execute(
GetDataStreamAction.INSTANCE,
new GetDataStreamAction.Request(new String[] { "ds2" })
).get();
assertEquals(1, ds.getDataStreams().size());
assertEquals(1, ds.getDataStreams().get(0).getDataStream().getIndices().size());
assertEquals(DS2_BACKING_INDEX_NAME, ds.getDataStreams().get(0).getDataStream().getIndices().get(0).getName());
@ -228,7 +242,7 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
);
// delete data stream
client.admin().indices().deleteDataStream(new DeleteDataStreamAction.Request(new String[] { "ds" })).actionGet();
client.execute(DeleteDataStreamAction.INSTANCE, new DeleteDataStreamAction.Request(new String[] { "ds" })).actionGet();
// restore data stream attempting to rename the backing index
RestoreSnapshotResponse restoreSnapshotResponse = client.admin()
@ -243,7 +257,7 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
assertThat(restoreSnapshotResponse.status(), is(RestStatus.OK));
GetDataStreamAction.Request getDSRequest = new GetDataStreamAction.Request(new String[] { "ds" });
GetDataStreamAction.Response response = client.admin().indices().getDataStreams(getDSRequest).actionGet();
GetDataStreamAction.Response response = client.execute(GetDataStreamAction.INSTANCE, getDSRequest).actionGet();
assertThat(response.getDataStreams().get(0).getDataStream().getIndices().get(0).getName(), is(DS_BACKING_INDEX_NAME));
}
@ -278,7 +292,7 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
// assert "ds" was restored as "test-ds" and the backing index has a valid name
GetDataStreamAction.Request getRenamedDS = new GetDataStreamAction.Request(new String[] { "test-ds" });
GetDataStreamAction.Response response = client.admin().indices().getDataStreams(getRenamedDS).actionGet();
GetDataStreamAction.Response response = client.execute(GetDataStreamAction.INSTANCE, getRenamedDS).actionGet();
assertThat(
response.getDataStreams().get(0).getDataStream().getIndices().get(0).getName(),
is(DataStream.getDefaultBackingIndexName("test-ds", 1L))
@ -286,7 +300,7 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
// data stream "ds" should still exist in the system
GetDataStreamAction.Request getDSRequest = new GetDataStreamAction.Request(new String[] { "ds" });
response = client.admin().indices().getDataStreams(getDSRequest).actionGet();
response = client.execute(GetDataStreamAction.INSTANCE, getDSRequest).actionGet();
assertThat(response.getDataStreams().get(0).getDataStream().getIndices().get(0).getName(), is(DS_BACKING_INDEX_NAME));
}
@ -313,10 +327,10 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
assertEquals(RestStatus.OK, restoreSnapshotResponse.status());
GetDataStreamAction.Response ds = client.admin()
.indices()
.getDataStreams(new GetDataStreamAction.Request(new String[] { "ds2" }))
.get();
GetDataStreamAction.Response ds = client.execute(
GetDataStreamAction.INSTANCE,
new GetDataStreamAction.Request(new String[] { "ds2" })
).get();
assertEquals(1, ds.getDataStreams().size());
assertEquals(1, ds.getDataStreams().get(0).getDataStream().getIndices().size());
assertEquals(DS2_BACKING_INDEX_NAME, ds.getDataStreams().get(0).getDataStream().getIndices().get(0).getName());
@ -327,7 +341,7 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
);
}
public void testDataStreamNotStoredWhenIndexRequested() throws Exception {
public void testDataStreamNotStoredWhenIndexRequested() {
CreateSnapshotResponse createSnapshotResponse = client.admin()
.cluster()
.prepareCreateSnapshot(REPO, "snap2")
@ -357,7 +371,9 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
assertEquals(RestStatus.OK, status);
assertTrue(
client.admin().indices().deleteDataStream(new DeleteDataStreamAction.Request(new String[] { "ds" })).get().isAcknowledged()
client.execute(DeleteDataStreamAction.INSTANCE, new DeleteDataStreamAction.Request(new String[] { "ds" }))
.get()
.isAcknowledged()
);
RestoreSnapshotResponse restoreSnapshotResponse = client.admin()
@ -370,7 +386,7 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
assertEquals(RestStatus.OK, restoreSnapshotResponse.status());
GetDataStreamAction.Request getRequest = new GetDataStreamAction.Request(new String[] { "ds" });
expectThrows(ResourceNotFoundException.class, () -> client.admin().indices().getDataStreams(getRequest).actionGet());
expectThrows(ResourceNotFoundException.class, () -> client.execute(GetDataStreamAction.INSTANCE, getRequest).actionGet());
}
public void testDataStreamNotIncludedInLimitedSnapshot() throws ExecutionException, InterruptedException {
@ -385,7 +401,9 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
assertThat(createSnapshotResponse.getSnapshotInfo().state(), Matchers.is(SnapshotState.SUCCESS));
assertThat(
client().admin().indices().deleteDataStream(new DeleteDataStreamAction.Request(new String[] { "*" })).get().isAcknowledged(),
client().execute(DeleteDataStreamAction.INSTANCE, new DeleteDataStreamAction.Request(new String[] { "*" }))
.get()
.isAcknowledged(),
is(true)
);

View File

@ -10,7 +10,9 @@ import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.common.collect.List;
@ -24,16 +26,19 @@ import org.elasticsearch.snapshots.SnapshotInfo;
import org.elasticsearch.snapshots.SnapshotState;
import org.elasticsearch.snapshots.mockstore.MockRepository;
import org.elasticsearch.xpack.datastreams.DataStreamsPlugin;
import org.junit.After;
import java.util.Collection;
import java.util.Collections;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
// The tests in here do a lot of state updates and other writes to disk and are slowed down too much by WindowsFS
@LuceneTestCase.SuppressFileSystems(value = "WindowsFS")
@ESIntegTestCase.ClusterScope(transportClientRatio = 0)
public class ShardClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCase {
@Override
@ -41,6 +46,15 @@ public class ShardClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCase
return List.of(MockRepository.Plugin.class, DataStreamsPlugin.class);
}
@After
public void cleanup() {
AcknowledgedResponse response = client().execute(
DeleteDataStreamAction.INSTANCE,
new DeleteDataStreamAction.Request(new String[] { "*" })
).actionGet();
assertAcked(response);
}
public void testDeleteDataStreamDuringSnapshot() throws Exception {
Client client = client();
@ -83,7 +97,7 @@ public class ShardClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCase
// non-partial snapshots do not allow delete operations on data streams where snapshot has not been completed
try {
logger.info("--> delete index while non-partial snapshot is running");
client.admin().indices().deleteDataStream(new DeleteDataStreamAction.Request(new String[] { dataStream })).actionGet();
client.execute(DeleteDataStreamAction.INSTANCE, new DeleteDataStreamAction.Request(new String[] { dataStream })).actionGet();
fail("Expected deleting index to fail during snapshot");
} catch (SnapshotInProgressException e) {
assertThat(e.getMessage(), containsString("Cannot delete data streams that are being snapshotted: [" + dataStream));

View File

@ -3,23 +3,45 @@
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.xpack.core.action.CreateDataStreamAction;
import org.elasticsearch.xpack.core.action.DataStreamsStatsAction;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction;
import org.elasticsearch.xpack.core.action.GetDataStreamAction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.IndexScopedSettings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.plugins.ActionPlugin;
import org.elasticsearch.plugins.MapperPlugin;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestHandler;
import org.elasticsearch.xpack.datastreams.action.DataStreamsStatsTransportAction;
import org.elasticsearch.xpack.datastreams.rest.RestCreateDataStreamAction;
import org.elasticsearch.xpack.datastreams.rest.RestDataStreamsStatsAction;
import org.elasticsearch.xpack.datastreams.rest.RestDeleteDataStreamAction;
import org.elasticsearch.xpack.datastreams.rest.RestGetDataStreamsAction;
import org.elasticsearch.xpack.core.XPackPlugin;
import org.elasticsearch.xpack.datastreams.action.CreateDataStreamTransportAction;
import org.elasticsearch.xpack.datastreams.action.DeleteDataStreamTransportAction;
import org.elasticsearch.xpack.datastreams.action.GetDataStreamsTransportAction;
import org.elasticsearch.xpack.datastreams.mapper.DataStreamTimestampFieldMapper;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
public class DataStreamsPlugin extends Plugin implements ActionPlugin, MapperPlugin {
@ -34,6 +56,33 @@ public class DataStreamsPlugin extends Plugin implements ActionPlugin, MapperPlu
return Collections.singletonMap(DataStreamTimestampFieldMapper.NAME, new DataStreamTimestampFieldMapper.TypeParser());
}
@Override
public List<ActionHandler<? extends ActionRequest, ? extends ActionResponse>> getActions() {
return Arrays.asList(
new ActionHandler<>(CreateDataStreamAction.INSTANCE, CreateDataStreamTransportAction.class),
new ActionHandler<>(DeleteDataStreamAction.INSTANCE, DeleteDataStreamTransportAction.class),
new ActionHandler<>(GetDataStreamAction.INSTANCE, GetDataStreamsTransportAction.class),
new ActionHandler<>(DataStreamsStatsAction.INSTANCE, DataStreamsStatsTransportAction.class)
);
}
@Override
public List<RestHandler> getRestHandlers(
Settings settings,
RestController restController,
ClusterSettings clusterSettings,
IndexScopedSettings indexScopedSettings,
SettingsFilter settingsFilter,
IndexNameExpressionResolver indexNameExpressionResolver,
Supplier<DiscoveryNodes> nodesInCluster
) {
RestHandler createDsAction = new RestCreateDataStreamAction();
RestHandler deleteDsAction = new RestDeleteDataStreamAction();
RestHandler getDsAction = new RestGetDataStreamsAction();
RestHandler dsStatsAction = new RestDataStreamsStatsAction();
return Arrays.asList(createDsAction, deleteDsAction, getDsAction, dsStatsAction);
}
public Collection<Module> createGuiceModules() {
List<Module> modules = new ArrayList<>();

View File

@ -0,0 +1,80 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams.action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetadataCreateDataStreamService;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.core.action.CreateDataStreamAction;
import java.io.IOException;
public class CreateDataStreamTransportAction extends TransportMasterNodeAction<CreateDataStreamAction.Request, AcknowledgedResponse> {
private final MetadataCreateDataStreamService metadataCreateDataStreamService;
@Inject
public CreateDataStreamTransportAction(
TransportService transportService,
ClusterService clusterService,
ThreadPool threadPool,
ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver,
MetadataCreateDataStreamService metadataCreateDataStreamService
) {
super(
CreateDataStreamAction.NAME,
transportService,
clusterService,
threadPool,
actionFilters,
CreateDataStreamAction.Request::new,
indexNameExpressionResolver
);
this.metadataCreateDataStreamService = metadataCreateDataStreamService;
}
@Override
protected String executor() {
return ThreadPool.Names.SAME;
}
@Override
protected AcknowledgedResponse read(StreamInput in) throws IOException {
return new AcknowledgedResponse(in);
}
@Override
protected void masterOperation(
CreateDataStreamAction.Request request,
ClusterState state,
ActionListener<AcknowledgedResponse> listener
) throws Exception {
MetadataCreateDataStreamService.CreateDataStreamClusterStateUpdateRequest updateRequest =
new MetadataCreateDataStreamService.CreateDataStreamClusterStateUpdateRequest(
request.getName(),
request.masterNodeTimeout(),
request.timeout()
);
metadataCreateDataStreamService.createDataStream(updateRequest, listener);
}
@Override
protected ClusterBlockException checkBlock(CreateDataStreamAction.Request request, ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
}
}

View File

@ -0,0 +1,217 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams.action;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.PointValues;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexAbstractionResolver;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.index.store.StoreStats;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.core.action.DataStreamsStatsAction;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedMap;
import java.util.stream.Stream;
public class DataStreamsStatsTransportAction extends TransportBroadcastByNodeAction<
DataStreamsStatsAction.Request,
DataStreamsStatsAction.Response,
DataStreamsStatsAction.DataStreamShardStats> {
private final ClusterService clusterService;
private final IndicesService indicesService;
private final IndexAbstractionResolver indexAbstractionResolver;
@Inject
public DataStreamsStatsTransportAction(
ClusterService clusterService,
TransportService transportService,
IndicesService indicesService,
ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver
) {
super(
DataStreamsStatsAction.NAME,
clusterService,
transportService,
actionFilters,
indexNameExpressionResolver,
DataStreamsStatsAction.Request::new,
ThreadPool.Names.MANAGEMENT
);
this.clusterService = clusterService;
this.indicesService = indicesService;
this.indexAbstractionResolver = new IndexAbstractionResolver(indexNameExpressionResolver);
}
@Override
protected DataStreamsStatsAction.Request readRequestFrom(StreamInput in) throws IOException {
return new DataStreamsStatsAction.Request(in);
}
@Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, DataStreamsStatsAction.Request request) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ);
}
@Override
protected ClusterBlockException checkRequestBlock(
ClusterState state,
DataStreamsStatsAction.Request request,
String[] concreteIndices
) {
return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, concreteIndices);
}
@Override
protected ShardsIterator shards(ClusterState clusterState, DataStreamsStatsAction.Request request, String[] concreteIndices) {
String[] requestIndices = request.indices();
if (requestIndices == null || requestIndices.length == 0) {
requestIndices = new String[] { "*" };
}
List<String> abstractionNames = indexAbstractionResolver.resolveIndexAbstractions(
requestIndices,
request.indicesOptions(),
clusterState.getMetadata(),
true
); // Always include data streams for data streams stats api
SortedMap<String, IndexAbstraction> indicesLookup = clusterState.getMetadata().getIndicesLookup();
String[] concreteDatastreamIndices = abstractionNames.stream().flatMap(abstractionName -> {
IndexAbstraction indexAbstraction = indicesLookup.get(abstractionName);
assert indexAbstraction != null;
if (indexAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM) {
IndexAbstraction.DataStream dataStream = (IndexAbstraction.DataStream) indexAbstraction;
List<IndexMetadata> indices = dataStream.getIndices();
return indices.stream().map(idx -> idx.getIndex().getName());
} else {
return Stream.empty();
}
}).toArray(String[]::new);
return clusterState.getRoutingTable().allShards(concreteDatastreamIndices);
}
@Override
protected DataStreamsStatsAction.DataStreamShardStats shardOperation(DataStreamsStatsAction.Request request, ShardRouting shardRouting)
throws IOException {
IndexService indexService = indicesService.indexServiceSafe(shardRouting.shardId().getIndex());
IndexShard indexShard = indexService.getShard(shardRouting.shardId().id());
// if we don't have the routing entry yet, we need it stats wise, we treat it as if the shard is not ready yet
if (indexShard.routingEntry() == null) {
throw new ShardNotFoundException(indexShard.shardId());
}
StoreStats storeStats = indexShard.storeStats();
IndexAbstraction indexAbstraction = clusterService.state().getMetadata().getIndicesLookup().get(shardRouting.getIndexName());
assert indexAbstraction != null;
IndexAbstraction.DataStream dataStream = indexAbstraction.getParentDataStream();
assert dataStream != null;
long maxTimestamp = 0L;
try (Engine.Searcher searcher = indexShard.acquireSearcher("data_stream_stats")) {
IndexReader indexReader = searcher.getIndexReader();
String fieldName = dataStream.getDataStream().getTimeStampField().getName();
byte[] maxPackedValue = PointValues.getMaxPackedValue(indexReader, fieldName);
if (maxPackedValue != null) {
maxTimestamp = LongPoint.decodeDimension(maxPackedValue, 0);
}
}
return new DataStreamsStatsAction.DataStreamShardStats(indexShard.routingEntry(), storeStats, maxTimestamp);
}
@Override
protected DataStreamsStatsAction.DataStreamShardStats readShardResult(StreamInput in) throws IOException {
return new DataStreamsStatsAction.DataStreamShardStats(in);
}
@Override
protected DataStreamsStatsAction.Response newResponse(
DataStreamsStatsAction.Request request,
int totalShards,
int successfulShards,
int failedShards,
List<DataStreamsStatsAction.DataStreamShardStats> dataStreamShardStats,
List<DefaultShardOperationFailedException> shardFailures,
ClusterState clusterState
) {
Map<String, AggregatedStats> aggregatedDataStreamsStats = new HashMap<>();
Set<String> allBackingIndices = new HashSet<>();
long totalStoreSizeBytes = 0L;
SortedMap<String, IndexAbstraction> indicesLookup = clusterState.getMetadata().getIndicesLookup();
for (DataStreamsStatsAction.DataStreamShardStats shardStat : dataStreamShardStats) {
String indexName = shardStat.getShardRouting().getIndexName();
IndexAbstraction indexAbstraction = indicesLookup.get(indexName);
IndexAbstraction.DataStream dataStream = indexAbstraction.getParentDataStream();
assert dataStream != null;
// Aggregate global stats
totalStoreSizeBytes += shardStat.getStoreStats().sizeInBytes();
allBackingIndices.add(indexName);
// Aggregate data stream stats
AggregatedStats stats = aggregatedDataStreamsStats.computeIfAbsent(dataStream.getName(), s -> new AggregatedStats());
stats.storageBytes += shardStat.getStoreStats().sizeInBytes();
stats.maxTimestamp = Math.max(stats.maxTimestamp, shardStat.getMaxTimestamp());
stats.backingIndices.add(indexName);
}
DataStreamsStatsAction.DataStreamStats[] dataStreamStats = aggregatedDataStreamsStats.entrySet()
.stream()
.map(
entry -> new DataStreamsStatsAction.DataStreamStats(
entry.getKey(),
entry.getValue().backingIndices.size(),
new ByteSizeValue(entry.getValue().storageBytes),
entry.getValue().maxTimestamp
)
)
.toArray(DataStreamsStatsAction.DataStreamStats[]::new);
return new DataStreamsStatsAction.Response(
totalShards,
successfulShards,
failedShards,
shardFailures,
aggregatedDataStreamsStats.size(),
allBackingIndices.size(),
new ByteSizeValue(totalStoreSizeBytes),
dataStreamStats
);
}
private static class AggregatedStats {
Set<String> backingIndices = new HashSet<>();
long storageBytes = 0L;
long maxTimestamp = 0L;
}
}

View File

@ -0,0 +1,160 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams.action;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
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.DataStream;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.MetadataDeleteIndexService;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.snapshots.SnapshotInProgressException;
import org.elasticsearch.snapshots.SnapshotsService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction;
import java.io.IOException;
import java.util.HashSet;
import java.util.Set;
public class DeleteDataStreamTransportAction extends TransportMasterNodeAction<DeleteDataStreamAction.Request, AcknowledgedResponse> {
private static final Logger LOGGER = LogManager.getLogger(DeleteDataStreamTransportAction.class);
private final MetadataDeleteIndexService deleteIndexService;
@Inject
public DeleteDataStreamTransportAction(
TransportService transportService,
ClusterService clusterService,
ThreadPool threadPool,
ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver,
MetadataDeleteIndexService deleteIndexService
) {
super(
DeleteDataStreamAction.NAME,
transportService,
clusterService,
threadPool,
actionFilters,
DeleteDataStreamAction.Request::new,
indexNameExpressionResolver
);
this.deleteIndexService = deleteIndexService;
}
@Override
protected String executor() {
return ThreadPool.Names.SAME;
}
@Override
protected AcknowledgedResponse read(StreamInput in) throws IOException {
return new AcknowledgedResponse(in);
}
@Override
protected void masterOperation(
DeleteDataStreamAction.Request request,
ClusterState state,
ActionListener<AcknowledgedResponse> listener
) throws Exception {
clusterService.submitStateUpdateTask(
"remove-data-stream [" + Strings.arrayToCommaDelimitedString(request.getNames()) + "]",
new ClusterStateUpdateTask(Priority.HIGH) {
@Override
public TimeValue timeout() {
return request.masterNodeTimeout();
}
@Override
public void onFailure(String source, Exception e) {
listener.onFailure(e);
}
@Override
public ClusterState execute(ClusterState currentState) {
return removeDataStream(deleteIndexService, currentState, request);
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
listener.onResponse(new AcknowledgedResponse(true));
}
}
);
}
static ClusterState removeDataStream(
MetadataDeleteIndexService deleteIndexService,
ClusterState currentState,
DeleteDataStreamAction.Request request
) {
Set<String> dataStreams = new HashSet<>();
Set<String> snapshottingDataStreams = new HashSet<>();
for (String name : request.getNames()) {
for (String dataStreamName : currentState.metadata().dataStreams().keySet()) {
if (Regex.simpleMatch(name, dataStreamName)) {
dataStreams.add(dataStreamName);
}
}
snapshottingDataStreams.addAll(SnapshotsService.snapshottingDataStreams(currentState, dataStreams));
}
if (snapshottingDataStreams.isEmpty() == false) {
throw new SnapshotInProgressException(
"Cannot delete data streams that are being snapshotted: "
+ snapshottingDataStreams
+ ". Try again after snapshot finishes or cancel the currently running snapshot."
);
}
Set<Index> backingIndicesToRemove = new HashSet<>();
for (String dataStreamName : dataStreams) {
DataStream dataStream = currentState.metadata().dataStreams().get(dataStreamName);
assert dataStream != null;
backingIndicesToRemove.addAll(dataStream.getIndices());
}
// first delete the data streams and then the indices:
// (this to avoid data stream validation from failing when deleting an index that is part of a data stream
// without updating the data stream)
// TODO: change order when delete index api also updates the data stream the index to be removed is member of
Metadata.Builder metadata = Metadata.builder(currentState.metadata());
for (String ds : dataStreams) {
LOGGER.info("removing data stream [{}]", ds);
metadata.removeDataStream(ds);
}
currentState = ClusterState.builder(currentState).metadata(metadata).build();
return deleteIndexService.deleteIndices(currentState, backingIndicesToRemove);
}
@Override
protected ClusterBlockException checkBlock(DeleteDataStreamAction.Request request, ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
}
}

View File

@ -0,0 +1,119 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams.action;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.health.ClusterStateHealth;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.core.action.GetDataStreamAction;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class GetDataStreamsTransportAction extends TransportMasterNodeReadAction<
GetDataStreamAction.Request,
GetDataStreamAction.Response> {
private static final Logger LOGGER = LogManager.getLogger(GetDataStreamsTransportAction.class);
@Inject
public GetDataStreamsTransportAction(
TransportService transportService,
ClusterService clusterService,
ThreadPool threadPool,
ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver
) {
super(
GetDataStreamAction.NAME,
transportService,
clusterService,
threadPool,
actionFilters,
GetDataStreamAction.Request::new,
indexNameExpressionResolver
);
}
@Override
protected String executor() {
return ThreadPool.Names.SAME;
}
@Override
protected GetDataStreamAction.Response read(StreamInput in) throws IOException {
return new GetDataStreamAction.Response(in);
}
@Override
protected void masterOperation(
GetDataStreamAction.Request request,
ClusterState state,
ActionListener<GetDataStreamAction.Response> listener
) throws Exception {
List<DataStream> dataStreams = getDataStreams(state, indexNameExpressionResolver, request);
List<GetDataStreamAction.Response.DataStreamInfo> dataStreamInfos = new ArrayList<>(dataStreams.size());
for (DataStream dataStream : dataStreams) {
String indexTemplate = MetadataIndexTemplateService.findV2Template(state.metadata(), dataStream.getName(), false);
String ilmPolicyName = null;
if (indexTemplate != null) {
Settings settings = MetadataIndexTemplateService.resolveSettings(state.metadata(), indexTemplate);
ilmPolicyName = settings.get("index.lifecycle.name");
} else {
LOGGER.warn(
"couldn't find any matching template for data stream [{}]. has it been restored (and possibly renamed)"
+ "from a snapshot?",
dataStream.getName()
);
}
ClusterStateHealth streamHealth = new ClusterStateHealth(
state,
dataStream.getIndices().stream().map(Index::getName).toArray(String[]::new)
);
dataStreamInfos.add(
new GetDataStreamAction.Response.DataStreamInfo(dataStream, streamHealth.getStatus(), indexTemplate, ilmPolicyName)
);
}
listener.onResponse(new GetDataStreamAction.Response(dataStreamInfos));
}
static List<DataStream> getDataStreams(
ClusterState clusterState,
IndexNameExpressionResolver iner,
GetDataStreamAction.Request request
) {
List<String> results = iner.dataStreamNames(clusterState, request.indicesOptions(), request.getNames());
Map<String, DataStream> dataStreams = clusterState.metadata().dataStreams();
return results.stream().map(dataStreams::get).sorted(Comparator.comparing(DataStream::getName)).collect(Collectors.toList());
}
@Override
protected ClusterBlockException checkBlock(GetDataStreamAction.Request request, ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
}
}

View File

@ -0,0 +1,35 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams.rest;
import org.elasticsearch.xpack.core.action.CreateDataStreamAction;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
public class RestCreateDataStreamAction extends BaseRestHandler {
@Override
public String getName() {
return "create_data_stream_action";
}
@Override
public List<Route> routes() {
return Collections.singletonList(new Route(RestRequest.Method.PUT, "/_data_stream/{name}"));
}
@Override
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
CreateDataStreamAction.Request putDataStreamRequest = new CreateDataStreamAction.Request(request.param("name"));
return channel -> client.execute(CreateDataStreamAction.INSTANCE, putDataStreamRequest, new RestToXContentListener<>(channel));
}
}

View File

@ -1,31 +1,17 @@
/*
* 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.
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams.rest;
package org.elasticsearch.rest.action.admin.indices;
import org.elasticsearch.action.admin.indices.datastream.DataStreamsStatsAction;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.common.Strings;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
import org.elasticsearch.xpack.core.action.DataStreamsStatsAction;
import java.io.IOException;
import java.util.Arrays;
@ -49,10 +35,11 @@ public class RestDataStreamsStatsAction extends BaseRestHandler {
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
DataStreamsStatsAction.Request dataStreamsStatsRequest = new DataStreamsStatsAction.Request();
boolean forbidClosedIndices = request.paramAsBoolean("forbid_closed_indices", true);
IndicesOptions defaultIndicesOption = forbidClosedIndices ? dataStreamsStatsRequest.indicesOptions()
IndicesOptions defaultIndicesOption = forbidClosedIndices
? dataStreamsStatsRequest.indicesOptions()
: IndicesOptions.strictExpandOpen();
assert dataStreamsStatsRequest.indicesOptions() == IndicesOptions.strictExpandOpenAndForbidClosed() : "DataStreamStats default " +
"indices options changed";
assert dataStreamsStatsRequest.indicesOptions() == IndicesOptions.strictExpandOpenAndForbidClosed() : "DataStreamStats default "
+ "indices options changed";
dataStreamsStatsRequest.indicesOptions(IndicesOptions.fromRequest(request, defaultIndicesOption));
dataStreamsStatsRequest.indices(Strings.splitStringByCommaToArray(request.param("name")));
return channel -> client.execute(DataStreamsStatsAction.INSTANCE, dataStreamsStatsRequest, new RestToXContentListener<>(channel));

View File

@ -0,0 +1,38 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams.rest;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.common.Strings;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
public class RestDeleteDataStreamAction extends BaseRestHandler {
@Override
public String getName() {
return "delete_data_stream_action";
}
@Override
public List<Route> routes() {
return Collections.singletonList(new Route(RestRequest.Method.DELETE, "/_data_stream/{name}"));
}
@Override
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
DeleteDataStreamAction.Request deleteDataStreamRequest = new DeleteDataStreamAction.Request(
Strings.splitStringByCommaToArray(request.param("name"))
);
return channel -> client.execute(DeleteDataStreamAction.INSTANCE, deleteDataStreamRequest, new RestToXContentListener<>(channel));
}
}

View File

@ -0,0 +1,38 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams.rest;
import org.elasticsearch.xpack.core.action.GetDataStreamAction;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.common.Strings;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
public class RestGetDataStreamsAction extends BaseRestHandler {
@Override
public String getName() {
return "get_data_streams_action";
}
@Override
public List<Route> routes() {
return Arrays.asList(new Route(RestRequest.Method.GET, "/_data_stream"), new Route(RestRequest.Method.GET, "/_data_stream/{name}"));
}
@Override
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
GetDataStreamAction.Request getDataStreamsRequest = new GetDataStreamAction.Request(
Strings.splitStringByCommaToArray(request.param("name"))
);
return channel -> client.execute(GetDataStreamAction.INSTANCE, getDataStreamsRequest, new RestToXContentListener<>(channel));
}
}

View File

@ -1,25 +1,12 @@
/*
* 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.
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.action.admin.indices.datastream;
package org.elasticsearch.xpack.core.action;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction.Request;
import org.elasticsearch.xpack.core.action.CreateDataStreamAction.Request;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.test.AbstractWireSerializingTestCase;

View File

@ -1,23 +1,9 @@
/*
* 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.
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.action.admin.indices.datastream;
package org.elasticsearch.xpack.core.action;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
@ -53,20 +39,33 @@ public class DataStreamsStatsResponseTests extends AbstractWireSerializingTestCa
long storeSize = randomLongBetween(250, 1000000000);
totalStoreSize += storeSize;
long maximumTimestamp = randomRecentTimestamp();
dataStreamStats.add(new DataStreamsStatsAction.DataStreamStats(dataStreamName, backingIndices,
new ByteSizeValue(storeSize), maximumTimestamp));
dataStreamStats.add(
new DataStreamsStatsAction.DataStreamStats(dataStreamName, backingIndices, new ByteSizeValue(storeSize), maximumTimestamp)
);
}
int totalShards = randomIntBetween(backingIndicesTotal, backingIndicesTotal * 3);
int successfulShards = randomInt(totalShards);
int failedShards = totalShards - successfulShards;
List<DefaultShardOperationFailedException> exceptions = new ArrayList<>();
for (int i = 0; i < failedShards; i++) {
exceptions.add(new DefaultShardOperationFailedException(randomAlphaOfLength(8).toLowerCase(Locale.getDefault()),
randomInt(totalShards), new ElasticsearchException("boom")));
exceptions.add(
new DefaultShardOperationFailedException(
randomAlphaOfLength(8).toLowerCase(Locale.getDefault()),
randomInt(totalShards),
new ElasticsearchException("boom")
)
);
}
return new DataStreamsStatsAction.Response(totalShards, successfulShards, failedShards, exceptions,
dataStreamCount, backingIndicesTotal, new ByteSizeValue(totalStoreSize),
dataStreamStats.toArray(new DataStreamsStatsAction.DataStreamStats[0]));
return new DataStreamsStatsAction.Response(
totalShards,
successfulShards,
failedShards,
exceptions,
dataStreamCount,
backingIndicesTotal,
new ByteSizeValue(totalStoreSize),
dataStreamStats.toArray(new DataStreamsStatsAction.DataStreamStats[0])
);
}
private static long randomRecentTimestamp() {

View File

@ -0,0 +1,42 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.core.action;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.test.AbstractWireSerializingTestCase;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction.Request;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
public class DeleteDataStreamRequestTests extends AbstractWireSerializingTestCase<Request> {
@Override
protected Writeable.Reader<Request> instanceReader() {
return Request::new;
}
@Override
protected Request createTestInstance() {
return new Request(randomArray(1, 3, String[]::new, () -> randomAlphaOfLength(6)));
}
public void testValidateRequest() {
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[] { "my-data-stream" });
ActionRequestValidationException e = req.validate();
assertNull(e);
}
public void testValidateRequestWithoutName() {
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[0]);
ActionRequestValidationException e = req.validate();
assertNotNull(e);
assertThat(e.validationErrors().size(), equalTo(1));
assertThat(e.validationErrors().get(0), containsString("no data stream(s) specified"));
}
}

View File

@ -0,0 +1,43 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.core.action;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.test.AbstractWireSerializingTestCase;
import org.elasticsearch.xpack.core.action.GetDataStreamAction.Request;
public class GetDataStreamsRequestTests extends AbstractWireSerializingTestCase<Request> {
@Override
protected Writeable.Reader<Request> instanceReader() {
return Request::new;
}
@Override
protected Request createTestInstance() {
final String[] searchParameter;
switch (randomIntBetween(1, 4)) {
case 1:
searchParameter = generateRandomStringArray(3, 8, false, false);
break;
case 2:
String[] parameters = generateRandomStringArray(3, 8, false, false);
for (int k = 0; k < parameters.length; k++) {
parameters[k] = parameters[k] + "*";
}
searchParameter = parameters;
break;
case 3:
searchParameter = new String[] { "*" };
break;
default:
searchParameter = null;
break;
}
return new Request(searchParameter);
}
}

View File

@ -0,0 +1,40 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.core.action;
import org.elasticsearch.cluster.DataStreamTestHelper;
import org.elasticsearch.xpack.core.action.GetDataStreamAction.Response;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.test.AbstractWireSerializingTestCase;
import java.util.ArrayList;
import java.util.List;
public class GetDataStreamsResponseTests extends AbstractWireSerializingTestCase<Response> {
@Override
protected Writeable.Reader<Response> instanceReader() {
return Response::new;
}
@Override
protected Response createTestInstance() {
int numDataStreams = randomIntBetween(0, 8);
List<Response.DataStreamInfo> dataStreams = new ArrayList<>();
for (int i = 0; i < numDataStreams; i++) {
dataStreams.add(
new Response.DataStreamInfo(
DataStreamTestHelper.randomInstance(),
ClusterHealthStatus.GREEN,
randomAlphaOfLengthBetween(2, 10),
randomAlphaOfLengthBetween(2, 10)
)
);
}
return new Response(dataStreams);
}
}

View File

@ -7,9 +7,6 @@
package org.elasticsearch.xpack.datastreams;
import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.DataStreamsStatsAction;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.admin.indices.template.delete.DeleteComposableIndexTemplateAction;
@ -22,6 +19,9 @@ import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.xpack.core.action.CreateDataStreamAction;
import org.elasticsearch.xpack.core.action.DataStreamsStatsAction;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction;
import org.junit.After;
import java.util.Collection;

View File

@ -0,0 +1,160 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams.action;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.MetadataDeleteIndexService;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.index.Index;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.snapshots.SnapshotInProgressException;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import static org.elasticsearch.cluster.DataStreamTestHelper.getClusterStateWithDataStreams;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.equalTo;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class DeleteDataStreamTransportActionTests extends ESTestCase {
public void testDeleteDataStream() {
final String dataStreamName = "my-data-stream";
final List<String> otherIndices = randomSubsetOf(Arrays.asList("foo", "bar", "baz"));
ClusterState cs = getClusterStateWithDataStreams(Collections.singletonList(new Tuple<>(dataStreamName, 2)), otherIndices);
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[] { dataStreamName });
ClusterState newState = DeleteDataStreamTransportAction.removeDataStream(getMetadataDeleteIndexService(), cs, req);
assertThat(newState.metadata().dataStreams().size(), equalTo(0));
assertThat(newState.metadata().indices().size(), equalTo(otherIndices.size()));
for (String indexName : otherIndices) {
assertThat(newState.metadata().indices().get(indexName).getIndex().getName(), equalTo(indexName));
}
}
public void testDeleteMultipleDataStreams() {
String[] dataStreamNames = { "foo", "bar", "baz", "eggplant" };
ClusterState cs = getClusterStateWithDataStreams(
Arrays.asList(
new Tuple<>(dataStreamNames[0], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[1], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[2], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[3], randomIntBetween(1, 3))
),
Collections.emptyList()
);
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[] { "ba*", "eggplant" });
ClusterState newState = DeleteDataStreamTransportAction.removeDataStream(getMetadataDeleteIndexService(), cs, req);
assertThat(newState.metadata().dataStreams().size(), equalTo(1));
DataStream remainingDataStream = newState.metadata().dataStreams().get(dataStreamNames[0]);
assertNotNull(remainingDataStream);
assertThat(newState.metadata().indices().size(), equalTo(remainingDataStream.getIndices().size()));
for (Index i : remainingDataStream.getIndices()) {
assertThat(newState.metadata().indices().get(i.getName()).getIndex(), equalTo(i));
}
}
public void testDeleteSnapshottingDataStream() {
final String dataStreamName = "my-data-stream1";
final String dataStreamName2 = "my-data-stream2";
final List<String> otherIndices = randomSubsetOf(Arrays.asList("foo", "bar", "baz"));
ClusterState cs = getClusterStateWithDataStreams(
Arrays.asList(new Tuple<>(dataStreamName, 2), new Tuple<>(dataStreamName2, 2)),
otherIndices
);
SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.of(
Arrays.asList(createEntry(dataStreamName, "repo1", false), createEntry(dataStreamName2, "repo2", true))
);
ClusterState snapshotCs = ClusterState.builder(cs).putCustom(SnapshotsInProgress.TYPE, snapshotsInProgress).build();
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[] { dataStreamName });
SnapshotInProgressException e = expectThrows(
SnapshotInProgressException.class,
() -> DeleteDataStreamTransportAction.removeDataStream(getMetadataDeleteIndexService(), snapshotCs, req)
);
assertThat(
e.getMessage(),
equalTo(
"Cannot delete data streams that are being snapshotted: [my-data-stream1]. Try again after "
+ "snapshot finishes or cancel the currently running snapshot."
)
);
}
private SnapshotsInProgress.Entry createEntry(String dataStreamName, String repo, boolean partial) {
return new SnapshotsInProgress.Entry(
new Snapshot(repo, new SnapshotId("", "")),
false,
partial,
SnapshotsInProgress.State.STARTED,
Collections.emptyList(),
Collections.singletonList(dataStreamName),
0,
1,
ImmutableOpenMap.of(),
null,
null,
null
);
}
public void testDeleteNonexistentDataStream() {
final String dataStreamName = "my-data-stream";
String[] dataStreamNames = { "foo", "bar", "baz", "eggplant" };
ClusterState cs = getClusterStateWithDataStreams(
Arrays.asList(
new Tuple<>(dataStreamNames[0], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[1], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[2], randomIntBetween(1, 3)),
new Tuple<>(dataStreamNames[3], randomIntBetween(1, 3))
),
Collections.emptyList()
);
DeleteDataStreamAction.Request req = new DeleteDataStreamAction.Request(new String[] { dataStreamName });
ClusterState newState = DeleteDataStreamTransportAction.removeDataStream(getMetadataDeleteIndexService(), cs, req);
assertThat(newState.metadata().dataStreams().size(), equalTo(cs.metadata().dataStreams().size()));
assertThat(
newState.metadata().dataStreams().keySet(),
containsInAnyOrder(cs.metadata().dataStreams().keySet().toArray(Strings.EMPTY_ARRAY))
);
}
@SuppressWarnings("unchecked")
private static MetadataDeleteIndexService getMetadataDeleteIndexService() {
MetadataDeleteIndexService s = mock(MetadataDeleteIndexService.class);
when(s.deleteIndices(any(ClusterState.class), any(Set.class))).thenAnswer(mockInvocation -> {
ClusterState currentState = (ClusterState) mockInvocation.getArguments()[0];
Set<Index> indices = (Set<Index>) mockInvocation.getArguments()[1];
final Metadata.Builder b = Metadata.builder(currentState.metadata());
for (Index index : indices) {
b.remove(index.getName());
}
return ClusterState.builder(currentState).metadata(b.build()).build();
});
return s;
}
}

View File

@ -0,0 +1,111 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.datastreams.action;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.core.action.GetDataStreamAction;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import static org.elasticsearch.cluster.DataStreamTestHelper.getClusterStateWithDataStreams;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
public class GetDataStreamsTransportActionTests extends ESTestCase {
public void testGetDataStream() {
final String dataStreamName = "my-data-stream";
ClusterState cs = getClusterStateWithDataStreams(
Collections.singletonList(new Tuple<>(dataStreamName, 1)),
Collections.emptyList()
);
GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[] { dataStreamName });
List<DataStream> dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(1));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamName));
}
public void testGetDataStreamsWithWildcards() {
final String[] dataStreamNames = { "my-data-stream", "another-data-stream" };
ClusterState cs = getClusterStateWithDataStreams(
Arrays.asList(new Tuple<>(dataStreamNames[0], 1), new Tuple<>(dataStreamNames[1], 1)),
Collections.emptyList()
);
GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[] { dataStreamNames[1].substring(0, 5) + "*" });
List<DataStream> dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(1));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1]));
req = new GetDataStreamAction.Request(new String[] { "*" });
dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(2));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1]));
assertThat(dataStreams.get(1).getName(), equalTo(dataStreamNames[0]));
req = new GetDataStreamAction.Request((String[]) null);
dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(2));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1]));
assertThat(dataStreams.get(1).getName(), equalTo(dataStreamNames[0]));
req = new GetDataStreamAction.Request(new String[] { "matches-none*" });
dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(0));
}
public void testGetDataStreamsWithoutWildcards() {
final String[] dataStreamNames = { "my-data-stream", "another-data-stream" };
ClusterState cs = getClusterStateWithDataStreams(
Arrays.asList(new Tuple<>(dataStreamNames[0], 1), new Tuple<>(dataStreamNames[1], 1)),
Collections.emptyList()
);
GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[] { dataStreamNames[0], dataStreamNames[1] });
List<DataStream> dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(2));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1]));
assertThat(dataStreams.get(1).getName(), equalTo(dataStreamNames[0]));
req = new GetDataStreamAction.Request(new String[] { dataStreamNames[1] });
dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(1));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1]));
req = new GetDataStreamAction.Request(new String[] { dataStreamNames[0] });
dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req);
assertThat(dataStreams.size(), equalTo(1));
assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[0]));
GetDataStreamAction.Request req2 = new GetDataStreamAction.Request(new String[] { "foo" });
IndexNotFoundException e = expectThrows(
IndexNotFoundException.class,
() -> GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req2)
);
assertThat(e.getMessage(), containsString("no such index [foo]"));
}
public void testGetNonexistentDataStream() {
final String dataStreamName = "my-data-stream";
ClusterState cs = ClusterState.builder(new ClusterName("_name")).build();
GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[] { dataStreamName });
IndexNotFoundException e = expectThrows(
IndexNotFoundException.class,
() -> GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req)
);
assertThat(e.getMessage(), containsString("no such index [" + dataStreamName + "]"));
}
}

View File

@ -3,26 +3,6 @@
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
/*
* 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.xpack.datastreams.mapper;
import org.elasticsearch.common.Strings;

View File

@ -8,6 +8,7 @@ dependencies {
testImplementation project(path: xpackModule('ml'))
testImplementation project(path: xpackModule('ml'), configuration: 'testArtifacts')
testImplementation project(path: ':modules:ingest-common')
testImplementation project(path: xpackModule('data-streams'))
}
// location for keys and certificates

View File

@ -6,7 +6,7 @@
package org.elasticsearch.xpack.ml.integration;
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.xpack.core.action.CreateDataStreamAction;
import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.Client;
@ -41,6 +41,7 @@ import org.elasticsearch.transport.Netty4Plugin;
import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin;
import org.elasticsearch.xpack.core.XPackClientPlugin;
import org.elasticsearch.xpack.core.XPackSettings;
import org.elasticsearch.xpack.core.action.DeleteDataStreamAction;
import org.elasticsearch.xpack.core.ilm.DeleteAction;
import org.elasticsearch.xpack.core.ilm.IndexLifecycleMetadata;
import org.elasticsearch.xpack.core.ilm.LifecycleAction;
@ -68,6 +69,7 @@ import org.elasticsearch.xpack.core.ml.notifications.NotificationsIndex;
import org.elasticsearch.xpack.core.security.SecurityField;
import org.elasticsearch.xpack.core.security.authc.TokenMetadata;
import org.elasticsearch.xpack.core.slm.history.SnapshotLifecycleTemplateRegistry;
import org.elasticsearch.xpack.datastreams.DataStreamsPlugin;
import org.elasticsearch.xpack.ilm.IndexLifecycle;
import java.io.IOException;
@ -85,6 +87,7 @@ import java.util.function.Function;
import static org.elasticsearch.test.XContentTestUtils.convertToMap;
import static org.elasticsearch.test.XContentTestUtils.differenceBetweenMapsIgnoringArrayOrder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.is;
/**
@ -109,7 +112,8 @@ abstract class MlNativeIntegTestCase extends ESIntegTestCase {
// This is to reduce log spam
MockPainlessScriptEngine.TestPlugin.class,
// ILM is required for .ml-state template index settings
IndexLifecycle.class);
IndexLifecycle.class,
DataStreamsPlugin.class);
}
@Override
@ -119,7 +123,8 @@ abstract class MlNativeIntegTestCase extends ESIntegTestCase {
Netty4Plugin.class,
ReindexPlugin.class,
// ILM is required for .ml-state template index settings
IndexLifecycle.class);
IndexLifecycle.class,
DataStreamsPlugin.class);
}
@Override
@ -149,6 +154,7 @@ abstract class MlNativeIntegTestCase extends ESIntegTestCase {
protected void cleanUp() {
setUpgradeModeTo(false);
deleteAllDataStreams();
cleanUpResources();
waitForPendingTasks();
}
@ -289,6 +295,14 @@ abstract class MlNativeIntegTestCase extends ESIntegTestCase {
client().execute(CreateDataStreamAction.INSTANCE, new CreateDataStreamAction.Request(dataStreamName)).actionGet();
}
protected static void deleteAllDataStreams() {
AcknowledgedResponse response = client().execute(
DeleteDataStreamAction.INSTANCE,
new DeleteDataStreamAction.Request(new String[]{"*"})
).actionGet();
assertAcked(response);
}
public static class MockPainlessScriptEngine extends MockScriptEngine {
public static final String NAME = "painless";

View File

@ -15,7 +15,7 @@ import org.elasticsearch.action.StepListener;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesAction;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.xpack.core.action.CreateDataStreamAction;
import org.elasticsearch.action.bulk.BulkItemRequest;
import org.elasticsearch.action.bulk.BulkShardRequest;
import org.elasticsearch.action.bulk.TransportShardBulkAction;