Merge branch 'master' into feature-suggest-refactoring

This commit is contained in:
Christoph Büscher 2016-02-04 23:50:57 +01:00
commit 225adcfaeb
113 changed files with 3058 additions and 1607 deletions

View File

@ -82,4 +82,7 @@
(c-set-offset 'func-decl-cont '++)
))
(c-basic-offset . 4)
(c-comment-only-line-offset . (0 . 0)))))
(c-comment-only-line-offset . (0 . 0))
(fill-column . 140)
(fci-rule-column . 140)
(compile-command . "gradle compileTestJava"))))

View File

@ -11,11 +11,12 @@
</module>
<module name="TreeWalker">
<!-- ~3500 violations
<!-- Its our official line length! See checkstyle_suppressions.xml for the files that don't pass this. For now we
suppress the check there but enforce it everywhere else. This prevents the list from getting longer even if it is
unfair. -->
<module name="LineLength">
<property name="max" value="140"/>
</module>
-->
<module name="AvoidStarImport" />
<!-- Doesn't pass but we could make it pass pretty quick.

File diff suppressed because it is too large Load Diff

View File

@ -190,14 +190,11 @@ import org.elasticsearch.action.termvectors.TermVectorsAction;
import org.elasticsearch.action.termvectors.TransportMultiTermVectorsAction;
import org.elasticsearch.action.termvectors.TransportShardMultiTermsVectorAction;
import org.elasticsearch.action.termvectors.TransportTermVectorsAction;
import org.elasticsearch.action.termvectors.dfs.TransportDfsOnlyAction;
import org.elasticsearch.action.update.TransportUpdateAction;
import org.elasticsearch.action.update.UpdateAction;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.multibindings.MapBinder;
import org.elasticsearch.common.inject.multibindings.Multibinder;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.NodeModule;
import java.util.ArrayList;
import java.util.HashMap;
@ -323,8 +320,7 @@ public class ActionModule extends AbstractModule {
registerAction(IndexAction.INSTANCE, TransportIndexAction.class);
registerAction(GetAction.INSTANCE, TransportGetAction.class);
registerAction(TermVectorsAction.INSTANCE, TransportTermVectorsAction.class,
TransportDfsOnlyAction.class);
registerAction(TermVectorsAction.INSTANCE, TransportTermVectorsAction.class);
registerAction(MultiTermVectorsAction.INSTANCE, TransportMultiTermVectorsAction.class,
TransportShardMultiTermsVectorAction.class);
registerAction(DeleteAction.INSTANCE, TransportDeleteAction.class);

View File

@ -34,6 +34,7 @@ import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentBuilderString;
import org.elasticsearch.index.shard.ShardStateMetaData;
import java.io.IOException;
import java.util.ArrayList;
@ -55,7 +56,7 @@ public class IndicesShardStoresResponse extends ActionResponse implements ToXCon
*/
public static class StoreStatus implements Streamable, ToXContent, Comparable<StoreStatus> {
private DiscoveryNode node;
private long version;
private long legacyVersion;
private String allocationId;
private Throwable storeException;
private AllocationStatus allocationStatus;
@ -116,9 +117,9 @@ public class IndicesShardStoresResponse extends ActionResponse implements ToXCon
private StoreStatus() {
}
public StoreStatus(DiscoveryNode node, long version, String allocationId, AllocationStatus allocationStatus, Throwable storeException) {
public StoreStatus(DiscoveryNode node, long legacyVersion, String allocationId, AllocationStatus allocationStatus, Throwable storeException) {
this.node = node;
this.version = version;
this.legacyVersion = legacyVersion;
this.allocationId = allocationId;
this.allocationStatus = allocationStatus;
this.storeException = storeException;
@ -132,10 +133,10 @@ public class IndicesShardStoresResponse extends ActionResponse implements ToXCon
}
/**
* Version of the store
* Version of the store for pre-3.0 shards that have not yet been active
*/
public long getVersion() {
return version;
public long getLegacyVersion() {
return legacyVersion;
}
/**
@ -173,7 +174,7 @@ public class IndicesShardStoresResponse extends ActionResponse implements ToXCon
@Override
public void readFrom(StreamInput in) throws IOException {
node = DiscoveryNode.readNode(in);
version = in.readLong();
legacyVersion = in.readLong();
allocationId = in.readOptionalString();
allocationStatus = AllocationStatus.readFrom(in);
if (in.readBoolean()) {
@ -184,7 +185,7 @@ public class IndicesShardStoresResponse extends ActionResponse implements ToXCon
@Override
public void writeTo(StreamOutput out) throws IOException {
node.writeTo(out);
out.writeLong(version);
out.writeLong(legacyVersion);
out.writeOptionalString(allocationId);
allocationStatus.writeTo(out);
if (storeException != null) {
@ -198,8 +199,12 @@ public class IndicesShardStoresResponse extends ActionResponse implements ToXCon
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
node.toXContent(builder, params);
builder.field(Fields.VERSION, version);
builder.field(Fields.ALLOCATION_ID, allocationId);
if (legacyVersion != ShardStateMetaData.NO_VERSION) {
builder.field(Fields.LEGACY_VERSION, legacyVersion);
}
if (allocationId != null) {
builder.field(Fields.ALLOCATION_ID, allocationId);
}
builder.field(Fields.ALLOCATED, allocationStatus.value());
if (storeException != null) {
builder.startObject(Fields.STORE_EXCEPTION);
@ -215,12 +220,23 @@ public class IndicesShardStoresResponse extends ActionResponse implements ToXCon
return 1;
} else if (other.storeException != null && storeException == null) {
return -1;
} else {
int compare = Long.compare(other.version, version);
}
if (allocationId != null && other.allocationId == null) {
return -1;
} else if (allocationId == null && other.allocationId != null) {
return 1;
} else if (allocationId == null && other.allocationId == null) {
int compare = Long.compare(other.legacyVersion, legacyVersion);
if (compare == 0) {
return Integer.compare(allocationStatus.id, other.allocationStatus.id);
}
return compare;
} else {
int compare = Integer.compare(allocationStatus.id, other.allocationStatus.id);
if (compare == 0) {
return allocationId.compareTo(other.allocationId);
}
return compare;
}
}
}
@ -390,7 +406,7 @@ public class IndicesShardStoresResponse extends ActionResponse implements ToXCon
static final XContentBuilderString FAILURES = new XContentBuilderString("failures");
static final XContentBuilderString STORES = new XContentBuilderString("stores");
// StoreStatus fields
static final XContentBuilderString VERSION = new XContentBuilderString("version");
static final XContentBuilderString LEGACY_VERSION = new XContentBuilderString("legacy_version");
static final XContentBuilderString ALLOCATION_ID = new XContentBuilderString("allocation_id");
static final XContentBuilderString STORE_EXCEPTION = new XContentBuilderString("store_exception");
static final XContentBuilderString ALLOCATED = new XContentBuilderString("allocation");

View File

@ -180,7 +180,7 @@ public class TransportIndicesShardStoresAction extends TransportMasterNodeReadAc
for (NodeGatewayStartedShards response : fetchResponse.responses) {
if (shardExistsInNode(response)) {
IndicesShardStoresResponse.StoreStatus.AllocationStatus allocationStatus = getAllocationStatus(fetchResponse.shardId.getIndexName(), fetchResponse.shardId.id(), response.getNode());
storeStatuses.add(new IndicesShardStoresResponse.StoreStatus(response.getNode(), response.version(), response.allocationId(), allocationStatus, response.storeException()));
storeStatuses.add(new IndicesShardStoresResponse.StoreStatus(response.getNode(), response.legacyVersion(), response.allocationId(), allocationStatus, response.storeException()));
}
}
CollectionUtil.timSort(storeStatuses);
@ -213,7 +213,7 @@ public class TransportIndicesShardStoresAction extends TransportMasterNodeReadAc
* A shard exists/existed in a node only if shard state file exists in the node
*/
private boolean shardExistsInNode(final NodeGatewayStartedShards response) {
return response.storeException() != null || response.version() != -1 || response.allocationId() != null;
return response.storeException() != null || response.legacyVersion() != -1 || response.allocationId() != null;
}
@Override

View File

@ -34,9 +34,7 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static org.elasticsearch.action.ValidateActions.addValidationError;
import static org.elasticsearch.ingest.core.IngestDocument.MetaData;
public class SimulatePipelineRequest extends ActionRequest<SimulatePipelineRequest> {
@ -140,7 +138,7 @@ public class SimulatePipelineRequest extends ActionRequest<SimulatePipelineReque
static Parsed parse(Map<String, Object> config, boolean verbose, PipelineStore pipelineStore) throws Exception {
Map<String, Object> pipelineConfig = ConfigurationUtils.readMap(null, null, config, Fields.PIPELINE);
Pipeline pipeline = PIPELINE_FACTORY.create(SIMULATED_PIPELINE_ID, pipelineConfig, pipelineStore.getProcessorFactoryRegistry());
Pipeline pipeline = PIPELINE_FACTORY.create(SIMULATED_PIPELINE_ID, pipelineConfig, pipelineStore.getProcessorRegistry());
List<IngestDocument> ingestDocumentList = parseDocs(config);
return new Parsed(pipeline, ingestDocumentList, verbose);
}

View File

@ -373,22 +373,6 @@ public class TermVectorsRequest extends SingleShardRequest<TermVectorsRequest> i
return this;
}
/**
* @return <code>true</code> if distributed frequencies should be returned. Otherwise
* <code>false</code>
*/
public boolean dfs() {
return flagsEnum.contains(Flag.Dfs);
}
/**
* Use distributed frequencies instead of shard statistics.
*/
public TermVectorsRequest dfs(boolean dfs) {
setFlag(Flag.Dfs, dfs);
return this;
}
/**
* Return only term vectors for special selected fields. Returns for term
* vectors for all fields if selectedFields == null
@ -583,7 +567,7 @@ public class TermVectorsRequest extends SingleShardRequest<TermVectorsRequest> i
public static enum Flag {
// Do not change the order of these flags we use
// the ordinal for encoding! Only append to the end!
Positions, Offsets, Payloads, FieldStatistics, TermStatistics, Dfs
Positions, Offsets, Payloads, FieldStatistics, TermStatistics
}
/**
@ -616,7 +600,7 @@ public class TermVectorsRequest extends SingleShardRequest<TermVectorsRequest> i
} else if (currentFieldName.equals("field_statistics") || currentFieldName.equals("fieldStatistics")) {
termVectorsRequest.fieldStatistics(parser.booleanValue());
} else if (currentFieldName.equals("dfs")) {
termVectorsRequest.dfs(parser.booleanValue());
throw new IllegalArgumentException("distributed frequencies is not supported anymore for term vectors");
} else if (currentFieldName.equals("per_field_analyzer") || currentFieldName.equals("perFieldAnalyzer")) {
termVectorsRequest.perFieldAnalyzer(readPerFieldAnalyzer(parser.map()));
} else if (currentFieldName.equals("filter")) {

View File

@ -149,14 +149,6 @@ public class TermVectorsRequestBuilder extends ActionRequestBuilder<TermVectorsR
return this;
}
/**
* Sets whether to use distributed frequencies instead of shard statistics.
*/
public TermVectorsRequestBuilder setDfs(boolean dfs) {
request.dfs(dfs);
return this;
}
/**
* Sets whether to return only term vectors for special selected fields. Returns the term
* vectors for all fields if selectedFields == null

View File

@ -32,6 +32,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.termvectors.TermVectorsService;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -80,7 +81,7 @@ public class TransportShardMultiTermsVectorAction extends TransportSingleShardAc
try {
IndexService indexService = indicesService.indexServiceSafe(request.index());
IndexShard indexShard = indexService.getShard(shardId.id());
TermVectorsResponse termVectorsResponse = indexShard.getTermVectors(termVectorsRequest);
TermVectorsResponse termVectorsResponse = TermVectorsService.getTermVectors(indexShard, termVectorsRequest);
termVectorsResponse.updateTookInMillis(termVectorsRequest.startTime());
response.add(request.locations.get(i), termVectorsResponse);
} catch (Throwable t) {

View File

@ -32,6 +32,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.termvectors.TermVectorsService;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -56,6 +57,7 @@ public class TransportTermVectorsAction extends TransportSingleShardAction<TermV
super(settings, TermVectorsAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
TermVectorsRequest::new, ThreadPool.Names.GET);
this.indicesService = indicesService;
}
@Override
@ -83,7 +85,7 @@ public class TransportTermVectorsAction extends TransportSingleShardAction<TermV
protected TermVectorsResponse shardOperation(TermVectorsRequest request, ShardId shardId) {
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
TermVectorsResponse response = indexShard.getTermVectors(request);
TermVectorsResponse response = TermVectorsService.getTermVectors(indexShard, request);
response.updateTookInMillis(request.startTime());
return response;
}

View File

@ -1,112 +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.termvectors.dfs;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.broadcast.BroadcastRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import java.io.IOException;
import java.util.Arrays;
import java.util.Set;
import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
public class DfsOnlyRequest extends BroadcastRequest<DfsOnlyRequest> {
private SearchRequest searchRequest = new SearchRequest();
long nowInMillis;
public DfsOnlyRequest() {
}
public DfsOnlyRequest(Fields termVectorsFields, String[] indices, String[] types, Set<String> selectedFields) throws IOException {
super(indices);
// build a search request with a query of all the terms
final BoolQueryBuilder boolBuilder = boolQuery();
for (String fieldName : termVectorsFields) {
if ((selectedFields != null) && (!selectedFields.contains(fieldName))) {
continue;
}
Terms terms = termVectorsFields.terms(fieldName);
TermsEnum iterator = terms.iterator();
while (iterator.next() != null) {
String text = iterator.term().utf8ToString();
boolBuilder.should(QueryBuilders.termQuery(fieldName, text));
}
}
// wrap a search request object
this.searchRequest = new SearchRequest(indices).types(types).source(new SearchSourceBuilder().query(boolBuilder));
}
public SearchRequest getSearchRequest() {
return searchRequest;
}
@Override
public ActionRequestValidationException validate() {
return searchRequest.validate();
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
this.searchRequest.readFrom(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
this.searchRequest.writeTo(out);
}
public String[] types() {
return this.searchRequest.types();
}
public String routing() {
return this.searchRequest.routing();
}
public String preference() {
return this.searchRequest.preference();
}
@Override
public String toString() {
String sSource = "_na_";
if (searchRequest.source() != null) {
sSource = searchRequest.source().toString();
}
return "[" + Arrays.toString(indices) + "]" + Arrays.toString(types()) + ", source[" + sSource + "]";
}
}

View File

@ -1,73 +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.termvectors.dfs;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.search.dfs.AggregatedDfs;
import java.io.IOException;
import java.util.List;
/**
* A response of a dfs only request.
*/
public class DfsOnlyResponse extends BroadcastResponse {
private AggregatedDfs dfs;
private long tookInMillis;
DfsOnlyResponse(AggregatedDfs dfs, int totalShards, int successfulShards, int failedShards,
List<ShardOperationFailedException> shardFailures, long tookInMillis) {
super(totalShards, successfulShards, failedShards, shardFailures);
this.dfs = dfs;
this.tookInMillis = tookInMillis;
}
public AggregatedDfs getDfs() {
return dfs;
}
public TimeValue getTook() {
return new TimeValue(tookInMillis);
}
public long getTookInMillis() {
return tookInMillis;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
AggregatedDfs.readAggregatedDfs(in);
tookInMillis = in.readVLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
dfs.writeTo(out);
out.writeVLong(tookInMillis);
}
}

View File

@ -1,62 +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.termvectors.dfs;
import org.elasticsearch.action.support.broadcast.BroadcastShardRequest;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.search.internal.ShardSearchRequest;
import org.elasticsearch.search.internal.ShardSearchTransportRequest;
import java.io.IOException;
public class ShardDfsOnlyRequest extends BroadcastShardRequest {
private ShardSearchTransportRequest shardSearchRequest = new ShardSearchTransportRequest();
public ShardDfsOnlyRequest() {
}
ShardDfsOnlyRequest(ShardRouting shardRouting, int numberOfShards, @Nullable String[] filteringAliases, long nowInMillis, DfsOnlyRequest request) {
super(shardRouting.shardId(), request);
this.shardSearchRequest = new ShardSearchTransportRequest(request.getSearchRequest(), shardRouting, numberOfShards,
filteringAliases, nowInMillis);
}
public ShardSearchRequest getShardSearchRequest() {
return shardSearchRequest;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
shardSearchRequest.readFrom(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
shardSearchRequest.writeTo(out);
}
}

View File

@ -1,62 +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.termvectors.dfs;
import org.elasticsearch.action.support.broadcast.BroadcastShardResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.dfs.DfsSearchResult;
import java.io.IOException;
/**
*
*/
class ShardDfsOnlyResponse extends BroadcastShardResponse {
private DfsSearchResult dfsSearchResult = new DfsSearchResult();
ShardDfsOnlyResponse() {
}
ShardDfsOnlyResponse(ShardId shardId, DfsSearchResult dfsSearchResult) {
super(shardId);
this.dfsSearchResult = dfsSearchResult;
}
public DfsSearchResult getDfsSearchResult() {
return dfsSearchResult;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
dfsSearchResult.readFrom(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
dfsSearchResult.writeTo(out);
}
}

View File

@ -1,146 +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.termvectors.dfs;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.TransportBroadcastAction;
import org.elasticsearch.cluster.ClusterService;
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.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.search.controller.SearchPhaseController;
import org.elasticsearch.search.dfs.AggregatedDfs;
import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReferenceArray;
/**
* Get the dfs only with no fetch phase. This is for internal use only.
*/
public class TransportDfsOnlyAction extends TransportBroadcastAction<DfsOnlyRequest, DfsOnlyResponse, ShardDfsOnlyRequest, ShardDfsOnlyResponse> {
public static final String NAME = "internal:index/termvectors/dfs";
private final SearchService searchService;
private final SearchPhaseController searchPhaseController;
@Inject
public TransportDfsOnlyAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, SearchService searchService, SearchPhaseController searchPhaseController) {
super(settings, NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
DfsOnlyRequest::new, ShardDfsOnlyRequest::new, ThreadPool.Names.SEARCH);
this.searchService = searchService;
this.searchPhaseController = searchPhaseController;
}
@Override
protected void doExecute(Task task, DfsOnlyRequest request, ActionListener<DfsOnlyResponse> listener) {
request.nowInMillis = System.currentTimeMillis();
super.doExecute(task, request, listener);
}
@Override
protected ShardDfsOnlyRequest newShardRequest(int numShards, ShardRouting shard, DfsOnlyRequest request) {
String[] filteringAliases = indexNameExpressionResolver.filteringAliases(clusterService.state(), shard.index().getName(), request.indices());
return new ShardDfsOnlyRequest(shard, numShards, filteringAliases, request.nowInMillis, request);
}
@Override
protected ShardDfsOnlyResponse newShardResponse() {
return new ShardDfsOnlyResponse();
}
@Override
protected GroupShardsIterator shards(ClusterState clusterState, DfsOnlyRequest request, String[] concreteIndices) {
Map<String, Set<String>> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, request.routing(), request.indices());
return clusterService.operationRouting().searchShards(clusterState, concreteIndices, routingMap, request.preference());
}
@Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, DfsOnlyRequest request) {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ);
}
@Override
protected ClusterBlockException checkRequestBlock(ClusterState state, DfsOnlyRequest countRequest, String[] concreteIndices) {
return state.blocks().indicesBlockedException(ClusterBlockLevel.READ, concreteIndices);
}
@Override
protected DfsOnlyResponse newResponse(DfsOnlyRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) {
int successfulShards = 0;
int failedShards = 0;
List<ShardOperationFailedException> shardFailures = null;
AtomicArray<DfsSearchResult> dfsResults = new AtomicArray<>(shardsResponses.length());
for (int i = 0; i < shardsResponses.length(); i++) {
Object shardResponse = shardsResponses.get(i);
if (shardResponse == null) {
// simply ignore non active shards
} else if (shardResponse instanceof BroadcastShardOperationFailedException) {
failedShards++;
if (shardFailures == null) {
shardFailures = new ArrayList<>();
}
shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse));
} else {
dfsResults.set(i, ((ShardDfsOnlyResponse) shardResponse).getDfsSearchResult());
successfulShards++;
}
}
AggregatedDfs dfs = searchPhaseController.aggregateDfs(dfsResults);
return new DfsOnlyResponse(dfs, shardsResponses.length(), successfulShards, failedShards, shardFailures, buildTookInMillis(request));
}
@Override
protected ShardDfsOnlyResponse shardOperation(ShardDfsOnlyRequest request) {
DfsSearchResult dfsSearchResult = searchService.executeDfsPhase(request.getShardSearchRequest());
searchService.freeContext(dfsSearchResult.id());
return new ShardDfsOnlyResponse(request.shardId(), dfsSearchResult);
}
/**
* Builds how long it took to execute the dfs request.
*/
protected final long buildTookInMillis(DfsOnlyRequest request) {
// protect ourselves against time going backwards
// negative values don't make sense and we want to be able to serialize that thing as a vLong
return Math.max(1, System.currentTimeMillis() - request.nowInMillis);
}
}

View File

@ -1,23 +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.
*/
/**
* Distributed frequencies.
*/
package org.elasticsearch.action.termvectors.dfs;

View File

@ -94,19 +94,6 @@ public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> imple
return index;
}
/**
* creates a new {@link IndexRoutingTable} with all shard versions normalized
*
* @return new {@link IndexRoutingTable}
*/
public IndexRoutingTable normalizeVersions() {
IndexRoutingTable.Builder builder = new Builder(this.index);
for (IntObjectCursor<IndexShardRoutingTable> cursor : shards) {
builder.addIndexShard(cursor.value.normalizeVersions());
}
return builder.build();
}
public void validate(RoutingTableValidation validation, MetaData metaData) {
if (!metaData.hasIndex(index.getName())) {
validation.addIndexFailure(index.getName(), "Exists in routing does not exists in metadata");

View File

@ -119,40 +119,6 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
this.allInitializingShards = Collections.unmodifiableList(allInitializingShards);
}
/**
* Normalizes all shard routings to the same version.
*/
public IndexShardRoutingTable normalizeVersions() {
if (shards.isEmpty()) {
return this;
}
if (shards.size() == 1) {
return this;
}
long highestVersion = shards.get(0).version();
boolean requiresNormalization = false;
for (int i = 1; i < shards.size(); i++) {
if (shards.get(i).version() != highestVersion) {
requiresNormalization = true;
}
if (shards.get(i).version() > highestVersion) {
highestVersion = shards.get(i).version();
}
}
if (!requiresNormalization) {
return this;
}
List<ShardRouting> shardRoutings = new ArrayList<>(shards.size());
for (int i = 0; i < shards.size(); i++) {
if (shards.get(i).version() == highestVersion) {
shardRoutings.add(shards.get(i));
} else {
shardRoutings.add(new ShardRouting(shards.get(i), highestVersion));
}
}
return new IndexShardRoutingTable(shardId, Collections.unmodifiableList(shardRoutings));
}
/**
* Returns the shards id
*

View File

@ -693,9 +693,9 @@ public class RoutingNodes implements Iterable<RoutingNode> {
/**
* Initializes the current unassigned shard and moves it from the unassigned list.
*/
public void initialize(String nodeId, long version, long expectedShardSize) {
public void initialize(String nodeId, long expectedShardSize) {
innerRemove();
nodes.initialize(new ShardRouting(current, version), nodeId, expectedShardSize);
nodes.initialize(new ShardRouting(current), nodeId, expectedShardSize);
}
/**
@ -711,7 +711,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
/**
* Unsupported operation, just there for the interface. Use {@link #removeAndIgnore()} or
* {@link #initialize(String, long, long)}.
* {@link #initialize(String, long)}.
*/
@Override
public void remove() {

View File

@ -588,7 +588,7 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
}
// normalize the versions right before we build it...
for (ObjectCursor<IndexRoutingTable> indexRoutingTable : indicesRouting.values()) {
indicesRouting.put(indexRoutingTable.value.getIndex().getName(), indexRoutingTable.value.normalizeVersions());
indicesRouting.put(indexRoutingTable.value.getIndex().getName(), indexRoutingTable.value);
}
RoutingTable table = new RoutingTable(version, indicesRouting.build());
indicesRouting = null;

View File

@ -51,7 +51,6 @@ public final class ShardRouting implements Streamable, ToXContent {
private String relocatingNodeId;
private boolean primary;
private ShardRoutingState state;
private long version;
private RestoreSource restoreSource;
private UnassignedInfo unassignedInfo;
private AllocationId allocationId;
@ -65,11 +64,7 @@ public final class ShardRouting implements Streamable, ToXContent {
}
public ShardRouting(ShardRouting copy) {
this(copy, copy.version());
}
public ShardRouting(ShardRouting copy, long version) {
this(copy.index(), copy.id(), copy.currentNodeId(), copy.relocatingNodeId(), copy.restoreSource(), copy.primary(), copy.state(), version, copy.unassignedInfo(), copy.allocationId(), true, copy.getExpectedShardSize());
this(copy.index(), copy.id(), copy.currentNodeId(), copy.relocatingNodeId(), copy.restoreSource(), copy.primary(), copy.state(), copy.unassignedInfo(), copy.allocationId(), true, copy.getExpectedShardSize());
}
/**
@ -77,7 +72,7 @@ public final class ShardRouting implements Streamable, ToXContent {
* by either this class or tests. Visible for testing.
*/
ShardRouting(Index index, int shardId, String currentNodeId,
String relocatingNodeId, RestoreSource restoreSource, boolean primary, ShardRoutingState state, long version,
String relocatingNodeId, RestoreSource restoreSource, boolean primary, ShardRoutingState state,
UnassignedInfo unassignedInfo, AllocationId allocationId, boolean internal, long expectedShardSize) {
this.index = index;
this.shardId = shardId;
@ -86,7 +81,6 @@ public final class ShardRouting implements Streamable, ToXContent {
this.primary = primary;
this.state = state;
this.asList = Collections.singletonList(this);
this.version = version;
this.restoreSource = restoreSource;
this.unassignedInfo = unassignedInfo;
this.allocationId = allocationId;
@ -107,7 +101,7 @@ public final class ShardRouting implements Streamable, ToXContent {
* Creates a new unassigned shard.
*/
public static ShardRouting newUnassigned(Index index, int shardId, RestoreSource restoreSource, boolean primary, UnassignedInfo unassignedInfo) {
return new ShardRouting(index, shardId, null, null, restoreSource, primary, ShardRoutingState.UNASSIGNED, 0, unassignedInfo, null, true, UNAVAILABLE_EXPECTED_SHARD_SIZE);
return new ShardRouting(index, shardId, null, null, restoreSource, primary, ShardRoutingState.UNASSIGNED, unassignedInfo, null, true, UNAVAILABLE_EXPECTED_SHARD_SIZE);
}
public Index index() {
@ -136,13 +130,6 @@ public final class ShardRouting implements Streamable, ToXContent {
}
/**
* The routing version associated with the shard.
*/
public long version() {
return this.version;
}
/**
* The shard is unassigned (not allocated to any node).
*/
@ -214,7 +201,7 @@ public final class ShardRouting implements Streamable, ToXContent {
*/
public ShardRouting buildTargetRelocatingShard() {
assert relocating();
return new ShardRouting(index, shardId, relocatingNodeId, currentNodeId, restoreSource, primary, ShardRoutingState.INITIALIZING, version, unassignedInfo,
return new ShardRouting(index, shardId, relocatingNodeId, currentNodeId, restoreSource, primary, ShardRoutingState.INITIALIZING, unassignedInfo,
AllocationId.newTargetRelocation(allocationId), true, expectedShardSize);
}
@ -313,7 +300,6 @@ public final class ShardRouting implements Streamable, ToXContent {
}
public void readFromThin(StreamInput in) throws IOException {
version = in.readLong();
if (in.readBoolean()) {
currentNodeId = in.readString();
}
@ -352,7 +338,6 @@ public final class ShardRouting implements Streamable, ToXContent {
* @throws IOException if something happens during write
*/
public void writeToThin(StreamOutput out) throws IOException {
out.writeLong(version);
if (currentNodeId != null) {
out.writeBoolean(true);
out.writeString(currentNodeId);
@ -414,7 +399,6 @@ public final class ShardRouting implements Streamable, ToXContent {
*/
void moveToUnassigned(UnassignedInfo unassignedInfo) {
ensureNotFrozen();
version++;
assert state != ShardRoutingState.UNASSIGNED : this;
state = ShardRoutingState.UNASSIGNED;
currentNodeId = null;
@ -429,7 +413,6 @@ public final class ShardRouting implements Streamable, ToXContent {
*/
void initialize(String nodeId, long expectedShardSize) {
ensureNotFrozen();
version++;
assert state == ShardRoutingState.UNASSIGNED : this;
assert relocatingNodeId == null : this;
state = ShardRoutingState.INITIALIZING;
@ -445,7 +428,6 @@ public final class ShardRouting implements Streamable, ToXContent {
*/
void relocate(String relocatingNodeId, long expectedShardSize) {
ensureNotFrozen();
version++;
assert state == ShardRoutingState.STARTED : "current shard has to be started in order to be relocated " + this;
state = ShardRoutingState.RELOCATING;
this.relocatingNodeId = relocatingNodeId;
@ -459,7 +441,6 @@ public final class ShardRouting implements Streamable, ToXContent {
*/
void cancelRelocation() {
ensureNotFrozen();
version++;
assert state == ShardRoutingState.RELOCATING : this;
assert assignedToNode() : this;
assert relocatingNodeId != null : this;
@ -475,7 +456,6 @@ public final class ShardRouting implements Streamable, ToXContent {
void reinitializeShard() {
ensureNotFrozen();
assert state == ShardRoutingState.STARTED;
version++;
state = ShardRoutingState.INITIALIZING;
allocationId = AllocationId.newInitializing();
this.unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.REINITIALIZED, null);
@ -488,7 +468,6 @@ public final class ShardRouting implements Streamable, ToXContent {
*/
void moveToStarted() {
ensureNotFrozen();
version++;
assert state == ShardRoutingState.INITIALIZING : "expected an initializing shard " + this;
relocatingNodeId = null;
restoreSource = null;
@ -507,7 +486,6 @@ public final class ShardRouting implements Streamable, ToXContent {
*/
void moveToPrimary() {
ensureNotFrozen();
version++;
if (primary) {
throw new IllegalShardRoutingStateException(this, "Already primary, can't move to primary");
}
@ -519,7 +497,6 @@ public final class ShardRouting implements Streamable, ToXContent {
*/
void moveFromPrimary() {
ensureNotFrozen();
version++;
if (!primary) {
throw new IllegalShardRoutingStateException(this, "Not primary, can't move to replica");
}
@ -638,26 +615,22 @@ public final class ShardRouting implements Streamable, ToXContent {
if (this == o) {
return true;
}
// we check on instanceof so we also handle the ImmutableShardRouting case as well
if (o == null || !(o instanceof ShardRouting)) {
return false;
}
ShardRouting that = (ShardRouting) o;
if (version != that.version) {
return false;
}
if (unassignedInfo != null ? !unassignedInfo.equals(that.unassignedInfo) : that.unassignedInfo != null) {
return false;
}
return equalsIgnoringMetaData(that);
}
private long hashVersion = version - 1;
private boolean usePreComputedHashCode = false;
private int hashCode = 0;
@Override
public int hashCode() {
if (hashVersion == version) {
if (frozen && usePreComputedHashCode) {
return hashCode;
}
int result = index != null ? index.hashCode() : 0;
@ -666,10 +639,12 @@ public final class ShardRouting implements Streamable, ToXContent {
result = 31 * result + (relocatingNodeId != null ? relocatingNodeId.hashCode() : 0);
result = 31 * result + (primary ? 1 : 0);
result = 31 * result + (state != null ? state.hashCode() : 0);
result = 31 * result + Long.hashCode(version);
result = 31 * result + (restoreSource != null ? restoreSource.hashCode() : 0);
result = 31 * result + (allocationId != null ? allocationId.hashCode() : 0);
result = 31 * result + (unassignedInfo != null ? unassignedInfo.hashCode() : 0);
if (frozen) {
usePreComputedHashCode = true;
}
return hashCode = result;
}
@ -693,7 +668,6 @@ public final class ShardRouting implements Streamable, ToXContent {
} else {
sb.append("[R]");
}
sb.append(", v[").append(version).append("]");
if (this.restoreSource != null) {
sb.append(", restoring[" + restoreSource + "]");
}
@ -718,8 +692,7 @@ public final class ShardRouting implements Streamable, ToXContent {
.field("node", currentNodeId())
.field("relocating_node", relocatingNodeId())
.field("shard", shardId().id())
.field("index", shardId().getIndex().getName())
.field("version", version);
.field("index", shardId().getIndex().getName());
if (expectedShardSize != UNAVAILABLE_EXPECTED_SHARD_SIZE) {
builder.field("expected_shard_size_in_bytes", expectedShardSize);
}

View File

@ -242,8 +242,7 @@ public abstract class AbstractAllocateAllocationCommand implements AllocationCom
if (shardRoutingChanges != null) {
shardRoutingChanges.accept(unassigned);
}
it.initialize(routingNode.nodeId(), unassigned.version(),
allocation.clusterInfo().getShardSize(unassigned, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
it.initialize(routingNode.nodeId(), allocation.clusterInfo().getShardSize(unassigned, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
return;
}
assert false : "shard to initialize not found in list of unassigned shards";

View File

@ -100,10 +100,10 @@ public abstract class CheckFileCommand extends CliTool.Command {
Set<PosixFilePermission> permissionsBeforeWrite = entry.getValue();
Set<PosixFilePermission> permissionsAfterWrite = Files.getPosixFilePermissions(entry.getKey());
if (!permissionsBeforeWrite.equals(permissionsAfterWrite)) {
terminal.printWarn("The file permissions of [" + entry.getKey() + "] have changed "
terminal.println(Terminal.Verbosity.SILENT, "WARNING: The file permissions of [" + entry.getKey() + "] have changed "
+ "from [" + PosixFilePermissions.toString(permissionsBeforeWrite) + "] "
+ "to [" + PosixFilePermissions.toString(permissionsAfterWrite) + "]");
terminal.printWarn("Please ensure that the user account running Elasticsearch has read access to this file!");
terminal.println(Terminal.Verbosity.SILENT, "Please ensure that the user account running Elasticsearch has read access to this file!");
}
}
@ -116,7 +116,7 @@ public abstract class CheckFileCommand extends CliTool.Command {
String ownerBeforeWrite = entry.getValue();
String ownerAfterWrite = Files.getOwner(entry.getKey()).getName();
if (!ownerAfterWrite.equals(ownerBeforeWrite)) {
terminal.printWarn("WARN: Owner of file [" + entry.getKey() + "] used to be [" + ownerBeforeWrite + "], but now is [" + ownerAfterWrite + "]");
terminal.println(Terminal.Verbosity.SILENT, "WARNING: Owner of file [" + entry.getKey() + "] used to be [" + ownerBeforeWrite + "], but now is [" + ownerAfterWrite + "]");
}
}
@ -129,7 +129,7 @@ public abstract class CheckFileCommand extends CliTool.Command {
String groupBeforeWrite = entry.getValue();
String groupAfterWrite = Files.readAttributes(entry.getKey(), PosixFileAttributes.class).group().getName();
if (!groupAfterWrite.equals(groupBeforeWrite)) {
terminal.printWarn("WARN: Group of file [" + entry.getKey() + "] used to be [" + groupBeforeWrite + "], but now is [" + groupAfterWrite + "]");
terminal.println(Terminal.Verbosity.SILENT, "WARNING: Group of file [" + entry.getKey() + "] used to be [" + groupBeforeWrite + "], but now is [" + groupAfterWrite + "]");
}
}

View File

@ -117,7 +117,7 @@ public abstract class CliTool {
} else {
if (args.length == 0) {
terminal.printError("command not specified");
terminal.println(Terminal.Verbosity.SILENT, "ERROR: command not specified");
config.printUsage(terminal);
return ExitStatus.USAGE;
}
@ -125,7 +125,7 @@ public abstract class CliTool {
String cmdName = args[0];
cmd = config.cmd(cmdName);
if (cmd == null) {
terminal.printError("unknown command [" + cmdName + "]. Use [-h] option to list available commands");
terminal.println(Terminal.Verbosity.SILENT, "ERROR: unknown command [" + cmdName + "]. Use [-h] option to list available commands");
return ExitStatus.USAGE;
}
@ -142,7 +142,7 @@ public abstract class CliTool {
try {
return parse(cmd, args).execute(settings, env);
} catch (UserError error) {
terminal.printError(error.getMessage());
terminal.println(Terminal.Verbosity.SILENT, "ERROR: " + error.getMessage());
return error.exitStatus;
}
}
@ -165,8 +165,14 @@ public abstract class CliTool {
// the stack trace into cli parsing lib is not important
throw new UserError(ExitStatus.USAGE, e.toString());
}
Terminal.Verbosity verbosity = Terminal.Verbosity.resolve(cli);
terminal.verbosity(verbosity);
if (cli.hasOption("v")) {
terminal.setVerbosity(Terminal.Verbosity.VERBOSE);
} else if (cli.hasOption("s")) {
terminal.setVerbosity(Terminal.Verbosity.SILENT);
} else {
terminal.setVerbosity(Terminal.Verbosity.NORMAL);
}
return parse(cmd.name(), cli);
}
@ -224,7 +230,7 @@ public abstract class CliTool {
public ExitStatus execute(Settings settings, Environment env) throws Exception {
if (msg != null) {
if (status != ExitStatus.OK) {
terminal.printError(msg);
terminal.println(Terminal.Verbosity.SILENT, "ERROR: " + msg);
} else {
terminal.println(msg);
}

View File

@ -41,7 +41,7 @@ public class HelpPrinter {
}
private static void print(Class clazz, String name, final Terminal terminal) {
terminal.println(Terminal.Verbosity.SILENT);
terminal.println(Terminal.Verbosity.SILENT, "");
try (InputStream input = clazz.getResourceAsStream(name + HELP_FILE_EXT)) {
Streams.readAllLines(input, new Callback<String>() {
@Override
@ -52,6 +52,6 @@ public class HelpPrinter {
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
terminal.println();
terminal.println(Terminal.Verbosity.SILENT, "");
}
}

View File

@ -19,114 +19,71 @@
package org.elasticsearch.common.cli;
import org.apache.commons.cli.CommandLine;
import org.elasticsearch.common.SuppressForbidden;
import java.io.BufferedReader;
import java.io.Console;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.PrintWriter;
import java.util.Locale;
import java.nio.charset.Charset;
import org.elasticsearch.common.SuppressForbidden;
/**
*
* A Terminal wraps access to reading input and writing output for a {@link CliTool}.
*
* The available methods are similar to those of {@link Console}, with the ability
* to read either normal text or a password, and the ability to print a line
* of text. Printing is also gated by the {@link Verbosity} of the terminal,
* which allows {@link #println(Verbosity,String)} calls which act like a logger,
* only actually printing if the verbosity level of the terminal is above
* the verbosity of the message.
*/
@SuppressForbidden(reason = "System#out")
public abstract class Terminal {
public static final Terminal DEFAULT = ConsoleTerminal.supported() ? new ConsoleTerminal() : new SystemTerminal();
/** The default terminal implementation, which will be a console if available, or stdout/stderr if not. */
public static final Terminal DEFAULT = ConsoleTerminal.isSupported() ? new ConsoleTerminal() : new SystemTerminal();
public static enum Verbosity {
SILENT(0), NORMAL(1), VERBOSE(2);
private final int level;
private Verbosity(int level) {
this.level = level;
}
public boolean enabled(Verbosity verbosity) {
return level >= verbosity.level;
}
public static Verbosity resolve(CommandLine cli) {
if (cli.hasOption("s")) {
return SILENT;
}
if (cli.hasOption("v")) {
return VERBOSE;
}
return NORMAL;
}
/** Defines the available verbosity levels of messages to be printed. */
public enum Verbosity {
SILENT, /* always printed */
NORMAL, /* printed when no options are given to cli */
VERBOSE /* printed only when cli is passed verbose option */
}
/** The current verbosity for the terminal, defaulting to {@link Verbosity#NORMAL}. */
private Verbosity verbosity = Verbosity.NORMAL;
public Terminal() {
this(Verbosity.NORMAL);
}
public Terminal(Verbosity verbosity) {
/** Sets the verbosity of the terminal. */
void setVerbosity(Verbosity verbosity) {
this.verbosity = verbosity;
}
public void verbosity(Verbosity verbosity) {
this.verbosity = verbosity;
}
/** Reads clear text from the terminal input. See {@link Console#readLine()}. */
public abstract String readText(String prompt);
public Verbosity verbosity() {
return verbosity;
}
/** Reads password text from the terminal input. See {@link Console#readPassword()}}. */
public abstract char[] readSecret(String prompt);
public abstract String readText(String text, Object... args);
/** Print a message directly to the terminal. */
protected abstract void doPrint(String msg);
public abstract char[] readSecret(String text, Object... args);
protected abstract void printStackTrace(Throwable t);
public void println() {
println(Verbosity.NORMAL);
}
public void println(String msg) {
/** Prints a line to the terminal at {@link Verbosity#NORMAL} verbosity level. */
public final void println(String msg) {
println(Verbosity.NORMAL, msg);
}
public void print(String msg) {
print(Verbosity.NORMAL, msg);
}
public void println(Verbosity verbosity) {
println(verbosity, "");
}
public void println(Verbosity verbosity, String msg) {
print(verbosity, msg + System.lineSeparator());
}
public void print(Verbosity verbosity, String msg) {
if (this.verbosity.enabled(verbosity)) {
doPrint(msg);
/** Prints a line to the terminal at {@code verbosity} level. */
public final void println(Verbosity verbosity, String msg) {
if (this.verbosity.ordinal() >= verbosity.ordinal()) {
doPrint(msg + System.lineSeparator());
}
}
public void printError(String msg) {
println(Verbosity.SILENT, "ERROR: " + msg);
}
public void printWarn(String msg) {
println(Verbosity.SILENT, "WARN: " + msg);
}
protected abstract void doPrint(String msg);
private static class ConsoleTerminal extends Terminal {
final Console console = System.console();
private static final Console console = System.console();
static boolean supported() {
return System.console() != null;
static boolean isSupported() {
return console != null;
}
@Override
@ -136,35 +93,29 @@ public abstract class Terminal {
}
@Override
public String readText(String text, Object... args) {
return console.readLine(text, args);
public String readText(String prompt) {
return console.readLine("%s", prompt);
}
@Override
public char[] readSecret(String text, Object... args) {
return console.readPassword(text, args);
}
@Override
public void printStackTrace(Throwable t) {
t.printStackTrace(console.writer());
public char[] readSecret(String prompt) {
return console.readPassword("%s", prompt);
}
}
@SuppressForbidden(reason = "System#out")
private static class SystemTerminal extends Terminal {
private final PrintWriter printWriter = new PrintWriter(System.out);
@Override
@SuppressForbidden(reason = "System#out")
public void doPrint(String msg) {
System.out.print(msg);
System.out.flush();
}
@Override
public String readText(String text, Object... args) {
print(text);
BufferedReader reader = new BufferedReader(new InputStreamReader(System.in));
public String readText(String text) {
doPrint(text);
BufferedReader reader = new BufferedReader(new InputStreamReader(System.in, Charset.defaultCharset()));
try {
return reader.readLine();
} catch (IOException ioe) {
@ -173,13 +124,8 @@ public abstract class Terminal {
}
@Override
public char[] readSecret(String text, Object... args) {
return readText(text, args).toCharArray();
}
@Override
public void printStackTrace(Throwable t) {
t.printStackTrace(printWriter);
public char[] readSecret(String text) {
return readText(text).toCharArray();
}
}
}

View File

@ -46,6 +46,24 @@ import java.util.stream.Collectors;
/**
* A setting. Encapsulates typical stuff like default value, parsing, and scope.
* Some (dynamic=true) can by modified at run time using the API.
* All settings inside elasticsearch or in any of the plugins should use this type-safe and generic settings infrastructure
* together with {@link AbstractScopedSettings}. This class contains several untility methods that makes it straight forward
* to add settings for the majority of the cases. For instance a simple boolean settings can be defined like this:
* <pre>{@code
* public static final Setting<Boolean>; MY_BOOLEAN = Setting.boolSetting("my.bool.setting", true, false, Scope.CLUSTER);}
* </pre>
* To retrieve the value of the setting a {@link Settings} object can be passed directly to the {@link Setting#get(Settings)} method.
* <pre>
* final boolean myBooleanValue = MY_BOOLEAN.get(settings);
* </pre>
* It's recommended to use typed settings rather than string based settings. For example adding a setting for an enum type:
* <pre>{@code
* public enum Color {
* RED, GREEN, BLUE;
* }
* public static final Setting<Color> MY_BOOLEAN = new Setting<>("my.color.setting", Color.RED.toString(), Color::valueOf, false, Scope.CLUSTER);
* }
* </pre>
*/
public class Setting<T> extends ToXContentToBytes {
private final String key;
@ -84,7 +102,9 @@ public class Setting<T> extends ToXContentToBytes {
}
/**
* Returns the settings key or a prefix if this setting is a group setting
* Returns the settings key or a prefix if this setting is a group setting.
* <b>Note: this method should not be used to retrieve a value from a {@link Settings} object.
* Use {@link #get(Settings)} instead</b>
*
* @see #isGroupSetting()
*/

View File

@ -31,8 +31,6 @@ import java.util.function.Predicate;
/**
* A module that binds the provided settings to the {@link Settings} interface.
*
*
*/
public class SettingsModule extends AbstractModule {
@ -42,7 +40,6 @@ public class SettingsModule extends AbstractModule {
private final Map<String, Setting<?>> indexSettings = new HashMap<>();
private static final Predicate<String> TRIBE_CLIENT_NODE_SETTINGS_PREDICATE = (s) -> s.startsWith("tribe.") && TribeService.TRIBE_SETTING_KEYS.contains(s) == false;
public SettingsModule(Settings settings) {
this.settings = settings;
for (Setting<?> setting : ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) {
@ -69,6 +66,11 @@ public class SettingsModule extends AbstractModule {
bind(IndexScopedSettings.class).toInstance(indexScopedSettings);
}
/**
* Registers a new setting. This method should be used by plugins in order to expose any custom settings the plugin defines.
* Unless a setting is registered the setting is unusable. If a setting is never the less specified the node will reject
* the setting during startup.
*/
public void registerSetting(Setting<?> setting) {
switch (setting.getScope()) {
case CLUSTER:
@ -86,6 +88,10 @@ public class SettingsModule extends AbstractModule {
}
}
/**
* Registers a settings filter pattern that allows to filter out certain settings that for instance contain sensitive information
* or if a setting is for internal purposes only. The given patter must either be a valid settings key or a simple regesp pattern.
*/
public void registerSettingsFilter(String filter) {
if (SettingsFilter.isValidPattern(filter) == false) {
throw new IllegalArgumentException("filter [" + filter +"] is invalid must be either a key or a regex pattern");
@ -103,7 +109,7 @@ public class SettingsModule extends AbstractModule {
}
public void validateTribeSettings(Settings settings, ClusterSettings clusterSettings) {
private void validateTribeSettings(Settings settings, ClusterSettings clusterSettings) {
Map<String, Settings> groups = settings.filter(TRIBE_CLIENT_NODE_SETTINGS_PREDICATE).getGroups("tribe.", true);
for (Map.Entry<String, Settings> tribeSettings : groups.entrySet()) {
Settings thisTribesSettings = tribeSettings.getValue();

View File

@ -93,4 +93,10 @@ public interface Discovery extends LifecycleComponent<Discovery> {
*/
DiscoveryStats stats();
/***
* @return the current value of minimum master nodes, or -1 for not set
*/
int getMinimumMasterNodes();
}

View File

@ -299,6 +299,11 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
return new DiscoveryStats(null);
}
@Override
public int getMinimumMasterNodes() {
return -1;
}
private LocalDiscovery[] members() {
ClusterGroup clusterGroup = clusterGroups.get(clusterName);
if (clusterGroup == null) {

View File

@ -354,6 +354,11 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
return new DiscoveryStats(queueStats);
}
@Override
public int getMinimumMasterNodes() {
return electMaster.minimumMasterNodes();
}
/**
* returns true if zen discovery is started and there is a currently a background thread active for (re)joining
* the cluster used for testing.

View File

@ -25,18 +25,18 @@ import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.env.NodeEnvironment;
import java.nio.file.Path;
import java.util.function.Supplier;
/**
*
@ -51,23 +51,17 @@ public class Gateway extends AbstractComponent implements ClusterStateListener {
private final TransportNodesListGatewayMetaState listGatewayMetaState;
private final String initialMeta;
private final ClusterName clusterName;
private final Supplier<Integer> minimumMasterNodesProvider;
@Inject
public Gateway(Settings settings, ClusterService clusterService, NodeEnvironment nodeEnv, GatewayMetaState metaState,
TransportNodesListGatewayMetaState listGatewayMetaState, ClusterName clusterName) {
TransportNodesListGatewayMetaState listGatewayMetaState, Discovery discovery) {
super(settings);
this.clusterService = clusterService;
this.nodeEnv = nodeEnv;
this.metaState = metaState;
this.listGatewayMetaState = listGatewayMetaState;
this.clusterName = clusterName;
this.minimumMasterNodesProvider = discovery::getMinimumMasterNodes;
clusterService.addLast(this);
// we define what is our minimum "master" nodes, use that to allow for recovery
this.initialMeta = settings.get("gateway.initial_meta", settings.get("gateway.local.initial_meta", settings.get("discovery.zen.minimum_master_nodes", "1")));
}
public void performStateRecovery(final GatewayStateRecoveredListener listener) throws GatewayException {
@ -76,7 +70,7 @@ public class Gateway extends AbstractComponent implements ClusterStateListener {
TransportNodesListGatewayMetaState.NodesGatewayMetaState nodesState = listGatewayMetaState.list(nodesIds.toArray(String.class), null).actionGet();
int requiredAllocation = calcRequiredAllocations(this.initialMeta, nodesIds.size());
int requiredAllocation = Math.max(1, minimumMasterNodesProvider.get());
if (nodesState.failures().length > 0) {
@ -139,39 +133,10 @@ public class Gateway extends AbstractComponent implements ClusterStateListener {
}
}
}
ClusterState.Builder builder = ClusterState.builder(clusterName);
ClusterState.Builder builder = ClusterState.builder(clusterService.state().getClusterName());
builder.metaData(metaDataBuilder);
listener.onSuccess(builder.build());
}
protected int calcRequiredAllocations(final String setting, final int nodeCount) {
int requiredAllocation = 1;
try {
if ("quorum".equals(setting)) {
if (nodeCount > 2) {
requiredAllocation = (nodeCount / 2) + 1;
}
} else if ("quorum-1".equals(setting) || "half".equals(setting)) {
if (nodeCount > 2) {
requiredAllocation = ((1 + nodeCount) / 2);
}
} else if ("one".equals(setting)) {
requiredAllocation = 1;
} else if ("full".equals(setting) || "all".equals(setting)) {
requiredAllocation = nodeCount;
} else if ("full-1".equals(setting) || "all-1".equals(setting)) {
if (nodeCount > 1) {
requiredAllocation = nodeCount - 1;
}
} else {
requiredAllocation = Integer.parseInt(setting);
}
} catch (Exception e) {
logger.warn("failed to derived initial_meta from value {}", setting);
}
return requiredAllocation;
}
public void reset() throws Exception {
try {
Path[] dataPaths = nodeEnv.nodeDataPaths();

View File

@ -21,28 +21,16 @@ package org.elasticsearch.gateway;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.ExtensionPoint;
/**
*
*/
public class GatewayModule extends AbstractModule {
public static final String GATEWAY_TYPE_KEY = "gateway.type";
private final ExtensionPoint.SelectedType<Gateway> gatewayTypes = new ExtensionPoint.SelectedType<>("gateway", Gateway.class);
private final Settings settings;
public GatewayModule(Settings settings) {
this.settings = settings;
registerGatewayType("default", Gateway.class);
}
/**
* Adds a custom Discovery type.
*/
public void registerGatewayType(String type, Class<? extends Gateway> clazz) {
gatewayTypes.registerExtension(type, clazz);
}
@Override
@ -50,7 +38,6 @@ public class GatewayModule extends AbstractModule {
bind(MetaStateService.class).asEagerSingleton();
bind(DanglingIndicesState.class).asEagerSingleton();
bind(GatewayService.class).asEagerSingleton();
gatewayTypes.bindType(binder(), settings, GATEWAY_TYPE_KEY, "default");
bind(TransportNodesListGatewayMetaState.class).asEagerSingleton();
bind(GatewayMetaState.class).asEagerSingleton();
bind(TransportNodesListGatewayStartedShards.class).asEagerSingleton();

View File

@ -39,7 +39,9 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoveryService;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
@ -92,9 +94,12 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
private final AtomicBoolean scheduledRecovery = new AtomicBoolean();
@Inject
public GatewayService(Settings settings, Gateway gateway, AllocationService allocationService, ClusterService clusterService, DiscoveryService discoveryService, ThreadPool threadPool) {
public GatewayService(Settings settings, AllocationService allocationService, ClusterService clusterService,
DiscoveryService discoveryService, ThreadPool threadPool,
NodeEnvironment nodeEnvironment, GatewayMetaState metaState,
TransportNodesListGatewayMetaState listGatewayMetaState, Discovery discovery) {
super(settings);
this.gateway = gateway;
this.gateway = new Gateway(settings, clusterService, nodeEnvironment, metaState, listGatewayMetaState, discovery);
this.allocationService = allocationService;
this.clusterService = clusterService;
this.discoveryService = discoveryService;
@ -233,6 +238,10 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
}
}
public Gateway getGateway() {
return gateway;
}
class GatewayRecoveryListener implements Gateway.GatewayStateRecoveredListener {
@Override

View File

@ -32,6 +32,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardStateMetaData;
import java.util.ArrayList;
import java.util.Collections;
@ -109,7 +110,7 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
final boolean snapshotRestore = shard.restoreSource() != null;
final boolean recoverOnAnyNode = recoverOnAnyNode(indexMetaData);
final NodesAndVersions nodesAndVersions;
final NodesResult nodesResult;
final boolean enoughAllocationsFound;
if (lastActiveAllocationIds.isEmpty()) {
@ -117,20 +118,20 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
// when we load an old index (after upgrading cluster) or restore a snapshot of an old index
// fall back to old version-based allocation mode
// Note that once the shard has been active, lastActiveAllocationIds will be non-empty
nodesAndVersions = buildNodesAndVersions(shard, snapshotRestore || recoverOnAnyNode, allocation.getIgnoreNodes(shard.shardId()), shardState);
nodesResult = buildVersionBasedNodes(shard, snapshotRestore || recoverOnAnyNode, allocation.getIgnoreNodes(shard.shardId()), shardState);
if (snapshotRestore || recoverOnAnyNode) {
enoughAllocationsFound = nodesAndVersions.allocationsFound > 0;
enoughAllocationsFound = nodesResult.allocationsFound > 0;
} else {
enoughAllocationsFound = isEnoughVersionBasedAllocationsFound(shard, indexMetaData, nodesAndVersions);
enoughAllocationsFound = isEnoughVersionBasedAllocationsFound(shard, indexMetaData, nodesResult);
}
logger.debug("[{}][{}]: version-based allocation for pre-{} index found {} allocations of {}, highest version: [{}]", shard.index(), shard.id(), Version.V_3_0_0, nodesAndVersions.allocationsFound, shard, nodesAndVersions.highestVersion);
logger.debug("[{}][{}]: version-based allocation for pre-{} index found {} allocations of {}", shard.index(), shard.id(), Version.V_3_0_0, nodesResult.allocationsFound, shard);
} else {
assert lastActiveAllocationIds.isEmpty() == false;
// use allocation ids to select nodes
nodesAndVersions = buildAllocationIdBasedNodes(shard, snapshotRestore || recoverOnAnyNode,
nodesResult = buildAllocationIdBasedNodes(shard, snapshotRestore || recoverOnAnyNode,
allocation.getIgnoreNodes(shard.shardId()), lastActiveAllocationIds, shardState);
enoughAllocationsFound = nodesAndVersions.allocationsFound > 0;
logger.debug("[{}][{}]: found {} allocations of {} based on allocation ids: [{}]", shard.index(), shard.id(), nodesAndVersions.allocationsFound, shard, lastActiveAllocationIds);
enoughAllocationsFound = nodesResult.allocationsFound > 0;
logger.debug("[{}][{}]: found {} allocations of {} based on allocation ids: [{}]", shard.index(), shard.id(), nodesResult.allocationsFound, shard, lastActiveAllocationIds);
}
if (enoughAllocationsFound == false){
@ -143,22 +144,22 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
} else {
// we can't really allocate, so ignore it and continue
unassignedIterator.removeAndIgnore();
logger.debug("[{}][{}]: not allocating, number_of_allocated_shards_found [{}]", shard.index(), shard.id(), nodesAndVersions.allocationsFound);
logger.debug("[{}][{}]: not allocating, number_of_allocated_shards_found [{}]", shard.index(), shard.id(), nodesResult.allocationsFound);
}
continue;
}
final NodesToAllocate nodesToAllocate = buildNodesToAllocate(shard, allocation, nodesAndVersions.nodes);
final NodesToAllocate nodesToAllocate = buildNodesToAllocate(shard, allocation, nodesResult.nodes);
if (nodesToAllocate.yesNodes.isEmpty() == false) {
DiscoveryNode node = nodesToAllocate.yesNodes.get(0);
logger.debug("[{}][{}]: allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, node);
changed = true;
unassignedIterator.initialize(node.id(), nodesAndVersions.highestVersion, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE);
unassignedIterator.initialize(node.id(), ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE);
} else if (nodesToAllocate.throttleNodes.isEmpty() == true && nodesToAllocate.noNodes.isEmpty() == false) {
DiscoveryNode node = nodesToAllocate.noNodes.get(0);
logger.debug("[{}][{}]: forcing allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, node);
changed = true;
unassignedIterator.initialize(node.id(), nodesAndVersions.highestVersion, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE);
unassignedIterator.initialize(node.id(), ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE);
} else {
// we are throttling this, but we have enough to allocate to this node, ignore it for now
logger.debug("[{}][{}]: throttling allocation [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodesToAllocate.throttleNodes);
@ -173,11 +174,10 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
* lastActiveAllocationIds are added to the list. Otherwise, any node that has a shard is added to the list, but
* entries with matching allocation id are always at the front of the list.
*/
protected NodesAndVersions buildAllocationIdBasedNodes(ShardRouting shard, boolean matchAnyShard, Set<String> ignoreNodes,
Set<String> lastActiveAllocationIds, AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> shardState) {
protected NodesResult buildAllocationIdBasedNodes(ShardRouting shard, boolean matchAnyShard, Set<String> ignoreNodes,
Set<String> lastActiveAllocationIds, AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> shardState) {
LinkedList<DiscoveryNode> matchingNodes = new LinkedList<>();
LinkedList<DiscoveryNode> nonMatchingNodes = new LinkedList<>();
long highestVersion = -1;
for (TransportNodesListGatewayStartedShards.NodeGatewayStartedShards nodeShardState : shardState.getData().values()) {
DiscoveryNode node = nodeShardState.getNode();
String allocationId = nodeShardState.allocationId();
@ -187,7 +187,7 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
}
if (nodeShardState.storeException() == null) {
if (allocationId == null && nodeShardState.version() != -1) {
if (allocationId == null && nodeShardState.legacyVersion() != ShardStateMetaData.NO_VERSION) {
// old shard with no allocation id, assign dummy value so that it gets added below in case of matchAnyShard
allocationId = "_n/a_";
}
@ -205,14 +205,12 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
} else {
matchingNodes.addLast(node);
}
highestVersion = Math.max(highestVersion, nodeShardState.version());
} else if (matchAnyShard) {
if (nodeShardState.primary()) {
nonMatchingNodes.addFirst(node);
} else {
nonMatchingNodes.addLast(node);
}
highestVersion = Math.max(highestVersion, nodeShardState.version());
}
}
}
@ -224,13 +222,13 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
if (logger.isTraceEnabled()) {
logger.trace("{} candidates for allocation: {}", shard, nodes.stream().map(DiscoveryNode::name).collect(Collectors.joining(", ")));
}
return new NodesAndVersions(nodes, nodes.size(), highestVersion);
return new NodesResult(nodes, nodes.size());
}
/**
* used by old version-based allocation
*/
private boolean isEnoughVersionBasedAllocationsFound(ShardRouting shard, IndexMetaData indexMetaData, NodesAndVersions nodesAndVersions) {
private boolean isEnoughVersionBasedAllocationsFound(ShardRouting shard, IndexMetaData indexMetaData, NodesResult nodesAndVersions) {
// check if the counts meets the minimum set
int requiredAllocation = 1;
// if we restore from a repository one copy is more then enough
@ -288,29 +286,29 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
* are added to the list. Otherwise, any node that has a shard is added to the list, but entries with highest
* version are always at the front of the list.
*/
NodesAndVersions buildNodesAndVersions(ShardRouting shard, boolean matchAnyShard, Set<String> ignoreNodes,
AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> shardState) {
NodesResult buildVersionBasedNodes(ShardRouting shard, boolean matchAnyShard, Set<String> ignoreNodes,
AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> shardState) {
final Map<DiscoveryNode, Long> nodesWithVersion = new HashMap<>();
int numberOfAllocationsFound = 0;
long highestVersion = -1;
long highestVersion = ShardStateMetaData.NO_VERSION;
for (TransportNodesListGatewayStartedShards.NodeGatewayStartedShards nodeShardState : shardState.getData().values()) {
long version = nodeShardState.version();
long version = nodeShardState.legacyVersion();
DiscoveryNode node = nodeShardState.getNode();
if (ignoreNodes.contains(node.id())) {
continue;
}
// -1 version means it does not exists, which is what the API returns, and what we expect to
// no version means it does not exists, which is what the API returns, and what we expect to
if (nodeShardState.storeException() == null) {
logger.trace("[{}] on node [{}] has version [{}] of shard", shard, nodeShardState.getNode(), version);
} else {
// when there is an store exception, we disregard the reported version and assign it as -1 (same as shard does not exist)
logger.trace("[{}] on node [{}] has version [{}] but the store can not be opened, treating as version -1", nodeShardState.storeException(), shard, nodeShardState.getNode(), version);
version = -1;
// when there is an store exception, we disregard the reported version and assign it as no version (same as shard does not exist)
logger.trace("[{}] on node [{}] has version [{}] but the store can not be opened, treating no version", nodeShardState.storeException(), shard, nodeShardState.getNode(), version);
version = ShardStateMetaData.NO_VERSION;
}
if (version != -1) {
if (version != ShardStateMetaData.NO_VERSION) {
numberOfAllocationsFound++;
// If we've found a new "best" candidate, clear the
// current candidates and add it
@ -348,7 +346,7 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
logger.trace("{} candidates for allocation: {}", shard, sb.toString());
}
return new NodesAndVersions(Collections.unmodifiableList(nodesWithHighestVersion), numberOfAllocationsFound, highestVersion);
return new NodesResult(Collections.unmodifiableList(nodesWithHighestVersion), numberOfAllocationsFound);
}
/**
@ -362,15 +360,13 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
protected abstract AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> fetchData(ShardRouting shard, RoutingAllocation allocation);
static class NodesAndVersions {
static class NodesResult {
public final List<DiscoveryNode> nodes;
public final int allocationsFound;
public final long highestVersion;
public NodesAndVersions(List<DiscoveryNode> nodes, int allocationsFound, long highestVersion) {
public NodesResult(List<DiscoveryNode> nodes, int allocationsFound) {
this.nodes = nodes;
this.allocationsFound = allocationsFound;
this.highestVersion = highestVersion;
}
}

View File

@ -173,7 +173,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
logger.debug("[{}][{}]: allocating [{}] to [{}] in order to reuse its unallocated persistent store", shard.index(), shard.id(), shard, nodeWithHighestMatch.node());
// we found a match
changed = true;
unassignedIterator.initialize(nodeWithHighestMatch.nodeId(), shard.version(), allocation.clusterInfo().getShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
unassignedIterator.initialize(nodeWithHighestMatch.nodeId(), allocation.clusterInfo().getShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
}
} else if (matchingNodes.hasAnyData() == false) {
// if we didn't manage to find *any* data (regardless of matching sizes), check if the allocation of the replica shard needs to be delayed

View File

@ -138,7 +138,7 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
} catch (Exception exception) {
logger.trace("{} can't open index for shard [{}] in path [{}]", exception, shardId, shardStateMetaData, (shardPath != null) ? shardPath.resolveIndex() : "");
String allocationId = shardStateMetaData.allocationId != null ? shardStateMetaData.allocationId.getId() : null;
return new NodeGatewayStartedShards(clusterService.localNode(), shardStateMetaData.version, allocationId, shardStateMetaData.primary, exception);
return new NodeGatewayStartedShards(clusterService.localNode(), shardStateMetaData.legacyVersion, allocationId, shardStateMetaData.primary, exception);
}
}
// old shard metadata doesn't have the actual index UUID so we need to check if the actual uuid in the metadata
@ -149,11 +149,11 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
} else {
logger.debug("{} shard state info found: [{}]", shardId, shardStateMetaData);
String allocationId = shardStateMetaData.allocationId != null ? shardStateMetaData.allocationId.getId() : null;
return new NodeGatewayStartedShards(clusterService.localNode(), shardStateMetaData.version, allocationId, shardStateMetaData.primary);
return new NodeGatewayStartedShards(clusterService.localNode(), shardStateMetaData.legacyVersion, allocationId, shardStateMetaData.primary);
}
}
logger.trace("{} no local shard info found", shardId);
return new NodeGatewayStartedShards(clusterService.localNode(), -1, null, false);
return new NodeGatewayStartedShards(clusterService.localNode(), ShardStateMetaData.NO_VERSION, null, false);
} catch (Exception e) {
throw new ElasticsearchException("failed to load started shards", e);
}
@ -276,27 +276,27 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
public static class NodeGatewayStartedShards extends BaseNodeResponse {
private long version = -1;
private long legacyVersion = ShardStateMetaData.NO_VERSION; // for pre-3.0 shards that have not yet been active
private String allocationId = null;
private boolean primary = false;
private Throwable storeException = null;
public NodeGatewayStartedShards() {
}
public NodeGatewayStartedShards(DiscoveryNode node, long version, String allocationId, boolean primary) {
this(node, version, allocationId, primary, null);
public NodeGatewayStartedShards(DiscoveryNode node, long legacyVersion, String allocationId, boolean primary) {
this(node, legacyVersion, allocationId, primary, null);
}
public NodeGatewayStartedShards(DiscoveryNode node, long version, String allocationId, boolean primary, Throwable storeException) {
public NodeGatewayStartedShards(DiscoveryNode node, long legacyVersion, String allocationId, boolean primary, Throwable storeException) {
super(node);
this.version = version;
this.legacyVersion = legacyVersion;
this.allocationId = allocationId;
this.primary = primary;
this.storeException = storeException;
}
public long version() {
return this.version;
public long legacyVersion() {
return this.legacyVersion;
}
public String allocationId() {
@ -314,7 +314,7 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
version = in.readLong();
legacyVersion = in.readLong();
allocationId = in.readOptionalString();
primary = in.readBoolean();
if (in.readBoolean()) {
@ -325,7 +325,7 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeLong(version);
out.writeLong(legacyVersion);
out.writeOptionalString(allocationId);
out.writeBoolean(primary);
if (storeException != null) {

View File

@ -23,7 +23,6 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.termvectors.TermVectorsService;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
@ -41,7 +40,6 @@ public final class NodeServicesProvider {
private final ThreadPool threadPool;
private final IndicesQueryCache indicesQueryCache;
private final TermVectorsService termVectorsService;
private final IndicesWarmer warmer;
private final BigArrays bigArrays;
private final Client client;
@ -51,10 +49,9 @@ public final class NodeServicesProvider {
private final CircuitBreakerService circuitBreakerService;
@Inject
public NodeServicesProvider(ThreadPool threadPool, IndicesQueryCache indicesQueryCache, TermVectorsService termVectorsService, @Nullable IndicesWarmer warmer, BigArrays bigArrays, Client client, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry, IndicesFieldDataCache indicesFieldDataCache, CircuitBreakerService circuitBreakerService) {
public NodeServicesProvider(ThreadPool threadPool, IndicesQueryCache indicesQueryCache, @Nullable IndicesWarmer warmer, BigArrays bigArrays, Client client, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry, IndicesFieldDataCache indicesFieldDataCache, CircuitBreakerService circuitBreakerService) {
this.threadPool = threadPool;
this.indicesQueryCache = indicesQueryCache;
this.termVectorsService = termVectorsService;
this.warmer = warmer;
this.bigArrays = bigArrays;
this.client = client;
@ -72,10 +69,6 @@ public final class NodeServicesProvider {
return indicesQueryCache;
}
public TermVectorsService getTermVectorsService() {
return termVectorsService;
}
public IndicesWarmer getWarmer() {
return warmer;
}

View File

@ -287,8 +287,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
.offsets(false)
.payloads(false)
.fieldStatistics(false)
.termStatistics(false)
.dfs(false);
.termStatistics(false);
// for artificial docs to make sure that the id has changed in the item too
if (doc != null) {
termVectorsRequest.doc(doc, true);

View File

@ -34,8 +34,6 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest;
import org.elasticsearch.action.termvectors.TermVectorsRequest;
import org.elasticsearch.action.termvectors.TermVectorsResponse;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
@ -102,7 +100,6 @@ import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.index.store.StoreStats;
import org.elasticsearch.index.suggest.stats.ShardSuggestMetric;
import org.elasticsearch.index.suggest.stats.SuggestStats;
import org.elasticsearch.index.termvectors.TermVectorsService;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.index.translog.TranslogStats;
@ -148,7 +145,6 @@ public class IndexShard extends AbstractIndexShardComponent {
private final ShardRequestCache shardQueryCache;
private final ShardFieldData shardFieldData;
private final PercolatorQueriesRegistry percolatorQueriesRegistry;
private final TermVectorsService termVectorsService;
private final IndexFieldDataService indexFieldDataService;
private final ShardSuggestMetric shardSuggestMetric = new ShardSuggestMetric();
private final ShardBitsetFilterCache shardBitsetFilterCache;
@ -232,7 +228,6 @@ public class IndexShard extends AbstractIndexShardComponent {
listenersList.add(internalIndexingStats);
this.indexingOperationListeners = new IndexingOperationListener.CompositeListener(listenersList, logger);
this.getService = new ShardGetService(indexSettings, this, mapperService);
this.termVectorsService = provider.getTermVectorsService();
this.searchService = new ShardSearchStats(slowLog);
this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings);
this.indicesQueryCache = provider.getIndicesQueryCache();
@ -345,9 +340,8 @@ public class IndexShard extends AbstractIndexShardComponent {
if (!newRouting.primary() && currentRouting.primary()) {
logger.warn("suspect illegal state: trying to move shard from primary mode to replica mode");
}
// if its the same routing except for some metadata info, return
if (currentRouting.equalsIgnoringMetaData(newRouting)) {
this.shardRouting = newRouting; // might have a new version
// if its the same routing, return
if (currentRouting.equals(newRouting)) {
return;
}
}
@ -657,10 +651,6 @@ public class IndexShard extends AbstractIndexShardComponent {
return segmentsStats;
}
public TermVectorsResponse getTermVectors(TermVectorsRequest request) {
return this.termVectorsService.getTermVectors(this, request);
}
public WarmerStats warmerStats() {
return shardWarmerService.stats();
}
@ -1365,21 +1355,16 @@ public class IndexShard extends AbstractIndexShardComponent {
try {
final String writeReason;
if (currentRouting == null) {
writeReason = "freshly started, version [" + newRouting.version() + "]";
} else if (currentRouting.version() < newRouting.version()) {
writeReason = "version changed from [" + currentRouting.version() + "] to [" + newRouting.version() + "]";
writeReason = "freshly started, allocation id [" + newRouting.allocationId() + "]";
} else if (currentRouting.equals(newRouting) == false) {
writeReason = "routing changed from " + currentRouting + " to " + newRouting;
} else {
logger.trace("skip writing shard state, has been written before; previous version: [" +
currentRouting.version() + "] current version [" + newRouting.version() + "]");
assert currentRouting.version() <= newRouting.version() : "version should not go backwards for shardID: " + shardId +
" previous version: [" + currentRouting.version() + "] current version [" + newRouting.version() + "]";
logger.trace("{} skip writing shard state, has been written before", shardId);
return;
}
final ShardStateMetaData newShardStateMetadata = new ShardStateMetaData(newRouting.version(), newRouting.primary(), getIndexUUID(), newRouting.allocationId());
final ShardStateMetaData newShardStateMetadata = new ShardStateMetaData(newRouting.primary(), getIndexUUID(), newRouting.allocationId());
logger.trace("{} writing shard state, reason [{}]", shardId, writeReason);
ShardStateMetaData.FORMAT.write(newShardStateMetadata, newShardStateMetadata.version, shardPath().getShardStatePath());
ShardStateMetaData.FORMAT.write(newShardStateMetadata, newShardStateMetadata.legacyVersion, shardPath().getShardStatePath());
} catch (IOException e) { // this is how we used to handle it.... :(
logger.warn("failed to write shard state", e);
// we failed to write the shard state, we will try and write

View File

@ -41,15 +41,21 @@ public final class ShardStateMetaData {
private static final String INDEX_UUID_KEY = "index_uuid";
private static final String ALLOCATION_ID_KEY = "allocation_id";
public final long version;
public static final long NO_VERSION = -1L;
public final long legacyVersion; // for pre-3.0 shards that have not yet been active
public final String indexUUID;
public final boolean primary;
@Nullable
public final AllocationId allocationId; // can be null if we read from legacy format (see fromXContent and MultiDataPathUpgrader)
public ShardStateMetaData(long version, boolean primary, String indexUUID, AllocationId allocationId) {
public ShardStateMetaData(boolean primary, String indexUUID, AllocationId allocationId) {
this(NO_VERSION, primary, indexUUID, allocationId);
}
ShardStateMetaData(long legacyVersion, boolean primary, String indexUUID, AllocationId allocationId) {
assert indexUUID != null;
this.version = version;
this.legacyVersion = legacyVersion;
this.primary = primary;
this.indexUUID = indexUUID;
this.allocationId = allocationId;
@ -69,7 +75,7 @@ public final class ShardStateMetaData {
if (primary != that.primary) {
return false;
}
if (version != that.version) {
if (legacyVersion != that.legacyVersion) {
return false;
}
if (indexUUID != null ? !indexUUID.equals(that.indexUUID) : that.indexUUID != null) {
@ -84,7 +90,7 @@ public final class ShardStateMetaData {
@Override
public int hashCode() {
int result = Long.hashCode(version);
int result = Long.hashCode(legacyVersion);
result = 31 * result + (indexUUID != null ? indexUUID.hashCode() : 0);
result = 31 * result + (allocationId != null ? allocationId.hashCode() : 0);
result = 31 * result + (primary ? 1 : 0);
@ -93,7 +99,7 @@ public final class ShardStateMetaData {
@Override
public String toString() {
return "version [" + version + "], primary [" + primary + "], allocation [" + allocationId + "]";
return "version [" + legacyVersion + "], primary [" + primary + "], allocation [" + allocationId + "]";
}
public static final MetaDataStateFormat<ShardStateMetaData> FORMAT = new MetaDataStateFormat<ShardStateMetaData>(XContentType.JSON, SHARD_STATE_FILE_PREFIX) {
@ -107,7 +113,7 @@ public final class ShardStateMetaData {
@Override
public void toXContent(XContentBuilder builder, ShardStateMetaData shardStateMetaData) throws IOException {
builder.field(VERSION_KEY, shardStateMetaData.version);
builder.field(VERSION_KEY, shardStateMetaData.legacyVersion);
builder.field(PRIMARY_KEY, shardStateMetaData.primary);
builder.field(INDEX_UUID_KEY, shardStateMetaData.indexUUID);
if (shardStateMetaData.allocationId != null) {
@ -121,7 +127,7 @@ public final class ShardStateMetaData {
if (token == null) {
return null;
}
long version = -1;
long version = NO_VERSION;
Boolean primary = null;
String currentFieldName = null;
String indexUUID = IndexMetaData.INDEX_UUID_NA_VALUE;
@ -152,9 +158,6 @@ public final class ShardStateMetaData {
if (primary == null) {
throw new CorruptStateException("missing value for [primary] in shard state");
}
if (version == -1) {
throw new CorruptStateException("missing value for [version] in shard state");
}
return new ShardStateMetaData(version, primary, indexUUID, allocationId);
}
};

View File

@ -31,14 +31,9 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.termvectors.TermVectorsFilter;
import org.elasticsearch.action.termvectors.TermVectorsRequest;
import org.elasticsearch.action.termvectors.TermVectorsResponse;
import org.elasticsearch.action.termvectors.dfs.DfsOnlyRequest;
import org.elasticsearch.action.termvectors.dfs.DfsOnlyResponse;
import org.elasticsearch.action.termvectors.dfs.TransportDfsOnlyAction;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.get.GetField;
@ -71,15 +66,10 @@ import static org.elasticsearch.index.mapper.SourceToParse.source;
public class TermVectorsService {
private final TransportDfsOnlyAction dfsAction;
@Inject
public TermVectorsService(TransportDfsOnlyAction dfsAction) {
this.dfsAction = dfsAction;
}
private TermVectorsService() {}
public TermVectorsResponse getTermVectors(IndexShard indexShard, TermVectorsRequest request) {
public static TermVectorsResponse getTermVectors(IndexShard indexShard, TermVectorsRequest request) {
final TermVectorsResponse termVectorsResponse = new TermVectorsResponse(indexShard.shardId().getIndex().getName(), request.type(), request.id());
final Term uidTerm = new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(request.type(), request.id()));
@ -137,10 +127,6 @@ public class TermVectorsService {
}
/* if there are term vectors, optional compute dfs and/or terms filtering */
if (termVectorsByField != null) {
if (useDfs(request)) {
dfs = getAggregatedDfs(termVectorsByField, request);
}
if (request.filterSettings() != null) {
termVectorsFilter = new TermVectorsFilter(termVectorsByField, topLevelFields, request.selectedFields(), dfs);
termVectorsFilter.setSettings(request.filterSettings());
@ -162,7 +148,7 @@ public class TermVectorsService {
return termVectorsResponse;
}
private void handleFieldWildcards(IndexShard indexShard, TermVectorsRequest request) {
private static void handleFieldWildcards(IndexShard indexShard, TermVectorsRequest request) {
Set<String> fieldNames = new HashSet<>();
for (String pattern : request.selectedFields()) {
fieldNames.addAll(indexShard.mapperService().simpleMatchToIndexNames(pattern));
@ -170,7 +156,7 @@ public class TermVectorsService {
request.selectedFields(fieldNames.toArray(Strings.EMPTY_ARRAY));
}
private boolean isValidField(MappedFieldType fieldType) {
private static boolean isValidField(MappedFieldType fieldType) {
// must be a string
if (!(fieldType instanceof StringFieldMapper.StringFieldType)) {
return false;
@ -182,7 +168,7 @@ public class TermVectorsService {
return true;
}
private Fields addGeneratedTermVectors(IndexShard indexShard, Engine.GetResult get, Fields termVectorsByField, TermVectorsRequest request, Set<String> selectedFields) throws IOException {
private static Fields addGeneratedTermVectors(IndexShard indexShard, Engine.GetResult get, Fields termVectorsByField, TermVectorsRequest request, Set<String> selectedFields) throws IOException {
/* only keep valid fields */
Set<String> validFields = new HashSet<>();
for (String field : selectedFields) {
@ -215,7 +201,7 @@ public class TermVectorsService {
}
}
private Analyzer getAnalyzerAtField(IndexShard indexShard, String field, @Nullable Map<String, String> perFieldAnalyzer) {
private static Analyzer getAnalyzerAtField(IndexShard indexShard, String field, @Nullable Map<String, String> perFieldAnalyzer) {
MapperService mapperService = indexShard.mapperService();
Analyzer analyzer;
if (perFieldAnalyzer != null && perFieldAnalyzer.containsKey(field)) {
@ -229,7 +215,7 @@ public class TermVectorsService {
return analyzer;
}
private Set<String> getFieldsToGenerate(Map<String, String> perAnalyzerField, Fields fieldsObject) {
private static Set<String> getFieldsToGenerate(Map<String, String> perAnalyzerField, Fields fieldsObject) {
Set<String> selectedFields = new HashSet<>();
for (String fieldName : fieldsObject) {
if (perAnalyzerField.containsKey(fieldName)) {
@ -239,7 +225,7 @@ public class TermVectorsService {
return selectedFields;
}
private Fields generateTermVectors(IndexShard indexShard, Collection<GetField> getFields, boolean withOffsets, @Nullable Map<String, String> perFieldAnalyzer, Set<String> fields)
private static Fields generateTermVectors(IndexShard indexShard, Collection<GetField> getFields, boolean withOffsets, @Nullable Map<String, String> perFieldAnalyzer, Set<String> fields)
throws IOException {
/* store document in memory index */
MemoryIndex index = new MemoryIndex(withOffsets);
@ -258,7 +244,7 @@ public class TermVectorsService {
return MultiFields.getFields(index.createSearcher().getIndexReader());
}
private Fields generateTermVectorsFromDoc(IndexShard indexShard, TermVectorsRequest request, boolean doAllFields) throws Throwable {
private static Fields generateTermVectorsFromDoc(IndexShard indexShard, TermVectorsRequest request, boolean doAllFields) throws Throwable {
// parse the document, at the moment we do update the mapping, just like percolate
ParsedDocument parsedDocument = parseDocument(indexShard, indexShard.shardId().getIndexName(), request.type(), request.doc());
@ -289,7 +275,7 @@ public class TermVectorsService {
return generateTermVectors(indexShard, getFields, request.offsets(), request.perFieldAnalyzer(), seenFields);
}
private ParsedDocument parseDocument(IndexShard indexShard, String index, String type, BytesReference doc) throws Throwable {
private static ParsedDocument parseDocument(IndexShard indexShard, String index, String type, BytesReference doc) throws Throwable {
MapperService mapperService = indexShard.mapperService();
DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(type);
ParsedDocument parsedDocument = docMapper.getDocumentMapper().parse(source(doc).index(index).type(type).id("_id_for_tv_api"));
@ -299,7 +285,7 @@ public class TermVectorsService {
return parsedDocument;
}
private Fields mergeFields(Fields fields1, Fields fields2) throws IOException {
private static Fields mergeFields(Fields fields1, Fields fields2) throws IOException {
ParallelFields parallelFields = new ParallelFields();
for (String fieldName : fields2) {
Terms terms = fields2.terms(fieldName);
@ -346,14 +332,4 @@ public class TermVectorsService {
}
}
private boolean useDfs(TermVectorsRequest request) {
return request.dfs() && (request.fieldStatistics() || request.termStatistics());
}
private AggregatedDfs getAggregatedDfs(Fields termVectorsFields, TermVectorsRequest request) throws IOException {
DfsOnlyRequest dfsOnlyRequest = new DfsOnlyRequest(termVectorsFields, new String[]{request.index()},
new String[]{request.type()}, request.selectedFields());
DfsOnlyResponse response = dfsAction.execute(dfsOnlyRequest).actionGet();
return response.getDfs();
}
}

View File

@ -172,7 +172,6 @@ public class IndicesModule extends AbstractModule {
bind(UpdateHelper.class).asEagerSingleton();
bind(MetaDataIndexUpgradeService.class).asEagerSingleton();
bind(IndicesFieldDataCacheListener.class).asEagerSingleton();
bind(TermVectorsService.class).asEagerSingleton();
bind(NodeServicesProvider.class).asEagerSingleton();
}

View File

@ -33,10 +33,10 @@ public class IngestService implements Closeable {
private final PipelineStore pipelineStore;
private final PipelineExecutionService pipelineExecutionService;
private final ProcessorsRegistry processorsRegistry;
private final ProcessorsRegistry.Builder processorsRegistryBuilder;
public IngestService(Settings settings, ThreadPool threadPool, ProcessorsRegistry processorsRegistry) {
this.processorsRegistry = processorsRegistry;
public IngestService(Settings settings, ThreadPool threadPool, ProcessorsRegistry.Builder processorsRegistryBuilder) {
this.processorsRegistryBuilder = processorsRegistryBuilder;
this.pipelineStore = new PipelineStore(settings);
this.pipelineExecutionService = new PipelineExecutionService(pipelineStore, threadPool);
}
@ -50,7 +50,7 @@ public class IngestService implements Closeable {
}
public void setScriptService(ScriptService scriptService) {
pipelineStore.buildProcessorFactoryRegistry(processorsRegistry, scriptService);
pipelineStore.buildProcessorFactoryRegistry(processorsRegistryBuilder, scriptService);
}
@Override

View File

@ -19,7 +19,6 @@
package org.elasticsearch.ingest;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener;
@ -48,12 +47,11 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
public class PipelineStore extends AbstractComponent implements Closeable, ClusterStateListener {
private final Pipeline.Factory factory = new Pipeline.Factory();
private Map<String, Processor.Factory> processorFactoryRegistry;
private ProcessorsRegistry processorRegistry;
// Ideally this should be in IngestMetadata class, but we don't have the processor factories around there.
// We know of all the processor factories when a node with all its plugin have been initialized. Also some
@ -65,27 +63,16 @@ public class PipelineStore extends AbstractComponent implements Closeable, Clust
super(settings);
}
public void buildProcessorFactoryRegistry(ProcessorsRegistry processorsRegistry, ScriptService scriptService) {
Map<String, Processor.Factory> processorFactories = new HashMap<>();
public void buildProcessorFactoryRegistry(ProcessorsRegistry.Builder processorsRegistryBuilder, ScriptService scriptService) {
TemplateService templateService = new InternalTemplateService(scriptService);
for (Map.Entry<String, Function<TemplateService, Processor.Factory<?>>> entry : processorsRegistry.entrySet()) {
Processor.Factory processorFactory = entry.getValue().apply(templateService);
processorFactories.put(entry.getKey(), processorFactory);
}
this.processorFactoryRegistry = Collections.unmodifiableMap(processorFactories);
this.processorRegistry = processorsRegistryBuilder.build(templateService);
}
@Override
public void close() throws IOException {
// TODO: When org.elasticsearch.node.Node can close Closable instances we should try to remove this code,
// since any wired closable should be able to close itself
List<Closeable> closeables = new ArrayList<>();
for (Processor.Factory factory : processorFactoryRegistry.values()) {
if (factory instanceof Closeable) {
closeables.add((Closeable) factory);
}
}
IOUtils.close(closeables);
processorRegistry.close();
}
@Override
@ -102,7 +89,7 @@ public class PipelineStore extends AbstractComponent implements Closeable, Clust
Map<String, Pipeline> pipelines = new HashMap<>();
for (PipelineConfiguration pipeline : ingestMetadata.getPipelines().values()) {
try {
pipelines.put(pipeline.getId(), factory.create(pipeline.getId(), pipeline.getConfigAsMap(), processorFactoryRegistry));
pipelines.put(pipeline.getId(), factory.create(pipeline.getId(), pipeline.getConfigAsMap(), processorRegistry));
} catch (ElasticsearchParseException e) {
throw e;
} catch (Exception e) {
@ -156,7 +143,7 @@ public class PipelineStore extends AbstractComponent implements Closeable, Clust
// validates the pipeline and processor configuration before submitting a cluster update task:
Map<String, Object> pipelineConfig = XContentHelper.convertToMap(request.getSource(), false).v2();
try {
factory.create(request.getId(), pipelineConfig, processorFactoryRegistry);
factory.create(request.getId(), pipelineConfig, processorRegistry);
} catch(Exception e) {
listener.onFailure(e);
return;
@ -199,8 +186,8 @@ public class PipelineStore extends AbstractComponent implements Closeable, Clust
return pipelines.get(id);
}
public Map<String, Processor.Factory> getProcessorFactoryRegistry() {
return processorFactoryRegistry;
public ProcessorsRegistry getProcessorRegistry() {
return processorRegistry;
}
/**

View File

@ -19,29 +19,69 @@
package org.elasticsearch.ingest;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ingest.core.Processor;
import org.elasticsearch.ingest.core.TemplateService;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import java.util.function.BiFunction;
public class ProcessorsRegistry {
public final class ProcessorsRegistry implements Closeable {
private final Map<String, Function<TemplateService, Processor.Factory<?>>> processorFactoryProviders = new HashMap<>();
private final Map<String, Processor.Factory> processorFactories;
/**
* Adds a processor factory under a specific name.
*/
public void registerProcessor(String name, Function<TemplateService, Processor.Factory<?>> processorFactoryProvider) {
Function<TemplateService, Processor.Factory<?>> provider = processorFactoryProviders.putIfAbsent(name, processorFactoryProvider);
if (provider != null) {
throw new IllegalArgumentException("Processor factory already registered for name [" + name + "]");
private ProcessorsRegistry(TemplateService templateService,
Map<String, BiFunction<TemplateService, ProcessorsRegistry, Processor.Factory<?>>> providers) {
Map<String, Processor.Factory> processorFactories = new HashMap<>();
for (Map.Entry<String, BiFunction<TemplateService, ProcessorsRegistry, Processor.Factory<?>>> entry : providers.entrySet()) {
processorFactories.put(entry.getKey(), entry.getValue().apply(templateService, this));
}
this.processorFactories = Collections.unmodifiableMap(processorFactories);
}
public Set<Map.Entry<String, Function<TemplateService, Processor.Factory<?>>>> entrySet() {
return processorFactoryProviders.entrySet();
public Processor.Factory getProcessorFactory(String name) {
return processorFactories.get(name);
}
@Override
public void close() throws IOException {
List<Closeable> closeables = new ArrayList<>();
for (Processor.Factory factory : processorFactories.values()) {
if (factory instanceof Closeable) {
closeables.add((Closeable) factory);
}
}
IOUtils.close(closeables);
}
// For testing:
Map<String, Processor.Factory> getProcessorFactories() {
return processorFactories;
}
public static final class Builder {
private final Map<String, BiFunction<TemplateService, ProcessorsRegistry, Processor.Factory<?>>> providers = new HashMap<>();
/**
* Adds a processor factory under a specific name.
*/
public void registerProcessor(String name, BiFunction<TemplateService, ProcessorsRegistry, Processor.Factory<?>> provider) {
BiFunction<TemplateService, ProcessorsRegistry, Processor.Factory<?>> previous = this.providers.putIfAbsent(name, provider);
if (previous != null) {
throw new IllegalArgumentException("Processor factory already registered for name [" + name + "]");
}
}
public ProcessorsRegistry build(TemplateService templateService) {
return new ProcessorsRegistry(templateService, providers);
}
}
}

View File

@ -19,9 +19,12 @@
package org.elasticsearch.ingest.core;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.ingest.ProcessorsRegistry;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@ -178,4 +181,34 @@ public final class ConfigurationUtils {
}
return exception;
}
public static List<Processor> readProcessorConfigs(List<Map<String, Map<String, Object>>> processorConfigs, ProcessorsRegistry processorRegistry) throws Exception {
List<Processor> processors = new ArrayList<>();
if (processorConfigs != null) {
for (Map<String, Map<String, Object>> processorConfigWithKey : processorConfigs) {
for (Map.Entry<String, Map<String, Object>> entry : processorConfigWithKey.entrySet()) {
processors.add(readProcessor(processorRegistry, entry.getKey(), entry.getValue()));
}
}
}
return processors;
}
private static Processor readProcessor(ProcessorsRegistry processorRegistry, String type, Map<String, Object> config) throws Exception {
Processor.Factory factory = processorRegistry.getProcessorFactory(type);
if (factory != null) {
List<Map<String, Map<String, Object>>> onFailureProcessorConfigs = ConfigurationUtils.readOptionalList(null, null, config, Pipeline.ON_FAILURE_KEY);
List<Processor> onFailureProcessors = readProcessorConfigs(onFailureProcessorConfigs, processorRegistry);
Processor processor;
processor = factory.create(config);
if (!config.isEmpty()) {
throw new ElasticsearchParseException("processor [" + type + "] doesn't support one or more provided configuration parameters " + Arrays.toString(config.keySet().toArray()));
}
if (onFailureProcessors.isEmpty()) {
return processor;
}
return new CompoundProcessor(Collections.singletonList(processor), onFailureProcessors);
}
throw new ElasticsearchParseException("No processor type exists with name [" + type + "]");
}
}

View File

@ -20,8 +20,8 @@
package org.elasticsearch.ingest.core;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.ingest.ProcessorsRegistry;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@ -85,12 +85,12 @@ public final class Pipeline {
public final static class Factory {
public Pipeline create(String id, Map<String, Object> config, Map<String, Processor.Factory> processorRegistry) throws Exception {
public Pipeline create(String id, Map<String, Object> config, ProcessorsRegistry processorRegistry) throws Exception {
String description = ConfigurationUtils.readOptionalStringProperty(null, null, config, DESCRIPTION_KEY);
List<Map<String, Map<String, Object>>> processorConfigs = ConfigurationUtils.readList(null, null, config, PROCESSORS_KEY);
List<Processor> processors = readProcessorConfigs(processorConfigs, processorRegistry);
List<Processor> processors = ConfigurationUtils.readProcessorConfigs(processorConfigs, processorRegistry);
List<Map<String, Map<String, Object>>> onFailureProcessorConfigs = ConfigurationUtils.readOptionalList(null, null, config, ON_FAILURE_KEY);
List<Processor> onFailureProcessors = readProcessorConfigs(onFailureProcessorConfigs, processorRegistry);
List<Processor> onFailureProcessors = ConfigurationUtils.readProcessorConfigs(onFailureProcessorConfigs, processorRegistry);
if (config.isEmpty() == false) {
throw new ElasticsearchParseException("pipeline [" + id + "] doesn't support one or more provided configuration parameters " + Arrays.toString(config.keySet().toArray()));
}
@ -98,35 +98,5 @@ public final class Pipeline {
return new Pipeline(id, description, compoundProcessor);
}
private List<Processor> readProcessorConfigs(List<Map<String, Map<String, Object>>> processorConfigs, Map<String, Processor.Factory> processorRegistry) throws Exception {
List<Processor> processors = new ArrayList<>();
if (processorConfigs != null) {
for (Map<String, Map<String, Object>> processorConfigWithKey : processorConfigs) {
for (Map.Entry<String, Map<String, Object>> entry : processorConfigWithKey.entrySet()) {
processors.add(readProcessor(processorRegistry, entry.getKey(), entry.getValue()));
}
}
}
return processors;
}
private Processor readProcessor(Map<String, Processor.Factory> processorRegistry, String type, Map<String, Object> config) throws Exception {
Processor.Factory factory = processorRegistry.get(type);
if (factory != null) {
List<Map<String, Map<String, Object>>> onFailureProcessorConfigs = ConfigurationUtils.readOptionalList(null, null, config, ON_FAILURE_KEY);
List<Processor> onFailureProcessors = readProcessorConfigs(onFailureProcessorConfigs, processorRegistry);
Processor processor;
processor = factory.create(config);
if (!config.isEmpty()) {
throw new ElasticsearchParseException("processor [" + type + "] doesn't support one or more provided configuration parameters " + Arrays.toString(config.keySet().toArray()));
}
if (onFailureProcessors.isEmpty()) {
return processor;
}
return new CompoundProcessor(Collections.singletonList(processor), onFailureProcessors);
}
throw new ElasticsearchParseException("No processor type exists with name [" + type + "]");
}
}
}

View File

@ -0,0 +1,105 @@
/*
* 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.ingest.processor;
import org.elasticsearch.ingest.ProcessorsRegistry;
import org.elasticsearch.ingest.core.AbstractProcessor;
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
import org.elasticsearch.ingest.core.ConfigurationUtils;
import org.elasticsearch.ingest.core.IngestDocument;
import org.elasticsearch.ingest.core.Processor;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.ingest.core.ConfigurationUtils.readList;
import static org.elasticsearch.ingest.core.ConfigurationUtils.readStringProperty;
/**
* A processor that for each value in a list executes a one or more processors.
*
* This can be useful in cases to do string operations on json array of strings,
* or remove a field from objects inside a json array.
*/
public final class ForEachProcessor extends AbstractProcessor {
public static final String TYPE = "foreach";
private final String field;
private final List<Processor> processors;
ForEachProcessor(String tag, String field, List<Processor> processors) {
super(tag);
this.field = field;
this.processors = processors;
}
@Override
public void execute(IngestDocument ingestDocument) throws Exception {
List<Object> values = ingestDocument.getFieldValue(field, List.class);
List<Object> newValues = new ArrayList<>(values.size());
for (Object value : values) {
Map<String, Object> innerSource = new HashMap<>();
innerSource.put("_value", value);
for (IngestDocument.MetaData metaData : IngestDocument.MetaData.values()) {
innerSource.put(metaData.getFieldName(), ingestDocument.getSourceAndMetadata().get(metaData.getFieldName()));
}
IngestDocument innerIngestDocument = new IngestDocument(innerSource, ingestDocument.getIngestMetadata());
for (Processor processor : processors) {
processor.execute(innerIngestDocument);
}
newValues.add(innerSource.get("_value"));
}
ingestDocument.setFieldValue(field, newValues);
}
@Override
public String getType() {
return TYPE;
}
String getField() {
return field;
}
List<Processor> getProcessors() {
return processors;
}
public static final class Factory extends AbstractProcessorFactory<ForEachProcessor> {
private final ProcessorsRegistry processorRegistry;
public Factory(ProcessorsRegistry processorRegistry) {
this.processorRegistry = processorRegistry;
}
@Override
protected ForEachProcessor doCreate(String tag, Map<String, Object> config) throws Exception {
String field = readStringProperty(TYPE, tag, config, "field");
List<Map<String, Map<String, Object>>> processorConfigs = readList(TYPE, tag, config, "processors");
List<Processor> processors = ConfigurationUtils.readProcessorConfigs(processorConfigs, processorRegistry);
return new ForEachProcessor(tag, field, Collections.unmodifiableList(processors));
}
}
}

View File

@ -29,6 +29,7 @@ import org.elasticsearch.ingest.processor.AppendProcessor;
import org.elasticsearch.ingest.processor.ConvertProcessor;
import org.elasticsearch.ingest.processor.DateProcessor;
import org.elasticsearch.ingest.processor.FailProcessor;
import org.elasticsearch.ingest.processor.ForEachProcessor;
import org.elasticsearch.ingest.processor.GsubProcessor;
import org.elasticsearch.ingest.processor.JoinProcessor;
import org.elasticsearch.ingest.processor.LowercaseProcessor;
@ -41,7 +42,7 @@ import org.elasticsearch.ingest.processor.UppercaseProcessor;
import org.elasticsearch.monitor.MonitorService;
import org.elasticsearch.node.service.NodeService;
import java.util.function.Function;
import java.util.function.BiFunction;
/**
*
@ -50,7 +51,7 @@ public class NodeModule extends AbstractModule {
private final Node node;
private final MonitorService monitorService;
private final ProcessorsRegistry processorsRegistry;
private final ProcessorsRegistry.Builder processorsRegistryBuilder;
// pkg private so tests can mock
Class<? extends PageCacheRecycler> pageCacheRecyclerImpl = PageCacheRecycler.class;
@ -59,21 +60,22 @@ public class NodeModule extends AbstractModule {
public NodeModule(Node node, MonitorService monitorService) {
this.node = node;
this.monitorService = monitorService;
this.processorsRegistry = new ProcessorsRegistry();
this.processorsRegistryBuilder = new ProcessorsRegistry.Builder();
registerProcessor(DateProcessor.TYPE, (templateService) -> new DateProcessor.Factory());
registerProcessor(SetProcessor.TYPE, SetProcessor.Factory::new);
registerProcessor(AppendProcessor.TYPE, AppendProcessor.Factory::new);
registerProcessor(RenameProcessor.TYPE, (templateService) -> new RenameProcessor.Factory());
registerProcessor(RemoveProcessor.TYPE, RemoveProcessor.Factory::new);
registerProcessor(SplitProcessor.TYPE, (templateService) -> new SplitProcessor.Factory());
registerProcessor(JoinProcessor.TYPE, (templateService) -> new JoinProcessor.Factory());
registerProcessor(UppercaseProcessor.TYPE, (templateService) -> new UppercaseProcessor.Factory());
registerProcessor(LowercaseProcessor.TYPE, (templateService) -> new LowercaseProcessor.Factory());
registerProcessor(TrimProcessor.TYPE, (templateService) -> new TrimProcessor.Factory());
registerProcessor(ConvertProcessor.TYPE, (templateService) -> new ConvertProcessor.Factory());
registerProcessor(GsubProcessor.TYPE, (templateService) -> new GsubProcessor.Factory());
registerProcessor(FailProcessor.TYPE, FailProcessor.Factory::new);
registerProcessor(DateProcessor.TYPE, (templateService, registry) -> new DateProcessor.Factory());
registerProcessor(SetProcessor.TYPE, (templateService, registry) -> new SetProcessor.Factory(templateService));
registerProcessor(AppendProcessor.TYPE, (templateService, registry) -> new AppendProcessor.Factory(templateService));
registerProcessor(RenameProcessor.TYPE, (templateService, registry) -> new RenameProcessor.Factory());
registerProcessor(RemoveProcessor.TYPE, (templateService, registry) -> new RemoveProcessor.Factory(templateService));
registerProcessor(SplitProcessor.TYPE, (templateService, registry) -> new SplitProcessor.Factory());
registerProcessor(JoinProcessor.TYPE, (templateService, registry) -> new JoinProcessor.Factory());
registerProcessor(UppercaseProcessor.TYPE, (templateService, registry) -> new UppercaseProcessor.Factory());
registerProcessor(LowercaseProcessor.TYPE, (templateService, registry) -> new LowercaseProcessor.Factory());
registerProcessor(TrimProcessor.TYPE, (templateService, registry) -> new TrimProcessor.Factory());
registerProcessor(ConvertProcessor.TYPE, (templateService, registry) -> new ConvertProcessor.Factory());
registerProcessor(GsubProcessor.TYPE, (templateService, registry) -> new GsubProcessor.Factory());
registerProcessor(FailProcessor.TYPE, (templateService, registry) -> new FailProcessor.Factory(templateService));
registerProcessor(ForEachProcessor.TYPE, (templateService, registry) -> new ForEachProcessor.Factory(registry));
}
@Override
@ -92,7 +94,7 @@ public class NodeModule extends AbstractModule {
bind(Node.class).toInstance(node);
bind(MonitorService.class).toInstance(monitorService);
bind(NodeService.class).asEagerSingleton();
bind(ProcessorsRegistry.class).toInstance(processorsRegistry);
bind(ProcessorsRegistry.Builder.class).toInstance(processorsRegistryBuilder);
}
/**
@ -105,7 +107,7 @@ public class NodeModule extends AbstractModule {
/**
* Adds a processor factory under a specific type name.
*/
public void registerProcessor(String type, Function<TemplateService, Processor.Factory<?>> processorFactoryProvider) {
processorsRegistry.registerProcessor(type, processorFactoryProvider);
public void registerProcessor(String type, BiFunction<TemplateService, ProcessorsRegistry, Processor.Factory<?>> provider) {
processorsRegistryBuilder.registerProcessor(type, provider);
}
}

View File

@ -237,8 +237,8 @@ public class InternalSettingsPreparer {
}
if (secret) {
return new String(terminal.readSecret("Enter value for [" + key + "]: ", key));
return new String(terminal.readSecret("Enter value for [" + key + "]: "));
}
return terminal.readText("Enter value for [" + key + "]: ", key);
return terminal.readText("Enter value for [" + key + "]: ");
}
}

View File

@ -29,8 +29,8 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.env.Environment;
import org.elasticsearch.http.HttpServer;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
@ -61,6 +61,7 @@ public class NodeService extends AbstractComponent implements Closeable {
private final PluginsService pluginService;
private final CircuitBreakerService circuitBreakerService;
private final IngestService ingestService;
private final SettingsFilter settingsFilter;
private ScriptService scriptService;
@Nullable
@ -73,10 +74,10 @@ public class NodeService extends AbstractComponent implements Closeable {
private final Discovery discovery;
@Inject
public NodeService(Settings settings, Environment environment, ThreadPool threadPool, MonitorService monitorService,
public NodeService(Settings settings, ThreadPool threadPool, MonitorService monitorService,
Discovery discovery, TransportService transportService, IndicesService indicesService,
PluginsService pluginService, CircuitBreakerService circuitBreakerService, Version version,
ProcessorsRegistry processorsRegistry, ClusterService clusterService) {
ProcessorsRegistry.Builder processorsRegistryBuilder, ClusterService clusterService, SettingsFilter settingsFilter) {
super(settings);
this.threadPool = threadPool;
this.monitorService = monitorService;
@ -87,7 +88,8 @@ public class NodeService extends AbstractComponent implements Closeable {
this.version = version;
this.pluginService = pluginService;
this.circuitBreakerService = circuitBreakerService;
this.ingestService = new IngestService(settings, threadPool, processorsRegistry);
this.ingestService = new IngestService(settings, threadPool, processorsRegistryBuilder);
this.settingsFilter = settingsFilter;
clusterService.add(ingestService.getPipelineStore());
}
@ -137,7 +139,7 @@ public class NodeService extends AbstractComponent implements Closeable {
public NodeInfo info(boolean settings, boolean os, boolean process, boolean jvm, boolean threadPool,
boolean transport, boolean http, boolean plugin) {
return new NodeInfo(version, Build.CURRENT, discovery.localNode(), serviceAttributes,
settings ? this.settings : null,
settings ? settingsFilter.filter(this.settings) : null,
os ? monitorService.osService().info() : null,
process ? monitorService.processService().info() : null,
jvm ? monitorService.jvmService().info() : null,

View File

@ -47,7 +47,7 @@ class PluginSecurity {
PermissionCollection permissions = parsePermissions(terminal, file, environment.tmpFile());
List<Permission> requested = Collections.list(permissions.elements());
if (requested.isEmpty()) {
terminal.print(Verbosity.VERBOSE, "plugin has a policy file with no additional permissions");
terminal.println(Verbosity.VERBOSE, "plugin has a policy file with no additional permissions");
return;
}
@ -92,7 +92,7 @@ class PluginSecurity {
terminal.println(Verbosity.NORMAL, "See http://docs.oracle.com/javase/8/docs/technotes/guides/security/permissions.html");
terminal.println(Verbosity.NORMAL, "for descriptions of what these permissions allow and the associated risks.");
if (!batch) {
terminal.println(Verbosity.NORMAL);
terminal.println(Verbosity.NORMAL, "");
String text = terminal.readText("Continue with installation? [y/N]");
if (!text.equalsIgnoreCase("y")) {
throw new RuntimeException("installation aborted by user");

View File

@ -91,7 +91,6 @@ public class RestTermVectorsAction extends BaseRestHandler {
termVectorsRequest.termStatistics(request.paramAsBoolean("term_statistics", termVectorsRequest.termStatistics()));
termVectorsRequest.fieldStatistics(request.paramAsBoolean("fieldStatistics", termVectorsRequest.fieldStatistics()));
termVectorsRequest.fieldStatistics(request.paramAsBoolean("field_statistics", termVectorsRequest.fieldStatistics()));
termVectorsRequest.dfs(request.paramAsBoolean("dfs", termVectorsRequest.dfs()));
}
static public void addFieldStringsFromParameter(TermVectorsRequest termVectorsRequest, String fields) {

View File

@ -226,7 +226,7 @@ public class ExceptionSerializationTests extends ESTestCase {
}
public void testIllegalShardRoutingStateException() throws IOException {
final ShardRouting routing = TestShardRouting.newShardRouting("test", 0, "xyz", "def", false, ShardRoutingState.STARTED, 0);
final ShardRouting routing = TestShardRouting.newShardRouting("test", 0, "xyz", "def", false, ShardRoutingState.STARTED);
final String routingAsString = routing.toString();
IllegalShardRoutingStateException serialize = serialize(
new IllegalShardRoutingStateException(routing, "foo", new NullPointerException()));

View File

@ -158,7 +158,7 @@ public class SyncedFlushUnitTests extends ESTestCase {
Map<ShardRouting, SyncedFlushService.ShardSyncedFlushResponse> shardResponses = new HashMap<>();
for (int copy = 0; copy < replicas + 1; copy++) {
final ShardRouting shardRouting = TestShardRouting.newShardRouting(index, shard, "node_" + shardId + "_" + copy, null,
copy == 0, ShardRoutingState.STARTED, 0);
copy == 0, ShardRoutingState.STARTED);
if (randomInt(5) < 2) {
// shard copy failure
failed++;

View File

@ -93,7 +93,6 @@ public class IndicesShardStoreRequestIT extends ESIntegTestCase {
assertThat(shardStores.values().size(), equalTo(2));
for (ObjectCursor<List<IndicesShardStoresResponse.StoreStatus>> shardStoreStatuses : shardStores.values()) {
for (IndicesShardStoresResponse.StoreStatus storeStatus : shardStoreStatuses.value) {
assertThat(storeStatus.getVersion(), greaterThan(-1L));
assertThat(storeStatus.getAllocationId(), notNullValue());
assertThat(storeStatus.getNode(), notNullValue());
assertThat(storeStatus.getStoreException(), nullValue());
@ -191,10 +190,10 @@ public class IndicesShardStoreRequestIT extends ESIntegTestCase {
for (IndicesShardStoresResponse.StoreStatus status : shardStatus.value) {
if (corruptedShardIDMap.containsKey(shardStatus.key)
&& corruptedShardIDMap.get(shardStatus.key).contains(status.getNode().name())) {
assertThat(status.getVersion(), greaterThanOrEqualTo(0L));
assertThat(status.getLegacyVersion(), greaterThanOrEqualTo(0L));
assertThat(status.getStoreException(), notNullValue());
} else {
assertThat(status.getVersion(), greaterThanOrEqualTo(0L));
assertThat(status.getLegacyVersion(), greaterThanOrEqualTo(0L));
assertNull(status.getStoreException());
}
}

View File

@ -32,6 +32,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.shard.ShardStateMetaData;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.transport.NodeDisconnectedException;
@ -54,8 +55,8 @@ public class IndicesShardStoreResponseTests extends ESTestCase {
DiscoveryNode node2 = new DiscoveryNode("node2", DummyTransportAddress.INSTANCE, Version.CURRENT);
List<IndicesShardStoresResponse.StoreStatus> storeStatusList = new ArrayList<>();
storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node1, 3, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null));
storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node2, 2, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null));
storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node1, 1, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.UNUSED, new IOException("corrupted")));
storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node2, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null));
storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.UNUSED, new IOException("corrupted")));
storeStatuses.put(0, storeStatusList);
storeStatuses.put(1, storeStatusList);
ImmutableOpenIntMap<List<IndicesShardStoresResponse.StoreStatus>> storesMap = storeStatuses.build();
@ -96,10 +97,16 @@ public class IndicesShardStoreResponseTests extends ESTestCase {
for (int i = 0; i < stores.size(); i++) {
HashMap storeInfo = ((HashMap) stores.get(i));
IndicesShardStoresResponse.StoreStatus storeStatus = storeStatusList.get(i);
assertThat(storeInfo.containsKey("version"), equalTo(true));
assertThat(((int) storeInfo.get("version")), equalTo(((int) storeStatus.getVersion())));
assertThat(storeInfo.containsKey("allocation_id"), equalTo(true));
assertThat(((String) storeInfo.get("allocation_id")), equalTo((storeStatus.getAllocationId())));
boolean eitherLegacyVersionOrAllocationIdSet = false;
if (storeInfo.containsKey("legacy_version")) {
assertThat(((int) storeInfo.get("legacy_version")), equalTo(((int) storeStatus.getLegacyVersion())));
eitherLegacyVersionOrAllocationIdSet = true;
}
if (storeInfo.containsKey("allocation_id")) {
assertThat(((String) storeInfo.get("allocation_id")), equalTo((storeStatus.getAllocationId())));
eitherLegacyVersionOrAllocationIdSet = true;
}
assertThat(eitherLegacyVersionOrAllocationIdSet, equalTo(true));
assertThat(storeInfo.containsKey("allocation"), equalTo(true));
assertThat(((String) storeInfo.get("allocation")), equalTo(storeStatus.getAllocationStatus().value()));
assertThat(storeInfo.containsKey(storeStatus.getNode().id()), equalTo(true));
@ -115,11 +122,15 @@ public class IndicesShardStoreResponseTests extends ESTestCase {
public void testStoreStatusOrdering() throws Exception {
DiscoveryNode node1 = new DiscoveryNode("node1", DummyTransportAddress.INSTANCE, Version.CURRENT);
List<IndicesShardStoresResponse.StoreStatus> orderedStoreStatuses = new ArrayList<>();
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 2, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 1, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 1, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 1, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.UNUSED, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 3, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, new IOException("corrupted")));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.UNUSED, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 2, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 1, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 1, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 1, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.UNUSED, null));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, new IOException("corrupted")));
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 3, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, new IOException("corrupted")));
List<IndicesShardStoresResponse.StoreStatus> storeStatuses = new ArrayList<>(orderedStoreStatuses);
Collections.shuffle(storeStatuses, random());

View File

@ -20,7 +20,9 @@
package org.elasticsearch.action.ingest;
import org.elasticsearch.ingest.PipelineStore;
import org.elasticsearch.ingest.ProcessorsRegistry;
import org.elasticsearch.ingest.TestProcessor;
import org.elasticsearch.ingest.TestTemplateService;
import org.elasticsearch.ingest.core.CompoundProcessor;
import org.elasticsearch.ingest.core.IngestDocument;
import org.elasticsearch.ingest.core.Pipeline;
@ -54,11 +56,12 @@ public class SimulatePipelineRequestParsingTests extends ESTestCase {
TestProcessor processor = new TestProcessor(ingestDocument -> {});
CompoundProcessor pipelineCompoundProcessor = new CompoundProcessor(processor);
Pipeline pipeline = new Pipeline(SimulatePipelineRequest.SIMULATED_PIPELINE_ID, null, pipelineCompoundProcessor);
Map<String, Processor.Factory> processorRegistry = new HashMap<>();
processorRegistry.put("mock_processor", mock(Processor.Factory.class));
ProcessorsRegistry.Builder processorRegistryBuilder = new ProcessorsRegistry.Builder();
processorRegistryBuilder.registerProcessor("mock_processor", ((templateService, registry) -> mock(Processor.Factory.class)));
ProcessorsRegistry processorRegistry = processorRegistryBuilder.build(TestTemplateService.instance());
store = mock(PipelineStore.class);
when(store.get(SimulatePipelineRequest.SIMULATED_PIPELINE_ID)).thenReturn(pipeline);
when(store.getProcessorFactoryRegistry()).thenReturn(processorRegistry);
when(store.getProcessorRegistry()).thenReturn(processorRegistry);
}
public void testParseUsingPipelineStore() throws Exception {

View File

@ -208,7 +208,7 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
int numberOfShards = randomIntBetween(1, 10);
for (int j = 0; j < numberOfShards; j++) {
final ShardId shardId = new ShardId(index, "_na_", ++shardIndex);
ShardRouting shard = TestShardRouting.newShardRouting(index, shardId.getId(), node.id(), true, ShardRoutingState.STARTED, 1);
ShardRouting shard = TestShardRouting.newShardRouting(index, shardId.getId(), node.id(), true, ShardRoutingState.STARTED);
IndexShardRoutingTable.Builder indexShard = new IndexShardRoutingTable.Builder(shardId);
indexShard.addShard(shard);
indexRoutingTable.addIndexShard(indexShard.build());

View File

@ -109,7 +109,7 @@ public class ClusterStateCreationUtils {
} else {
unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null);
}
indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, 0, primaryNode, relocatingNode, null, true, primaryState, 0, unassignedInfo));
indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, 0, primaryNode, relocatingNode, null, true, primaryState, unassignedInfo));
for (ShardRoutingState replicaState : replicaStates) {
String replicaNode = null;
@ -125,7 +125,7 @@ public class ClusterStateCreationUtils {
unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null);
}
indexShardRoutingBuilder.addShard(
TestShardRouting.newShardRouting(index, shardId.id(), replicaNode, relocatingNode, null, false, replicaState, 0, unassignedInfo));
TestShardRouting.newShardRouting(index, shardId.id(), replicaNode, relocatingNode, null, false, replicaState, unassignedInfo));
}
ClusterState.Builder state = ClusterState.builder(new ClusterName("test"));
@ -161,8 +161,8 @@ public class ClusterStateCreationUtils {
routing.addAsNew(indexMetaData);
final ShardId shardId = new ShardId(index, "_na_", i);
IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId);
indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(0).id(), null, null, true, ShardRoutingState.STARTED, 0, null));
indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(1).id(), null, null, false, ShardRoutingState.STARTED, 0, null));
indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(0).id(), null, null, true, ShardRoutingState.STARTED, null));
indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(1).id(), null, null, false, ShardRoutingState.STARTED, null));
indexRoutingTableBuilder.addIndexShard(indexShardRoutingBuilder.build());
}
state.routingTable(RoutingTable.builder().add(indexRoutingTableBuilder.build()).build());

View File

@ -134,8 +134,7 @@ public class GetTermVectorsIT extends AbstractTermVectorsTestCase {
ActionFuture<TermVectorsResponse> termVectors = client().termVectors(new TermVectorsRequest(indexOrAlias(), "type1", "0")
.selectedFields(randomBoolean() ? new String[]{"existingfield"} : null)
.termStatistics(true)
.fieldStatistics(true)
.dfs(true));
.fieldStatistics(true));
// lets see if the null term vectors are caught...
TermVectorsResponse actionGet = termVectors.actionGet();
@ -966,95 +965,6 @@ public class GetTermVectorsIT extends AbstractTermVectorsTestCase {
return randomBoolean() ? "test" : "alias";
}
public void testDfs() throws ExecutionException, InterruptedException, IOException {
logger.info("Setting up the index ...");
Settings.Builder settings = settingsBuilder()
.put(indexSettings())
.put("index.analysis.analyzer", "standard")
.put("index.number_of_shards", randomIntBetween(2, 10)); // we need at least 2 shards
assertAcked(prepareCreate("test")
.setSettings(settings)
.addMapping("type1", "text", "type=string"));
ensureGreen();
int numDocs = scaledRandomIntBetween(25, 100);
logger.info("Indexing {} documents...", numDocs);
List<IndexRequestBuilder> builders = new ArrayList<>();
for (int i = 0; i < numDocs; i++) {
builders.add(client().prepareIndex("test", "type1", i + "").setSource("text", "cat"));
}
indexRandom(true, builders);
XContentBuilder expectedStats = jsonBuilder()
.startObject()
.startObject("text")
.startObject("field_statistics")
.field("sum_doc_freq", numDocs)
.field("doc_count", numDocs)
.field("sum_ttf", numDocs)
.endObject()
.startObject("terms")
.startObject("cat")
.field("doc_freq", numDocs)
.field("ttf", numDocs)
.endObject()
.endObject()
.endObject()
.endObject();
logger.info("Without dfs 'cat' should appear strictly less than {} times.", numDocs);
TermVectorsResponse response = client().prepareTermVectors("test", "type1", randomIntBetween(0, numDocs - 1) + "")
.setSelectedFields("text")
.setFieldStatistics(true)
.setTermStatistics(true)
.get();
checkStats(response.getFields(), expectedStats, false);
logger.info("With dfs 'cat' should appear exactly {} times.", numDocs);
response = client().prepareTermVectors("test", "type1", randomIntBetween(0, numDocs - 1) + "")
.setSelectedFields("text")
.setFieldStatistics(true)
.setTermStatistics(true)
.setDfs(true)
.get();
checkStats(response.getFields(), expectedStats, true);
}
private void checkStats(Fields fields, XContentBuilder xContentBuilder, boolean isEqual) throws IOException {
Map<String, Object> stats = JsonXContent.jsonXContent.createParser(xContentBuilder.bytes()).map();
assertThat("number of fields expected:", fields.size(), equalTo(stats.size()));
for (String fieldName : fields) {
logger.info("Checking field statistics for field: {}", fieldName);
Terms terms = fields.terms(fieldName);
Map<String, Integer> fieldStatistics = getFieldStatistics(stats, fieldName);
String msg = "field: " + fieldName + " ";
assertThat(msg + "sum_doc_freq:",
(int) terms.getSumDocFreq(),
equalOrLessThanTo(fieldStatistics.get("sum_doc_freq"), isEqual));
assertThat(msg + "doc_count:",
terms.getDocCount(),
equalOrLessThanTo(fieldStatistics.get("doc_count"), isEqual));
assertThat(msg + "sum_ttf:",
(int) terms.getSumTotalTermFreq(),
equalOrLessThanTo(fieldStatistics.get("sum_ttf"), isEqual));
final TermsEnum termsEnum = terms.iterator();
BytesRef text;
while((text = termsEnum.next()) != null) {
String term = text.utf8ToString();
logger.info("Checking term statistics for term: ({}, {})", fieldName, term);
Map<String, Integer> termStatistics = getTermStatistics(stats, fieldName, term);
msg = "term: (" + fieldName + "," + term + ") ";
assertThat(msg + "doc_freq:",
termsEnum.docFreq(),
equalOrLessThanTo(termStatistics.get("doc_freq"), isEqual));
assertThat(msg + "ttf:",
(int) termsEnum.totalTermFreq(),
equalOrLessThanTo(termStatistics.get("ttf"), isEqual));
}
}
}
private Map<String, Integer> getFieldStatistics(Map<String, Object> stats, String fieldName) throws IOException {
return (Map<String, Integer>) ((Map<String, Object>) stats.get(fieldName)).get("field_statistics");
}

View File

@ -237,7 +237,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
}
indexShard.addShard(
TestShardRouting.newShardRouting(index, i, randomFrom(nodeIds), null, null, j == 0,
ShardRoutingState.fromValue((byte) randomIntBetween(2, 4)), 1, unassignedInfo));
ShardRoutingState.fromValue((byte) randomIntBetween(2, 4)), unassignedInfo));
}
builder.addIndexShard(indexShard.build());
}

View File

@ -202,7 +202,7 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
for (ShardStateAction.ShardRoutingEntry existingShard : existingShards) {
ShardRouting sr = existingShard.getShardRouting();
ShardRouting nonExistentShardRouting =
TestShardRouting.newShardRouting(sr.index(), sr.id(), sr.currentNodeId(), sr.relocatingNodeId(), sr.restoreSource(), sr.primary(), sr.state(), sr.version());
TestShardRouting.newShardRouting(sr.index(), sr.id(), sr.currentNodeId(), sr.relocatingNodeId(), sr.restoreSource(), sr.primary(), sr.state());
shardsWithMismatchedAllocationIds.add(new ShardStateAction.ShardRoutingEntry(nonExistentShardRouting, nonExistentShardRouting, existingShard.message, existingShard.failure));
}
@ -213,7 +213,7 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
}
private ShardRouting nonExistentShardRouting(Index index, List<String> nodeIds, boolean primary) {
return TestShardRouting.newShardRouting(index, 0, randomFrom(nodeIds), primary, randomFrom(ShardRoutingState.INITIALIZING, ShardRoutingState.RELOCATING, ShardRoutingState.STARTED), randomIntBetween(1, 8));
return TestShardRouting.newShardRouting(index, 0, randomFrom(nodeIds), primary, randomFrom(ShardRoutingState.INITIALIZING, ShardRoutingState.RELOCATING, ShardRoutingState.STARTED));
}
private static void assertTasksSuccessful(
@ -306,7 +306,7 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
return randomSubsetOf(1, shards.toArray(new ShardRouting[0])).get(0);
} else {
return
TestShardRouting.newShardRouting(shardRouting.index(), shardRouting.id(), DiscoveryService.generateNodeId(Settings.EMPTY), randomBoolean(), randomFrom(ShardRoutingState.values()), shardRouting.version());
TestShardRouting.newShardRouting(shardRouting.index(), shardRouting.id(), DiscoveryService.generateNodeId(Settings.EMPTY), randomBoolean(), randomFrom(ShardRoutingState.values()));
}
}

View File

@ -332,7 +332,7 @@ public class ShardStateActionTests extends ESTestCase {
AtomicReference<Throwable> failure = new AtomicReference<>();
CountDownLatch latch = new CountDownLatch(1);
ShardRouting sourceFailedShard = TestShardRouting.newShardRouting(failedShard.index(), failedShard.id(), nodeId, randomBoolean(), randomFrom(ShardRoutingState.values()), failedShard.version());
ShardRouting sourceFailedShard = TestShardRouting.newShardRouting(failedShard.index(), failedShard.id(), nodeId, randomBoolean(), randomFrom(ShardRoutingState.values()));
shardStateAction.shardFailed(failedShard, sourceFailedShard, "test", getSimulatedFailure(), new ShardStateAction.Listener() {
@Override
public void onSuccess() {

View File

@ -46,11 +46,11 @@ class RoutingTableGenerator {
switch (state) {
case STARTED:
return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), null, null, primary, ShardRoutingState.STARTED, 1);
return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), null, null, primary, ShardRoutingState.STARTED);
case INITIALIZING:
return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), null, null, primary, ShardRoutingState.INITIALIZING, 1);
return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), null, null, primary, ShardRoutingState.INITIALIZING);
case RELOCATING:
return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), "node_" + Integer.toString(node_id++), null, primary, ShardRoutingState.RELOCATING, 1);
return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), "node_" + Integer.toString(node_id++), null, primary, ShardRoutingState.RELOCATING);
default:
throw new ElasticsearchException("Unknown state: " + state.name());
}

View File

@ -53,6 +53,6 @@ public class ShardRoutingHelper {
}
public static ShardRouting newWithRestoreSource(ShardRouting routing, RestoreSource restoreSource) {
return new ShardRouting(routing.index(), routing.shardId().id(), routing.currentNodeId(), routing.relocatingNodeId(), restoreSource, routing.primary(), routing.state(), routing.version(), routing.unassignedInfo(), routing.allocationId(), true, routing.getExpectedShardSize());
return new ShardRouting(routing.index(), routing.shardId().id(), routing.currentNodeId(), routing.relocatingNodeId(), restoreSource, routing.primary(), routing.state(), routing.unassignedInfo(), routing.allocationId(), true, routing.getExpectedShardSize());
}
}

View File

@ -33,7 +33,7 @@ import java.io.IOException;
public class ShardRoutingTests extends ESTestCase {
public void testFrozenAfterRead() throws IOException {
ShardRouting routing = TestShardRouting.newShardRouting("foo", 1, "node_1", null, null, false, ShardRoutingState.INITIALIZING, 1);
ShardRouting routing = TestShardRouting.newShardRouting("foo", 1, "node_1", null, null, false, ShardRoutingState.INITIALIZING);
routing.moveToPrimary();
assertTrue(routing.primary());
routing.moveFromPrimary();
@ -50,10 +50,10 @@ public class ShardRoutingTests extends ESTestCase {
}
public void testIsSameAllocation() {
ShardRouting unassignedShard0 = TestShardRouting.newShardRouting("test", 0, null, false, ShardRoutingState.UNASSIGNED, 1);
ShardRouting unassignedShard1 = TestShardRouting.newShardRouting("test", 1, null, false, ShardRoutingState.UNASSIGNED, 1);
ShardRouting initializingShard0 = TestShardRouting.newShardRouting("test", 0, "1", randomBoolean(), ShardRoutingState.INITIALIZING, 1);
ShardRouting initializingShard1 = TestShardRouting.newShardRouting("test", 1, "1", randomBoolean(), ShardRoutingState.INITIALIZING, 1);
ShardRouting unassignedShard0 = TestShardRouting.newShardRouting("test", 0, null, false, ShardRoutingState.UNASSIGNED);
ShardRouting unassignedShard1 = TestShardRouting.newShardRouting("test", 1, null, false, ShardRoutingState.UNASSIGNED);
ShardRouting initializingShard0 = TestShardRouting.newShardRouting("test", 0, "1", randomBoolean(), ShardRoutingState.INITIALIZING);
ShardRouting initializingShard1 = TestShardRouting.newShardRouting("test", 1, "1", randomBoolean(), ShardRoutingState.INITIALIZING);
ShardRouting startedShard0 = new ShardRouting(initializingShard0);
startedShard0.moveToStarted();
ShardRouting startedShard1 = new ShardRouting(initializingShard1);
@ -91,13 +91,13 @@ public class ShardRoutingTests extends ESTestCase {
private ShardRouting randomShardRouting(String index, int shard) {
ShardRoutingState state = randomFrom(ShardRoutingState.values());
return TestShardRouting.newShardRouting(index, shard, state == ShardRoutingState.UNASSIGNED ? null : "1", state != ShardRoutingState.UNASSIGNED && randomBoolean(), state, randomInt(5));
return TestShardRouting.newShardRouting(index, shard, state == ShardRoutingState.UNASSIGNED ? null : "1", state != ShardRoutingState.UNASSIGNED && randomBoolean(), state);
}
public void testIsSourceTargetRelocation() {
ShardRouting unassignedShard0 = TestShardRouting.newShardRouting("test", 0, null, false, ShardRoutingState.UNASSIGNED, 1);
ShardRouting initializingShard0 = TestShardRouting.newShardRouting("test", 0, "node1", randomBoolean(), ShardRoutingState.INITIALIZING, 1);
ShardRouting initializingShard1 = TestShardRouting.newShardRouting("test", 1, "node1", randomBoolean(), ShardRoutingState.INITIALIZING, 1);
ShardRouting unassignedShard0 = TestShardRouting.newShardRouting("test", 0, null, false, ShardRoutingState.UNASSIGNED);
ShardRouting initializingShard0 = TestShardRouting.newShardRouting("test", 0, "node1", randomBoolean(), ShardRoutingState.INITIALIZING);
ShardRouting initializingShard1 = TestShardRouting.newShardRouting("test", 1, "node1", randomBoolean(), ShardRoutingState.INITIALIZING);
ShardRouting startedShard0 = new ShardRouting(initializingShard0);
assertFalse(startedShard0.isRelocationTarget());
startedShard0.moveToStarted();
@ -151,7 +151,7 @@ public class ShardRoutingTests extends ESTestCase {
ShardRouting otherRouting = new ShardRouting(routing);
assertTrue("expected equality\nthis " + routing + ",\nother " + otherRouting, routing.equalsIgnoringMetaData(otherRouting));
otherRouting = new ShardRouting(routing, 1);
otherRouting = new ShardRouting(routing);
assertTrue("expected equality\nthis " + routing + ",\nother " + otherRouting, routing.equalsIgnoringMetaData(otherRouting));
@ -162,35 +162,35 @@ public class ShardRoutingTests extends ESTestCase {
case 0:
// change index
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName() + "a", otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.unassignedInfo());
break;
case 1:
// change shard id
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName(), otherRouting.id() + 1, otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.unassignedInfo());
break;
case 2:
// change current node
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName(), otherRouting.id(), otherRouting.currentNodeId() == null ? "1" : otherRouting.currentNodeId() + "_1", otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.unassignedInfo());
break;
case 3:
// change relocating node
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName(), otherRouting.id(), otherRouting.currentNodeId(),
otherRouting.relocatingNodeId() == null ? "1" : otherRouting.relocatingNodeId() + "_1",
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.unassignedInfo());
break;
case 4:
// change restore source
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource() == null ? new RestoreSource(new SnapshotId("test", "s1"), Version.CURRENT, "test") :
new RestoreSource(otherRouting.restoreSource().snapshotId(), Version.CURRENT, otherRouting.index() + "_1"),
otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
otherRouting.primary(), otherRouting.state(), otherRouting.unassignedInfo());
break;
case 5:
// change primary flag
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary() == false, otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
otherRouting.restoreSource(), otherRouting.primary() == false, otherRouting.state(), otherRouting.unassignedInfo());
break;
case 6:
// change state
@ -205,19 +205,14 @@ public class ShardRoutingTests extends ESTestCase {
}
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary(), newState, otherRouting.version(), unassignedInfo);
otherRouting.restoreSource(), otherRouting.primary(), newState, unassignedInfo);
break;
}
if (randomBoolean()) {
// change version
otherRouting = new ShardRouting(otherRouting, otherRouting.version() + 1);
}
if (randomBoolean()) {
// change unassigned info
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(),
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(),
otherRouting.unassignedInfo() == null ? new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "test") :
new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, otherRouting.unassignedInfo().getMessage() + "_1"));
}
@ -237,7 +232,6 @@ public class ShardRoutingTests extends ESTestCase {
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
for (ShardRouting routing : clusterState.routingTable().allShards()) {
long version = routing.version();
assertTrue(routing.isFrozen());
try {
routing.moveToPrimary();
@ -290,7 +284,6 @@ public class ShardRoutingTests extends ESTestCase {
} catch (IllegalStateException ex) {
// expected
}
assertEquals(version, routing.version());
}
}

View File

@ -184,7 +184,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
* The unassigned meta is kept when a shard goes to INITIALIZING, but cleared when it moves to STARTED.
*/
public void testStateTransitionMetaHandling() {
ShardRouting shard = TestShardRouting.newShardRouting("test", 1, null, null, null, true, ShardRoutingState.UNASSIGNED, 1, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null));
ShardRouting shard = TestShardRouting.newShardRouting("test", 1, null, null, null, true, ShardRoutingState.UNASSIGNED, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null));
ShardRouting mutable = new ShardRouting(shard);
assertThat(mutable.unassignedInfo(), notNullValue());
mutable.initialize("test_node", -1);

View File

@ -82,7 +82,7 @@ public abstract class CatAllocationTestCase extends ESAllocationTestCase {
ShardRoutingState state = ShardRoutingState.valueOf(matcher.group(4));
String ip = matcher.group(5);
nodes.add(ip);
ShardRouting routing = TestShardRouting.newShardRouting(index, shard, ip, null, null, primary, state, 1);
ShardRouting routing = TestShardRouting.newShardRouting(index, shard, ip, null, null, primary, state);
idx.add(routing);
logger.debug("Add routing {}", routing);
} else {

View File

@ -311,14 +311,14 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
RoutingTable routingTable = RoutingTable.builder()
.add(IndexRoutingTable.builder(shard1.getIndex())
.addIndexShard(new IndexShardRoutingTable.Builder(shard1)
.addShard(TestShardRouting.newShardRouting(shard1.getIndexName(), shard1.getId(), newNode.id(), true, ShardRoutingState.STARTED, 10))
.addShard(TestShardRouting.newShardRouting(shard1.getIndexName(), shard1.getId(), oldNode1.id(), false, ShardRoutingState.STARTED, 10))
.addShard(TestShardRouting.newShardRouting(shard1.getIndexName(), shard1.getId(), newNode.id(), true, ShardRoutingState.STARTED))
.addShard(TestShardRouting.newShardRouting(shard1.getIndexName(), shard1.getId(), oldNode1.id(), false, ShardRoutingState.STARTED))
.build())
)
.add(IndexRoutingTable.builder(shard2.getIndex())
.addIndexShard(new IndexShardRoutingTable.Builder(shard2)
.addShard(TestShardRouting.newShardRouting(shard2.getIndexName(), shard2.getId(), newNode.id(), true, ShardRoutingState.STARTED, 10))
.addShard(TestShardRouting.newShardRouting(shard2.getIndexName(), shard2.getId(), oldNode1.id(), false, ShardRoutingState.STARTED, 10))
.addShard(TestShardRouting.newShardRouting(shard2.getIndexName(), shard2.getId(), newNode.id(), true, ShardRoutingState.STARTED))
.addShard(TestShardRouting.newShardRouting(shard2.getIndexName(), shard2.getId(), oldNode1.id(), false, ShardRoutingState.STARTED))
.build())
)
.build();

View File

@ -64,14 +64,12 @@ public class ShardVersioningTests extends ESAllocationTestCase {
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test1").shard(i).primaryShard().version(), equalTo(1L));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).primaryShard().version(), equalTo(1L));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
@ -84,17 +82,13 @@ public class ShardVersioningTests extends ESAllocationTestCase {
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).primaryShard().version(), equalTo(2L));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).version(), equalTo(2L));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).primaryShard().version(), equalTo(1L));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).version(), equalTo(1L));
}
}
}
}

View File

@ -53,9 +53,9 @@ public class StartedShardsRoutingTests extends ESAllocationTestCase {
.nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2")))
.metaData(MetaData.builder().put(indexMetaData, false));
final ShardRouting initShard = TestShardRouting.newShardRouting(index, 0, "node1", true, ShardRoutingState.INITIALIZING, 1);
final ShardRouting startedShard = TestShardRouting.newShardRouting(index, 1, "node2", true, ShardRoutingState.STARTED, 1);
final ShardRouting relocatingShard = TestShardRouting.newShardRouting(index, 2, "node1", "node2", true, ShardRoutingState.RELOCATING, 1);
final ShardRouting initShard = TestShardRouting.newShardRouting(index, 0, "node1", true, ShardRoutingState.INITIALIZING);
final ShardRouting startedShard = TestShardRouting.newShardRouting(index, 1, "node2", true, ShardRoutingState.STARTED);
final ShardRouting relocatingShard = TestShardRouting.newShardRouting(index, 2, "node1", "node2", true, ShardRoutingState.RELOCATING);
stateBuilder.routingTable(RoutingTable.builder().add(IndexRoutingTable.builder(index)
.addIndexShard(new IndexShardRoutingTable.Builder(initShard.shardId()).addShard(initShard).build())
.addIndexShard(new IndexShardRoutingTable.Builder(startedShard.shardId()).addShard(startedShard).build())
@ -67,7 +67,7 @@ public class StartedShardsRoutingTests extends ESAllocationTestCase {
RoutingAllocation.Result result = allocation.applyStartedShards(state, Arrays.asList(
TestShardRouting.newShardRouting(initShard.index(), initShard.id(), initShard.currentNodeId(), initShard.relocatingNodeId(), initShard.primary(),
ShardRoutingState.INITIALIZING, initShard.allocationId(), randomInt())), false);
ShardRoutingState.INITIALIZING, initShard.allocationId())), false);
assertTrue("failed to start " + initShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed());
assertTrue(initShard + "isn't started \ncurrent routing table:" + result.routingTable().prettyPrint(),
result.routingTable().index("test").shard(initShard.id()).allShardsStarted());
@ -77,13 +77,12 @@ public class StartedShardsRoutingTests extends ESAllocationTestCase {
result = allocation.applyStartedShards(state, Arrays.asList(
TestShardRouting.newShardRouting(initShard.index(), initShard.id(), initShard.currentNodeId(), initShard.relocatingNodeId(), initShard.primary(),
ShardRoutingState.INITIALIZING, 1)), false);
ShardRoutingState.INITIALIZING)), false);
assertFalse("wrong allocation id flag shouldn't start shard " + initShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed());
result = allocation.applyStartedShards(state, Arrays.asList(
TestShardRouting.newShardRouting(initShard.index(), initShard.id(), "some_node", initShard.currentNodeId(), initShard.primary(),
ShardRoutingState.INITIALIZING, AllocationId.newTargetRelocation(AllocationId.newRelocation(initShard.allocationId()))
, 1)), false);
ShardRoutingState.INITIALIZING, AllocationId.newTargetRelocation(AllocationId.newRelocation(initShard.allocationId())))), false);
assertFalse("relocating shard from node shouldn't start shard " + initShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed());
@ -92,14 +91,14 @@ public class StartedShardsRoutingTests extends ESAllocationTestCase {
result = allocation.applyStartedShards(state, Arrays.asList(
TestShardRouting.newShardRouting(startedShard.index(), startedShard.id(), startedShard.currentNodeId(), startedShard.relocatingNodeId(), startedShard.primary(),
ShardRoutingState.INITIALIZING, startedShard.allocationId(), 1)), false);
ShardRoutingState.INITIALIZING, startedShard.allocationId())), false);
assertFalse("duplicate starting of the same shard should be ignored \ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed());
logger.info("--> testing starting of relocating shards");
final AllocationId targetAllocationId = AllocationId.newTargetRelocation(relocatingShard.allocationId());
result = allocation.applyStartedShards(state, Arrays.asList(
TestShardRouting.newShardRouting(relocatingShard.index(), relocatingShard.id(), relocatingShard.relocatingNodeId(), relocatingShard.currentNodeId(), relocatingShard.primary(),
ShardRoutingState.INITIALIZING, targetAllocationId, randomInt())), false);
ShardRoutingState.INITIALIZING, targetAllocationId)), false);
assertTrue("failed to start " + relocatingShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed());
ShardRouting shardRouting = result.routingTable().index("test").shard(relocatingShard.id()).getShards().get(0);
@ -111,12 +110,12 @@ public class StartedShardsRoutingTests extends ESAllocationTestCase {
result = allocation.applyStartedShards(state, Arrays.asList(
TestShardRouting.newShardRouting(relocatingShard.index(), relocatingShard.id(), relocatingShard.relocatingNodeId(), relocatingShard.currentNodeId(), relocatingShard.primary(),
ShardRoutingState.INITIALIZING, relocatingShard.version())));
ShardRoutingState.INITIALIZING)));
assertFalse("wrong allocation id shouldn't start shard" + relocatingShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed());
result = allocation.applyStartedShards(state, Arrays.asList(
TestShardRouting.newShardRouting(relocatingShard.index(), relocatingShard.id(), relocatingShard.relocatingNodeId(), relocatingShard.currentNodeId(), relocatingShard.primary(),
ShardRoutingState.INITIALIZING, relocatingShard.allocationId(), randomInt())), false);
ShardRoutingState.INITIALIZING, relocatingShard.allocationId())), false);
assertFalse("wrong allocation id shouldn't start shard even if relocatingId==shard.id" + relocatingShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed());
}

View File

@ -843,8 +843,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.build();
// Two shards consuming each 80% of disk space while 70% is allowed, so shard 0 isn't allowed here
ShardRouting firstRouting = TestShardRouting.newShardRouting("test", 0, "node1", null, null, true, ShardRoutingState.STARTED, 1);
ShardRouting secondRouting = TestShardRouting.newShardRouting("test", 1, "node1", null, null, true, ShardRoutingState.STARTED, 1);
ShardRouting firstRouting = TestShardRouting.newShardRouting("test", 0, "node1", null, null, true, ShardRoutingState.STARTED);
ShardRouting secondRouting = TestShardRouting.newShardRouting("test", 1, "node1", null, null, true, ShardRoutingState.STARTED);
RoutingNode firstRoutingNode = new RoutingNode("node1", discoveryNode1, Arrays.asList(firstRouting, secondRouting));
RoutingTable.Builder builder = RoutingTable.builder().add(
IndexRoutingTable.builder(firstRouting.index())
@ -863,8 +863,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
assertThat(decision.type(), equalTo(Decision.Type.NO));
// Two shards consuming each 80% of disk space while 70% is allowed, but one is relocating, so shard 0 can stay
firstRouting = TestShardRouting.newShardRouting("test", 0, "node1", null, null, true, ShardRoutingState.STARTED, 1);
secondRouting = TestShardRouting.newShardRouting("test", 1, "node1", "node2", null, true, ShardRoutingState.RELOCATING, 1);
firstRouting = TestShardRouting.newShardRouting("test", 0, "node1", null, null, true, ShardRoutingState.STARTED);
secondRouting = TestShardRouting.newShardRouting("test", 1, "node1", "node2", null, true, ShardRoutingState.RELOCATING);
firstRoutingNode = new RoutingNode("node1", discoveryNode1, Arrays.asList(firstRouting, secondRouting));
builder = RoutingTable.builder().add(
IndexRoutingTable.builder(firstRouting.index())
@ -961,8 +961,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.build();
// Two shards consumes 80% of disk space in data node, but we have only one data node, shards should remain.
ShardRouting firstRouting = TestShardRouting.newShardRouting("test", 0, "node2", null, null, true, ShardRoutingState.STARTED, 1);
ShardRouting secondRouting = TestShardRouting.newShardRouting("test", 1, "node2", null, null, true, ShardRoutingState.STARTED, 1);
ShardRouting firstRouting = TestShardRouting.newShardRouting("test", 0, "node2", null, null, true, ShardRoutingState.STARTED);
ShardRouting secondRouting = TestShardRouting.newShardRouting("test", 1, "node2", null, null, true, ShardRoutingState.STARTED);
RoutingNode firstRoutingNode = new RoutingNode("node2", discoveryNode2, Arrays.asList(firstRouting, secondRouting));
RoutingTable.Builder builder = RoutingTable.builder().add(
@ -1019,8 +1019,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
ClusterState updateClusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.put(discoveryNode3)).build();
firstRouting = TestShardRouting.newShardRouting("test", 0, "node2", null, null, true, ShardRoutingState.STARTED, 1);
secondRouting = TestShardRouting.newShardRouting("test", 1, "node2", "node3", null, true, ShardRoutingState.RELOCATING, 1);
firstRouting = TestShardRouting.newShardRouting("test", 0, "node2", null, null, true, ShardRoutingState.STARTED);
secondRouting = TestShardRouting.newShardRouting("test", 1, "node2", "node3", null, true, ShardRoutingState.RELOCATING);
firstRoutingNode = new RoutingNode("node2", discoveryNode2, Arrays.asList(firstRouting, secondRouting));
builder = RoutingTable.builder().add(
IndexRoutingTable.builder(firstRouting.index())

View File

@ -19,6 +19,8 @@
package org.elasticsearch.cluster.settings;
import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Inject;
@ -64,8 +66,16 @@ public class SettingsFilteringIT extends ESIntegTestCase {
return "Settings Filtering Plugin";
}
@Override
public Settings additionalSettings() {
return Settings.builder().put("some.node.setting", true).put("some.other.node.setting", true).build();
}
public void onModule(SettingsModule module) {
module.registerSetting(Setting.groupSetting("index.filter_test.", false, Setting.Scope.INDEX));
module.registerSetting(Setting.boolSetting("some.node.setting", false, false, Setting.Scope.CLUSTER));
module.registerSetting(Setting.boolSetting("some.other.node.setting", false, false, Setting.Scope.CLUSTER));
module.registerSettingsFilter("some.node.setting");
module.registerSettingsFilter("index.filter_test.foo");
module.registerSettingsFilter("index.filter_test.bar*");
}
@ -88,4 +98,15 @@ public class SettingsFilteringIT extends ESIntegTestCase {
assertThat(settings.get("index.filter_test.notbar"), equalTo("test"));
assertThat(settings.get("index.filter_test.notfoo"), equalTo("test"));
}
public void testNodeInfoIsFiltered() {
NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().clear().setSettings(true).get();
for(NodeInfo info : nodeInfos.getNodes()) {
Settings settings = info.getSettings();
assertNotNull(settings);
assertNull(settings.get("some.node.setting"));
assertTrue(settings.getAsBoolean("some.other.node.setting", false));
assertEquals(settings.get("node.name"), info.getNode().getName());
}
}
}

View File

@ -22,9 +22,6 @@ package org.elasticsearch.common.cli;
import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.hasSize;
/**
*
*/
public class TerminalTests extends CliToolTestCase {
public void testVerbosity() throws Exception {
CaptureOutputTerminal terminal = new CaptureOutputTerminal(Terminal.Verbosity.SILENT);
@ -49,14 +46,14 @@ public class TerminalTests extends CliToolTestCase {
}
private void assertPrinted(CaptureOutputTerminal logTerminal, Terminal.Verbosity verbosity, String text) {
logTerminal.print(verbosity, text);
assertThat(logTerminal.getTerminalOutput(), hasSize(1));
assertThat(logTerminal.getTerminalOutput(), hasItem(text));
logTerminal.println(verbosity, text);
assertEquals(1, logTerminal.getTerminalOutput().size());
assertTrue(logTerminal.getTerminalOutput().get(0).contains(text));
logTerminal.terminalOutput.clear();
}
private void assertNotPrinted(CaptureOutputTerminal logTerminal, Terminal.Verbosity verbosity, String text) {
logTerminal.print(verbosity, text);
logTerminal.println(verbosity, text);
assertThat(logTerminal.getTerminalOutput(), hasSize(0));
}
}

View File

@ -19,19 +19,13 @@
package org.elasticsearch.discovery;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.Lifecycle;
import org.elasticsearch.common.component.LifecycleListener;
import org.elasticsearch.common.inject.ModuleTestCase;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.local.LocalDiscovery;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
import org.elasticsearch.node.Node;
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.test.NoopDiscovery;
/**
*/
@ -74,82 +68,9 @@ public class DiscoveryModuleTests extends ModuleTestCase {
Settings settings = Settings.builder().put(Node.NODE_LOCAL_SETTING.getKey(), local).
put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "custom").build();
DiscoveryModule module = new DiscoveryModule(settings);
module.addDiscoveryType("custom", DummyDisco.class);
assertBinding(module, Discovery.class, DummyDisco.class);
module.addDiscoveryType("custom", NoopDiscovery.class);
assertBinding(module, Discovery.class, NoopDiscovery.class);
}
public static class DummyDisco implements Discovery {
@Override
public DiscoveryNode localNode() {
return null;
}
@Override
public void addListener(InitialStateDiscoveryListener listener) {
}
@Override
public void removeListener(InitialStateDiscoveryListener listener) {
}
@Override
public String nodeDescription() {
return null;
}
@Override
public void setNodeService(@Nullable NodeService nodeService) {
}
@Override
public void setRoutingService(RoutingService routingService) {
}
@Override
public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) {
}
@Override
public DiscoveryStats stats() {
return null;
}
@Override
public Lifecycle.State lifecycleState() {
return null;
}
@Override
public void addLifecycleListener(LifecycleListener listener) {
}
@Override
public void removeLifecycleListener(LifecycleListener listener) {
}
@Override
public Discovery start() {
return null;
}
@Override
public Discovery stop() {
return null;
}
@Override
public void close() {
}
}
}

View File

@ -333,7 +333,7 @@ public class GatewayIndexStateIT extends ESIntegTestCase {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(true));
logger.info("--> restarting the nodes");
final Gateway gateway1 = internalCluster().getInstance(Gateway.class, node_1);
final Gateway gateway1 = internalCluster().getInstance(GatewayService.class, node_1).getGateway();
internalCluster().fullRestart(new RestartCallback() {
@Override
public Settings onNodeStopped(String nodeName) throws Exception {

View File

@ -1,48 +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.gateway;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.ModuleTestCase;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
public class GatewayModuleTests extends ModuleTestCase {
public void testCustomGateway() {
GatewayModule gatewayModule = new GatewayModule(Settings.builder().put(GatewayModule.GATEWAY_TYPE_KEY, "mock").build());
gatewayModule.registerGatewayType("mock", MockGateway.class);
assertBinding(gatewayModule, Gateway.class, MockGateway.class);
}
public void testDefaultGateway() {
GatewayModule gatewayModule = new GatewayModule(Settings.EMPTY);
assertBinding(gatewayModule, Gateway.class, Gateway.class);
}
public static class MockGateway extends Gateway {
@Inject
public MockGateway(Settings settings, ClusterService clusterService, NodeEnvironment nodeEnv, GatewayMetaState metaState, TransportNodesListGatewayMetaState listGatewayMetaState, ClusterName clusterName) {
super(settings, clusterService, nodeEnv, metaState, listGatewayMetaState, clusterName);
}
}
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.gateway;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.NoopDiscovery;
import org.elasticsearch.test.cluster.NoopClusterService;
import org.hamcrest.Matchers;
@ -32,7 +33,8 @@ public class GatewayServiceTests extends ESTestCase {
return new GatewayService(Settings.builder()
.put("http.enabled", "false")
.put("discovery.type", "local")
.put(settings.build()).build(), null, null, new NoopClusterService(), null, null);
.put(settings.build()).build(),
null, new NoopClusterService(), null, null, null, null, null, new NoopDiscovery());
}

View File

@ -1,67 +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.gateway;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.cluster.TestClusterService;
import java.util.HashMap;
import java.util.Map;
import static org.hamcrest.Matchers.equalTo;
public class GatewayTests extends ESTestCase {
public void testCalcRequiredAllocations() {
MockGateway gateway = new MockGateway(Settings.EMPTY, new TestClusterService());
int nodeCount = randomIntBetween(1, 6);
Map<String, Integer> expectedResult = new HashMap<>();
expectedResult.put("quorum", nodeCount > 2 ? nodeCount / 2 + 1 : 1);
expectedResult.put("quorum-1", nodeCount > 2 ? (nodeCount + 1) / 2 : 1);
expectedResult.put("half", expectedResult.get("quorum-1"));
expectedResult.put("one", 1);
expectedResult.put("full", nodeCount);
expectedResult.put("all", nodeCount);
expectedResult.put("full-1", Math.max(1, nodeCount - 1));
expectedResult.put("all-1", Math.max(1, nodeCount - 1));
int i = randomIntBetween(1, 20);
expectedResult.put("" + i, i);
expectedResult.put(randomUnicodeOfCodepointLength(10), 1);
for (String setting : expectedResult.keySet()) {
assertThat("unexpected result for setting [" + setting + "]", gateway.calcRequiredAllocations(setting, nodeCount), equalTo(expectedResult.get(setting).intValue()));
}
}
static class MockGateway extends Gateway {
MockGateway(Settings settings, ClusterService clusterService) {
super(settings, clusterService, null, null, null, ClusterName.DEFAULT);
}
@Override
public int calcRequiredAllocations(String setting, int nodeCount) {
return super.calcRequiredAllocations(setting, nodeCount);
}
}
}

View File

@ -39,6 +39,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardStateMetaData;
import org.elasticsearch.test.ESAllocationTestCase;
import org.junit.Before;
@ -94,7 +95,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
}
/**
* Tests when the node returns that no data was found for it (-1 for version and null for allocation id),
* Tests when the node returns that no data was found for it ({@link ShardStateMetaData#NO_VERSION} for version and null for allocation id),
* it will be moved to ignore unassigned.
*/
public void testNoAllocationFound() {
@ -104,7 +105,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
} else {
allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_0);
}
testAllocator.addData(node1, -1, null, randomBoolean());
testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, null, randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
@ -288,7 +289,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testRestoreDoesNotAssignIfNoShardAvailable() {
RoutingAllocation allocation = getRestoreRoutingAllocation(yesAllocationDeciders());
testAllocator.addData(node1, -1, null, false);
testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, null, false);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
@ -356,7 +357,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testRecoverOnAnyNodeDoesNotAssignIfNoShardAvailable() {
RoutingAllocation allocation = getRecoverOnAnyNodeRoutingAllocation(yesAllocationDeciders());
testAllocator.addData(node1, -1, null, randomBoolean());
testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, null, randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));

View File

@ -39,8 +39,8 @@ public class PriorityComparatorTests extends ESTestCase {
public void testPreferNewIndices() {
RoutingNodes.UnassignedShards shards = new RoutingNodes.UnassignedShards((RoutingNodes) null);
List<ShardRouting> shardRoutings = Arrays.asList(TestShardRouting.newShardRouting("oldest", 0, null, null, null,
randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), TestShardRouting.newShardRouting("newest", 0, null, null, null,
randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")));
randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), TestShardRouting.newShardRouting("newest", 0, null, null, null,
randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")));
Collections.shuffle(shardRoutings, random());
for (ShardRouting routing : shardRoutings) {
shards.add(routing);
@ -69,8 +69,8 @@ public class PriorityComparatorTests extends ESTestCase {
public void testPreferPriorityIndices() {
RoutingNodes.UnassignedShards shards = new RoutingNodes.UnassignedShards((RoutingNodes) null);
List<ShardRouting> shardRoutings = Arrays.asList(TestShardRouting.newShardRouting("oldest", 0, null, null, null,
randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), TestShardRouting.newShardRouting("newest", 0, null, null, null,
randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")));
randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), TestShardRouting.newShardRouting("newest", 0, null, null, null,
randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")));
Collections.shuffle(shardRoutings, random());
for (ShardRouting routing : shardRoutings) {
shards.add(routing);
@ -114,7 +114,7 @@ public class PriorityComparatorTests extends ESTestCase {
for (int i = 0; i < numShards; i++) {
IndexMeta indexMeta = randomFrom(indices);
shards.add(TestShardRouting.newShardRouting(indexMeta.name, randomIntBetween(1, 5), null, null, null,
randomBoolean(), ShardRoutingState.UNASSIGNED, randomIntBetween(0, 100), new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")));
randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")));
}
shards.sort(new PriorityComparator() {
@Override

View File

@ -284,7 +284,7 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase {
}
private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders deciders, Settings settings, UnassignedInfo.Reason reason) {
ShardRouting primaryShard = TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node1.id(), true, ShardRoutingState.STARTED, 10);
ShardRouting primaryShard = TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node1.id(), true, ShardRoutingState.STARTED);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndexName()).settings(settings(Version.CURRENT).put(settings))
.numberOfShards(1).numberOfReplicas(1)
@ -306,7 +306,7 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase {
}
private RoutingAllocation onePrimaryOnNode1And1ReplicaRecovering(AllocationDeciders deciders) {
ShardRouting primaryShard = TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node1.id(), true, ShardRoutingState.STARTED, 10);
ShardRouting primaryShard = TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node1.id(), true, ShardRoutingState.STARTED);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndexName()).settings(settings(Version.CURRENT))
.numberOfShards(1).numberOfReplicas(1)
@ -316,7 +316,7 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase {
.add(IndexRoutingTable.builder(shardId.getIndex())
.addIndexShard(new IndexShardRoutingTable.Builder(shardId)
.addShard(primaryShard)
.addShard(TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node2.id(), null, null, false, ShardRoutingState.INITIALIZING, 10, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null)))
.addShard(TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node2.id(), null, null, false, ShardRoutingState.INITIALIZING, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null)))
.build())
)
.build();

View File

@ -114,7 +114,7 @@ public class IndexModuleTests extends ESTestCase {
ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry);
ScriptService scriptService = new ScriptService(settings, environment, scriptEngines, new ResourceWatcherService(settings, threadPool), scriptEngineRegistry, scriptContextRegistry, scriptSettings);
IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(settings, emptyMap());
return new NodeServicesProvider(threadPool, indicesQueryCache, null, warmer, bigArrays, client, scriptService, indicesQueriesRegistry, indicesFieldDataCache, circuitBreakerService);
return new NodeServicesProvider(threadPool, indicesQueryCache, warmer, bigArrays, client, scriptService, indicesQueriesRegistry, indicesFieldDataCache, circuitBreakerService);
}
@Override

View File

@ -198,43 +198,33 @@ public class IndexShardTests extends ESSingleNodeTestCase {
IndexShard shard = test.getShardOrNull(0);
ShardStateMetaData shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(getShardStateMetadata(shard), shardStateMetaData);
ShardRouting routing = new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1);
ShardRouting routing = new ShardRouting(shard.shardRouting);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID(), routing.allocationId()));
routing = new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID(), routing.allocationId()));
routing = new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID(), routing.allocationId()));
assertEquals(shardStateMetaData, new ShardStateMetaData(routing.primary(), shard.indexSettings().getUUID(), routing.allocationId()));
// test if we still write it even if the shard is not active
ShardRouting inactiveRouting = TestShardRouting.newShardRouting(shard.shardRouting.index(), shard.shardRouting.shardId().id(), shard.shardRouting.currentNodeId(), null, null, true, ShardRoutingState.INITIALIZING, shard.shardRouting.version() + 1);
ShardRouting inactiveRouting = TestShardRouting.newShardRouting(shard.shardRouting.index(), shard.shardRouting.shardId().id(), shard.shardRouting.currentNodeId(), null, null, true, ShardRoutingState.INITIALIZING);
shard.persistMetadata(inactiveRouting, shard.shardRouting);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals("inactive shard state shouldn't be persisted", shardStateMetaData, getShardStateMetadata(shard));
assertEquals("inactive shard state shouldn't be persisted", shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID(), routing.allocationId()));
assertEquals("inactive shard state shouldn't be persisted", shardStateMetaData, new ShardStateMetaData(routing.primary(), shard.indexSettings().getUUID(), routing.allocationId()));
shard.updateRoutingEntry(new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1), false);
ShardRouting updatedRouting = new ShardRouting(shard.shardRouting);
TestShardRouting.relocate(updatedRouting, "some node", 42L);
shard.updateRoutingEntry(updatedRouting, false);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertFalse("shard state persisted despite of persist=false", shardStateMetaData.equals(getShardStateMetadata(shard)));
assertEquals("shard state persisted despite of persist=false", shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID(), routing.allocationId()));
assertEquals("shard state persisted despite of persist=false", shardStateMetaData, new ShardStateMetaData(routing.primary(), shard.indexSettings().getUUID(), routing.allocationId()));
routing = new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1);
shard.updateRoutingEntry(routing, false); // move back state in IndexShard
routing = new ShardRouting(updatedRouting);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID(), routing.allocationId()));
assertEquals(shardStateMetaData, new ShardStateMetaData(routing.primary(), shard.indexSettings().getUUID(), routing.allocationId()));
}
public void testDeleteShardState() throws IOException {
@ -255,7 +245,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
ShardStateMetaData shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
routing = TestShardRouting.newShardRouting(shard.shardId.getIndex(), shard.shardId.id(), routing.currentNodeId(), null, routing.primary(), ShardRoutingState.INITIALIZING, shard.shardRouting.allocationId(), shard.shardRouting.version() + 1);
routing = TestShardRouting.newShardRouting(shard.shardId.getIndex(), shard.shardId.id(), routing.currentNodeId(), null, routing.primary(), ShardRoutingState.INITIALIZING, shard.shardRouting.allocationId());
shard.updateRoutingEntry(routing, true);
shard.deleteShardState();
@ -287,7 +277,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
if (shardRouting == null) {
return null;
} else {
return new ShardStateMetaData(shardRouting.version(), shardRouting.primary(), shard.indexSettings().getUUID(), shardRouting.allocationId());
return new ShardStateMetaData(shardRouting.primary(), shard.indexSettings().getUUID(), shardRouting.allocationId());
}
}
@ -303,13 +293,13 @@ public class IndexShardTests extends ESSingleNodeTestCase {
AllocationId allocationId = randomBoolean() ? null : randomAllocationId();
ShardStateMetaData meta = new ShardStateMetaData(randomLong(), randomBoolean(), randomRealisticUnicodeOfCodepointLengthBetween(1, 10), allocationId);
assertEquals(meta, new ShardStateMetaData(meta.version, meta.primary, meta.indexUUID, meta.allocationId));
assertEquals(meta.hashCode(), new ShardStateMetaData(meta.version, meta.primary, meta.indexUUID, meta.allocationId).hashCode());
assertEquals(meta, new ShardStateMetaData(meta.legacyVersion, meta.primary, meta.indexUUID, meta.allocationId));
assertEquals(meta.hashCode(), new ShardStateMetaData(meta.legacyVersion, meta.primary, meta.indexUUID, meta.allocationId).hashCode());
assertFalse(meta.equals(new ShardStateMetaData(meta.version, !meta.primary, meta.indexUUID, meta.allocationId)));
assertFalse(meta.equals(new ShardStateMetaData(meta.version + 1, meta.primary, meta.indexUUID, meta.allocationId)));
assertFalse(meta.equals(new ShardStateMetaData(meta.version, !meta.primary, meta.indexUUID + "foo", meta.allocationId)));
assertFalse(meta.equals(new ShardStateMetaData(meta.version, !meta.primary, meta.indexUUID + "foo", randomAllocationId())));
assertFalse(meta.equals(new ShardStateMetaData(meta.legacyVersion, !meta.primary, meta.indexUUID, meta.allocationId)));
assertFalse(meta.equals(new ShardStateMetaData(meta.legacyVersion + 1, meta.primary, meta.indexUUID, meta.allocationId)));
assertFalse(meta.equals(new ShardStateMetaData(meta.legacyVersion, !meta.primary, meta.indexUUID + "foo", meta.allocationId)));
assertFalse(meta.equals(new ShardStateMetaData(meta.legacyVersion, !meta.primary, meta.indexUUID + "foo", randomAllocationId())));
Set<Integer> hashCodes = new HashSet<>();
for (int i = 0; i < 30; i++) { // just a sanity check that we impl hashcode
allocationId = randomBoolean() ? null : randomAllocationId();
@ -380,7 +370,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
public static void write(ShardStateMetaData shardStateMetaData,
Path... shardPaths) throws IOException {
ShardStateMetaData.FORMAT.write(shardStateMetaData, shardStateMetaData.version, shardPaths);
ShardStateMetaData.FORMAT.write(shardStateMetaData, shardStateMetaData.legacyVersion, shardPaths);
}
public void testDurableFlagHasEffect() {

View File

@ -393,14 +393,6 @@ public class IndicesStoreIntegrationIT extends ESIntegTestCase {
i++;
}
logger.info("Node [{}] has shards: {}", nonMasterNode, Arrays.toString(node2Shards));
final long shardVersions[] = new long[numShards];
final int shardIds[] = new int[numShards];
i = 0;
for (ShardRouting shardRouting : stateResponse.getState().getRoutingTable().allShards("test")) {
shardVersions[i] = shardRouting.version();
shardIds[i] = shardRouting.getId();
i++;
}
// disable relocations when we do this, to make sure the shards are not relocated from node2
// due to rebalancing, and delete its content
@ -412,7 +404,7 @@ public class IndicesStoreIntegrationIT extends ESIntegTestCase {
for (int i = 0; i < numShards; i++) {
indexRoutingTableBuilder.addIndexShard(
new IndexShardRoutingTable.Builder(new ShardId(index, i))
.addShard(TestShardRouting.newShardRouting("test", i, masterId, true, ShardRoutingState.STARTED, shardVersions[shardIds[i]]))
.addShard(TestShardRouting.newShardRouting("test", i, masterId, true, ShardRoutingState.STARTED))
.build()
);
}

View File

@ -98,7 +98,7 @@ public class IndicesStoreTests extends ESTestCase {
if (state == ShardRoutingState.UNASSIGNED) {
unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null);
}
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, null, j == 0, state, 0, unassignedInfo));
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, null, j == 0, state, unassignedInfo));
}
}
assertFalse(indicesStore.shardCanBeDeleted(clusterState.build(), routingTable.build()));
@ -116,9 +116,9 @@ public class IndicesStoreTests extends ESTestCase {
for (int i = 0; i < numShards; i++) {
String nodeId = i == localShardId ? localNode.getId() : randomBoolean() ? "abc" : "xyz";
String relocationNodeId = randomBoolean() ? null : randomBoolean() ? localNode.getId() : "xyz";
routingTable.addShard(TestShardRouting.newShardRouting("test", i, nodeId, relocationNodeId, true, ShardRoutingState.STARTED, 0));
routingTable.addShard(TestShardRouting.newShardRouting("test", i, nodeId, relocationNodeId, true, ShardRoutingState.STARTED));
for (int j = 0; j < numReplicas; j++) {
routingTable.addShard(TestShardRouting.newShardRouting("test", i, nodeId, relocationNodeId, false, ShardRoutingState.STARTED, 0));
routingTable.addShard(TestShardRouting.newShardRouting("test", i, nodeId, relocationNodeId, false, ShardRoutingState.STARTED));
}
}
@ -136,9 +136,9 @@ public class IndicesStoreTests extends ESTestCase {
IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", "_na_", 1));
for (int i = 0; i < numShards; i++) {
String relocatingNodeId = randomBoolean() ? null : "def";
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", relocatingNodeId, true, ShardRoutingState.STARTED, 0));
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", relocatingNodeId, true, ShardRoutingState.STARTED));
for (int j = 0; j < numReplicas; j++) {
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", relocatingNodeId, false, ShardRoutingState.STARTED, 0));
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", relocatingNodeId, false, ShardRoutingState.STARTED));
}
}
@ -157,9 +157,9 @@ public class IndicesStoreTests extends ESTestCase {
clusterState.nodes(DiscoveryNodes.builder().localNodeId(localNode.id()).put(localNode).put(new DiscoveryNode("xyz", new LocalTransportAddress("xyz"), nodeVersion)));
IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", "_na_", 1));
for (int i = 0; i < numShards; i++) {
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, true, ShardRoutingState.STARTED, 0));
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, true, ShardRoutingState.STARTED));
for (int j = 0; j < numReplicas; j++) {
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, false, ShardRoutingState.STARTED, 0));
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, false, ShardRoutingState.STARTED));
}
}
@ -182,9 +182,9 @@ public class IndicesStoreTests extends ESTestCase {
));
IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", "_na_", 1));
for (int i = 0; i < numShards; i++) {
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", "def", true, ShardRoutingState.STARTED, 0));
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", "def", true, ShardRoutingState.STARTED));
for (int j = 0; j < numReplicas; j++) {
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", "def", false, ShardRoutingState.STARTED, 0));
routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", "def", false, ShardRoutingState.STARTED));
}
}

View File

@ -244,7 +244,7 @@ public class IngestClientIT extends ESIntegTestCase {
}
public void onModule(NodeModule nodeModule) {
nodeModule.registerProcessor("test", templateService -> config ->
nodeModule.registerProcessor("test", (templateService, registry) -> config ->
new TestProcessor("id", "test", ingestDocument -> {
ingestDocument.setFieldValue("processed", true);
if (ingestDocument.getFieldValue("fail", Boolean.class)) {

View File

@ -50,9 +50,9 @@ public class PipelineStoreTests extends ESTestCase {
@Before
public void init() throws Exception {
store = new PipelineStore(Settings.EMPTY);
ProcessorsRegistry registry = new ProcessorsRegistry();
registry.registerProcessor("set", (templateService) -> new SetProcessor.Factory(TestTemplateService.instance()));
store.buildProcessorFactoryRegistry(registry, null);
ProcessorsRegistry.Builder registryBuilder = new ProcessorsRegistry.Builder();
registryBuilder.registerProcessor("set", (templateService, registry) -> new SetProcessor.Factory(TestTemplateService.instance()));
store.buildProcessorFactoryRegistry(registryBuilder, null);
}
public void testUpdatePipelines() {

View File

@ -19,42 +19,30 @@
package org.elasticsearch.ingest;
import org.elasticsearch.ingest.core.Processor;
import org.elasticsearch.ingest.core.TemplateService;
import org.elasticsearch.test.ESTestCase;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.sameInstance;
public class ProcessorsRegistryTests extends ESTestCase {
public void testAddProcessor() {
ProcessorsRegistry processorsRegistry = new ProcessorsRegistry();
public void testBuildProcessorRegistry() {
ProcessorsRegistry.Builder builder = new ProcessorsRegistry.Builder();
TestProcessor.Factory factory1 = new TestProcessor.Factory();
processorsRegistry.registerProcessor("1", (templateService) -> factory1);
builder.registerProcessor("1", (templateService, registry) -> factory1);
TestProcessor.Factory factory2 = new TestProcessor.Factory();
processorsRegistry.registerProcessor("2", (templateService) -> factory2);
builder.registerProcessor("2", (templateService, registry) -> factory2);
TestProcessor.Factory factory3 = new TestProcessor.Factory();
try {
processorsRegistry.registerProcessor("1", (templateService) -> factory3);
builder.registerProcessor("1", (templateService, registry) -> factory3);
fail("addProcessor should have failed");
} catch(IllegalArgumentException e) {
assertThat(e.getMessage(), equalTo("Processor factory already registered for name [1]"));
}
Set<Map.Entry<String, Function<TemplateService, Processor.Factory<?>>>> entrySet = processorsRegistry.entrySet();
assertThat(entrySet.size(), equalTo(2));
for (Map.Entry<String, Function<TemplateService, Processor.Factory<?>>> entry : entrySet) {
if (entry.getKey().equals("1")) {
assertThat(entry.getValue().apply(null), equalTo(factory1));
} else if (entry.getKey().equals("2")) {
assertThat(entry.getValue().apply(null), equalTo(factory2));
} else {
fail("unexpected processor id [" + entry.getKey() + "]");
}
}
ProcessorsRegistry registry = builder.build(TestTemplateService.instance());
assertThat(registry.getProcessorFactories().size(), equalTo(2));
assertThat(registry.getProcessorFactory("1"), sameInstance(factory1));
assertThat(registry.getProcessorFactory("2"), sameInstance(factory2));
}
}

View File

@ -20,6 +20,8 @@
package org.elasticsearch.ingest.core;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.ingest.ProcessorsRegistry;
import org.elasticsearch.ingest.TestTemplateService;
import org.elasticsearch.test.ESTestCase;
import org.junit.Before;
@ -31,6 +33,9 @@ import java.util.List;
import java.util.Map;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.sameInstance;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.mock;
public class ConfigurationUtilsTests extends ESTestCase {
@ -68,4 +73,31 @@ public class ConfigurationUtilsTests extends ESTestCase {
List<String> val = ConfigurationUtils.readList(null, null, config, "int");
assertThat(val, equalTo(Collections.singletonList(2)));
}
public void testReadProcessors() throws Exception {
Processor processor = mock(Processor.class);
ProcessorsRegistry.Builder builder = new ProcessorsRegistry.Builder();
builder.registerProcessor("test_processor", (templateService, registry) -> config -> processor);
ProcessorsRegistry registry = builder.build(TestTemplateService.instance());
List<Map<String, Map<String, Object>>> config = new ArrayList<>();
Map<String, Object> emptyConfig = Collections.emptyMap();
config.add(Collections.singletonMap("test_processor", emptyConfig));
config.add(Collections.singletonMap("test_processor", emptyConfig));
List<Processor> result = ConfigurationUtils.readProcessorConfigs(config, registry);
assertThat(result.size(), equalTo(2));
assertThat(result.get(0), sameInstance(processor));
assertThat(result.get(1), sameInstance(processor));
config.add(Collections.singletonMap("unknown_processor", emptyConfig));
try {
ConfigurationUtils.readProcessorConfigs(config, registry);
fail("exception expected");
} catch (ElasticsearchParseException e) {
assertThat(e.getMessage(), equalTo("No processor type exists with name [unknown_processor]"));
}
}
}

View File

@ -20,13 +20,16 @@
package org.elasticsearch.ingest.core;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.ingest.ProcessorsRegistry;
import org.elasticsearch.ingest.TestProcessor;
import org.elasticsearch.ingest.TestTemplateService;
import org.elasticsearch.test.ESTestCase;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.prefs.PreferencesFactory;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.Matchers.nullValue;
@ -41,7 +44,7 @@ public class PipelineFactoryTests extends ESTestCase {
pipelineConfig.put(Pipeline.DESCRIPTION_KEY, "_description");
pipelineConfig.put(Pipeline.PROCESSORS_KEY, Arrays.asList(Collections.singletonMap("test", processorConfig0), Collections.singletonMap("test", processorConfig1)));
Pipeline.Factory factory = new Pipeline.Factory();
Map<String, Processor.Factory> processorRegistry = Collections.singletonMap("test", new TestProcessor.Factory());
ProcessorsRegistry processorRegistry = createProcessorRegistry(Collections.singletonMap("test", new TestProcessor.Factory()));
Pipeline pipeline = factory.create("_id", pipelineConfig, processorRegistry);
assertThat(pipeline.getId(), equalTo("_id"));
assertThat(pipeline.getDescription(), equalTo("_description"));
@ -57,7 +60,7 @@ public class PipelineFactoryTests extends ESTestCase {
pipelineConfig.put(Pipeline.DESCRIPTION_KEY, "_description");
Pipeline.Factory factory = new Pipeline.Factory();
try {
factory.create("_id", pipelineConfig, Collections.emptyMap());
factory.create("_id", pipelineConfig, createProcessorRegistry(Collections.emptyMap()));
fail("should fail, missing required [processors] field");
} catch (ElasticsearchParseException e) {
assertThat(e.getMessage(), equalTo("[processors] required property is missing"));
@ -71,7 +74,7 @@ public class PipelineFactoryTests extends ESTestCase {
pipelineConfig.put(Pipeline.PROCESSORS_KEY, Collections.singletonList(Collections.singletonMap("test", processorConfig)));
pipelineConfig.put(Pipeline.ON_FAILURE_KEY, Collections.singletonList(Collections.singletonMap("test", processorConfig)));
Pipeline.Factory factory = new Pipeline.Factory();
Map<String, Processor.Factory> processorRegistry = Collections.singletonMap("test", new TestProcessor.Factory());
ProcessorsRegistry processorRegistry = createProcessorRegistry(Collections.singletonMap("test", new TestProcessor.Factory()));
Pipeline pipeline = factory.create("_id", pipelineConfig, processorRegistry);
assertThat(pipeline.getId(), equalTo("_id"));
assertThat(pipeline.getDescription(), equalTo("_description"));
@ -88,7 +91,7 @@ public class PipelineFactoryTests extends ESTestCase {
pipelineConfig.put(Pipeline.DESCRIPTION_KEY, "_description");
pipelineConfig.put(Pipeline.PROCESSORS_KEY, Collections.singletonList(Collections.singletonMap("test", processorConfig)));
Pipeline.Factory factory = new Pipeline.Factory();
Map<String, Processor.Factory> processorRegistry = Collections.singletonMap("test", new TestProcessor.Factory());
ProcessorsRegistry processorRegistry = createProcessorRegistry(Collections.singletonMap("test", new TestProcessor.Factory()));
try {
factory.create("_id", pipelineConfig, processorRegistry);
} catch (ElasticsearchParseException e) {
@ -104,11 +107,19 @@ public class PipelineFactoryTests extends ESTestCase {
pipelineConfig.put(Pipeline.DESCRIPTION_KEY, "_description");
pipelineConfig.put(Pipeline.PROCESSORS_KEY, Collections.singletonList(Collections.singletonMap("test", processorConfig)));
Pipeline.Factory factory = new Pipeline.Factory();
Map<String, Processor.Factory> processorRegistry = Collections.singletonMap("test", new TestProcessor.Factory());
ProcessorsRegistry processorRegistry = createProcessorRegistry(Collections.singletonMap("test", new TestProcessor.Factory()));
Pipeline pipeline = factory.create("_id", pipelineConfig, processorRegistry);
assertThat(pipeline.getId(), equalTo("_id"));
assertThat(pipeline.getDescription(), equalTo("_description"));
assertThat(pipeline.getProcessors().size(), equalTo(1));
assertThat(pipeline.getProcessors().get(0).getType(), equalTo("compound"));
}
private ProcessorsRegistry createProcessorRegistry(Map<String, Processor.Factory> processorRegistry) {
ProcessorsRegistry.Builder builder = new ProcessorsRegistry.Builder();
for (Map.Entry<String, Processor.Factory> entry : processorRegistry.entrySet()) {
builder.registerProcessor(entry.getKey(), ((templateService, registry) -> entry.getValue()));
}
return builder.build(TestTemplateService.instance());
}
}

View File

@ -0,0 +1,70 @@
/*
* 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.ingest.processor;
import org.elasticsearch.ingest.ProcessorsRegistry;
import org.elasticsearch.ingest.TestProcessor;
import org.elasticsearch.ingest.TestTemplateService;
import org.elasticsearch.ingest.core.Processor;
import org.elasticsearch.test.ESTestCase;
import org.hamcrest.Matchers;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
public class ForEachProcessorFactoryTests extends ESTestCase {
public void testCreate() throws Exception {
ProcessorsRegistry.Builder builder = new ProcessorsRegistry.Builder();
Processor processor = new TestProcessor(ingestDocument -> {});
builder.registerProcessor("_name", (templateService, registry) -> config -> processor);
ProcessorsRegistry registry = builder.build(TestTemplateService.instance());
ForEachProcessor.Factory forEachFactory = new ForEachProcessor.Factory(registry);
Map<String, Object> config = new HashMap<>();
config.put("field", "_field");
config.put("processors", Collections.singletonList(Collections.singletonMap("_name", Collections.emptyMap())));
ForEachProcessor forEachProcessor = forEachFactory.create(config);
assertThat(forEachProcessor, Matchers.notNullValue());
assertThat(forEachProcessor.getField(), Matchers.equalTo("_field"));
assertThat(forEachProcessor.getProcessors().size(), Matchers.equalTo(1));
assertThat(forEachProcessor.getProcessors().get(0), Matchers.sameInstance(processor));
config = new HashMap<>();
config.put("processors", Collections.singletonList(Collections.singletonMap("_name", Collections.emptyMap())));
try {
forEachFactory.create(config);
fail("exception expected");
} catch (Exception e) {
assertThat(e.getMessage(), Matchers.equalTo("[field] required property is missing"));
}
config = new HashMap<>();
config.put("field", "_field");
try {
forEachFactory.create(config);
fail("exception expected");
} catch (Exception e) {
assertThat(e.getMessage(), Matchers.equalTo("[processors] required property is missing"));
}
}
}

View File

@ -0,0 +1,169 @@
/*
* 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.ingest.processor;
import org.elasticsearch.ingest.TestProcessor;
import org.elasticsearch.ingest.core.CompoundProcessor;
import org.elasticsearch.ingest.core.IngestDocument;
import org.elasticsearch.ingest.core.Processor;
import org.elasticsearch.test.ESTestCase;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import static org.hamcrest.Matchers.equalTo;
public class ForEachProcessorTests extends ESTestCase {
public void testExecute() throws Exception {
List<String> values = new ArrayList<>();
values.add("foo");
values.add("bar");
values.add("baz");
IngestDocument ingestDocument = new IngestDocument(
"_index", "_type", "_id", null, null, null, null, Collections.singletonMap("values", values)
);
ForEachProcessor processor = new ForEachProcessor(
"_tag", "values", Collections.singletonList(new UppercaseProcessor("_tag", "_value"))
);
processor.execute(ingestDocument);
List<String> result = ingestDocument.getFieldValue("values", List.class);
assertThat(result.get(0), equalTo("FOO"));
assertThat(result.get(1), equalTo("BAR"));
assertThat(result.get(2), equalTo("BAZ"));
}
public void testExecuteWithFailure() throws Exception {
IngestDocument ingestDocument = new IngestDocument(
"_index", "_type", "_id", null, null, null, null, Collections.singletonMap("values", Arrays.asList("a", "b", "c"))
);
TestProcessor testProcessor = new TestProcessor(id -> {
if ("c".equals(id.getFieldValue("_value", String.class))) {
throw new RuntimeException("failure");
}
});
ForEachProcessor processor = new ForEachProcessor("_tag", "values", Collections.singletonList(testProcessor));
try {
processor.execute(ingestDocument);
fail("exception expected");
} catch (RuntimeException e) {
assertThat(e.getMessage(), equalTo("failure"));
}
assertThat(testProcessor.getInvokedCounter(), equalTo(3));
assertThat(ingestDocument.getFieldValue("values", List.class), equalTo(Arrays.asList("a", "b", "c")));
testProcessor = new TestProcessor(id -> {
String value = id.getFieldValue("_value", String.class);
if ("c".equals(value)) {
throw new RuntimeException("failure");
} else {
id.setFieldValue("_value", value.toUpperCase(Locale.ROOT));
}
});
Processor onFailureProcessor = new TestProcessor(ingestDocument1 -> {});
processor = new ForEachProcessor(
"_tag", "values",
Collections.singletonList(new CompoundProcessor(Arrays.asList(testProcessor), Arrays.asList(onFailureProcessor)))
);
processor.execute(ingestDocument);
assertThat(testProcessor.getInvokedCounter(), equalTo(3));
assertThat(ingestDocument.getFieldValue("values", List.class), equalTo(Arrays.asList("A", "B", "c")));
}
public void testMetaDataAvailable() throws Exception {
List<Map<String, Object>> values = new ArrayList<>();
values.add(new HashMap<>());
values.add(new HashMap<>());
IngestDocument ingestDocument = new IngestDocument(
"_index", "_type", "_id", null, null, null, null, Collections.singletonMap("values", values)
);
TestProcessor innerProcessor = new TestProcessor(id -> {
id.setFieldValue("_value.index", id.getSourceAndMetadata().get("_index"));
id.setFieldValue("_value.type", id.getSourceAndMetadata().get("_type"));
id.setFieldValue("_value.id", id.getSourceAndMetadata().get("_id"));
});
ForEachProcessor processor = new ForEachProcessor("_tag", "values", Collections.singletonList(innerProcessor));
processor.execute(ingestDocument);
assertThat(innerProcessor.getInvokedCounter(), equalTo(2));
assertThat(ingestDocument.getFieldValue("values.0.index", String.class), equalTo("_index"));
assertThat(ingestDocument.getFieldValue("values.0.type", String.class), equalTo("_type"));
assertThat(ingestDocument.getFieldValue("values.0.id", String.class), equalTo("_id"));
assertThat(ingestDocument.getFieldValue("values.1.index", String.class), equalTo("_index"));
assertThat(ingestDocument.getFieldValue("values.1.type", String.class), equalTo("_type"));
assertThat(ingestDocument.getFieldValue("values.1.id", String.class), equalTo("_id"));
}
public void testRandom() throws Exception {
int numProcessors = randomInt(8);
List<Processor> processors = new ArrayList<>(numProcessors);
for (int i = 0; i < numProcessors; i++) {
processors.add(new Processor() {
@Override
public void execute(IngestDocument ingestDocument) throws Exception {
String existingValue = ingestDocument.getFieldValue("_value", String.class);
ingestDocument.setFieldValue("_value", existingValue + ".");
}
@Override
public String getType() {
return null;
}
@Override
public String getTag() {
return null;
}
});
}
int numValues = randomIntBetween(1, 32);
List<String> values = new ArrayList<>(numValues);
for (int i = 0; i < numValues; i++) {
values.add("");
}
IngestDocument ingestDocument = new IngestDocument(
"_index", "_type", "_id", null, null, null, null, Collections.singletonMap("values", values)
);
ForEachProcessor processor = new ForEachProcessor("_tag", "values", processors);
processor.execute(ingestDocument);
List<String> result = ingestDocument.getFieldValue("values", List.class);
assertThat(result.size(), equalTo(numValues));
String expectedString = "";
for (int i = 0; i < numProcessors; i++) {
expectedString = expectedString + ".";
}
for (String r : result) {
assertThat(r, equalTo(expectedString));
}
}
}

View File

@ -81,22 +81,14 @@ public class InternalSettingsPreparerTests extends ESTestCase {
}
public void testReplacePromptPlaceholders() {
final List<String> replacedSecretProperties = new ArrayList<>();
final List<String> replacedTextProperties = new ArrayList<>();
final Terminal terminal = new CliToolTestCase.MockTerminal() {
@Override
public char[] readSecret(String message, Object... args) {
for (Object arg : args) {
replacedSecretProperties.add((String) arg);
}
public char[] readSecret(String message) {
return "replaced".toCharArray();
}
@Override
public String readText(String message, Object... args) {
for (Object arg : args) {
replacedTextProperties.add((String) arg);
}
public String readText(String message) {
return "text";
}
};
@ -112,8 +104,6 @@ public class InternalSettingsPreparerTests extends ESTestCase {
.put("replace_me", InternalSettingsPreparer.TEXT_PROMPT_VALUE);
Settings settings = InternalSettingsPreparer.prepareEnvironment(builder.build(), terminal).settings();
assertThat(replacedSecretProperties.size(), is(1));
assertThat(replacedTextProperties.size(), is(1));
assertThat(settings.get("password.replace"), equalTo("replaced"));
assertThat(settings.get("replace_me"), equalTo("text"));

View File

@ -29,6 +29,7 @@ import org.elasticsearch.action.termvectors.TermVectorsResponse;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.termvectors.TermVectorsService;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.SearchModule;
@ -129,9 +130,6 @@ public class FetchSubPhasePluginIT extends ESIntegTestCase {
}
};
public TermVectorsFetchSubPhase() {
}
public static final String[] NAMES = {"term_vectors_fetch"};
@Override
@ -158,14 +156,14 @@ public class FetchSubPhasePluginIT extends ESIntegTestCase {
String field = context.getFetchSubPhaseContext(CONTEXT_FACTORY).getField();
if (hitContext.hit().fieldsOrNull() == null) {
hitContext.hit().fields(new HashMap<String, SearchHitField>());
hitContext.hit().fields(new HashMap<>());
}
SearchHitField hitField = hitContext.hit().fields().get(NAMES[0]);
if (hitField == null) {
hitField = new InternalSearchHitField(NAMES[0], new ArrayList<>(1));
hitContext.hit().fields().put(NAMES[0], hitField);
}
TermVectorsResponse termVector = context.indexShard().getTermVectors(new TermVectorsRequest(context.indexShard().shardId().getIndex().getName(), hitContext.hit().type(), hitContext.hit().id()));
TermVectorsResponse termVector = TermVectorsService.getTermVectors(context.indexShard(), new TermVectorsRequest(context.indexShard().shardId().getIndex().getName(), hitContext.hit().type(), hitContext.hit().id()));
try {
Map<String, Integer> tv = new HashMap<>();
TermsEnum terms = termVector.getFields().terms(field).iterator();

View File

@ -0,0 +1,109 @@
/*
* 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.test;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.Lifecycle;
import org.elasticsearch.common.component.LifecycleListener;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoveryStats;
import org.elasticsearch.discovery.InitialStateDiscoveryListener;
import org.elasticsearch.node.service.NodeService;
public class NoopDiscovery implements Discovery {
@Override
public DiscoveryNode localNode() {
return null;
}
@Override
public void addListener(InitialStateDiscoveryListener listener) {
}
@Override
public void removeListener(InitialStateDiscoveryListener listener) {
}
@Override
public String nodeDescription() {
return null;
}
@Override
public void setNodeService(@Nullable NodeService nodeService) {
}
@Override
public void setRoutingService(RoutingService routingService) {
}
@Override
public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) {
}
@Override
public DiscoveryStats stats() {
return null;
}
@Override
public int getMinimumMasterNodes() {
return -1;
}
@Override
public Lifecycle.State lifecycleState() {
return null;
}
@Override
public void addLifecycleListener(LifecycleListener listener) {
}
@Override
public void removeLifecycleListener(LifecycleListener listener) {
}
@Override
public Discovery start() {
return null;
}
@Override
public Discovery stop() {
return null;
}
@Override
public void close() {
}
}

View File

@ -3,7 +3,7 @@
Provides store information for shard copies of indices.
Store information reports on which nodes shard copies exist, the shard
copy version, indicating how recent they are, and any exceptions
copy allocation ID, a unique identifer for each shard copy, and any exceptions
encountered while opening the shard index or from earlier engine failure.
By default, only lists store information for shards that have at least one
@ -51,8 +51,8 @@ The shard stores information is grouped by indices and shard ids.
"mode": "local"
}
},
"version": 4, <4>
"allocation_id": "2iNySv_OQVePRX-yaRH_lQ", <5>
"allocation_id": "2iNySv_OQVePRX-yaRH_lQ", <4>
"legacy_version": 42, <5>
"allocation" : "primary" | "replica" | "unused", <6>
"store_exception": ... <7>
},
@ -66,8 +66,9 @@ The shard stores information is grouped by indices and shard ids.
<2> A list of store information for all copies of the shard
<3> The node information that hosts a copy of the store, the key
is the unique node id.
<4> The version of the store copy
<5> The allocation id of the store copy
<4> The allocation id of the store copy
<5> The version of the store copy (available only for legacy shard copies that have
not yet been active in a current version of Elasticsearch)
<6> The status of the store copy, whether it is used as a
primary, replica or not used at all
<7> Any exception encountered while opening the shard index or

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