diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java index ad06f75074d..860137cf559 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java @@ -22,10 +22,10 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.EmptyClusterInfoService; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; +import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; -import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; @@ -35,9 +35,7 @@ import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.gateway.GatewayAllocator; -import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -52,12 +50,12 @@ public final class Allocators { } @Override - public void applyStartedShards(StartedRerouteAllocation allocation) { + public void applyStartedShards(RoutingAllocation allocation, List startedShards) { // noop } @Override - public void applyFailedShards(FailedRerouteAllocation allocation) { + public void applyFailedShards(RoutingAllocation allocation, List failedShards) { // noop } diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 3ac70271ce4..a995c201c47 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -342,7 +342,6 @@ - diff --git a/client/benchmark/src/main/resources/log4j.properties b/client/benchmark/src/main/resources/log4j.properties deleted file mode 100644 index 22f54ef68e5..00000000000 --- a/client/benchmark/src/main/resources/log4j.properties +++ /dev/null @@ -1,9 +0,0 @@ -es.logger.level=INFO -log4j.rootLogger=${es.logger.level}, out - -log4j.logger.org.apache.http=INFO, out -log4j.additivity.org.apache.http=false - -log4j.appender.out=org.apache.log4j.ConsoleAppender -log4j.appender.out.layout=org.apache.log4j.PatternLayout -log4j.appender.out.layout.conversionPattern=[%d{ISO8601}][%-5p][%-25c] %m%n diff --git a/client/benchmark/src/main/resources/log4j2.properties b/client/benchmark/src/main/resources/log4j2.properties new file mode 100644 index 00000000000..8652131bf49 --- /dev/null +++ b/client/benchmark/src/main/resources/log4j2.properties @@ -0,0 +1,7 @@ +appender.console.type = Console +appender.console.name = console +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%d{ISO8601}][%-5p][%-25c] %marker%m%n + +rootLogger.level = info +rootLogger.appenderRef.console.ref = console diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/analyze/TransportAnalyzeAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/analyze/TransportAnalyzeAction.java index f035bc0f4b7..7d7e9d2dd2e 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/analyze/TransportAnalyzeAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/analyze/TransportAnalyzeAction.java @@ -45,9 +45,9 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.AnalysisRegistry; -import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.analysis.CharFilterFactory; import org.elasticsearch.index.analysis.CustomAnalyzer; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.TokenFilterFactory; import org.elasticsearch.index.analysis.TokenizerFactory; @@ -145,45 +145,46 @@ public class TransportAnalyzeAction extends TransportSingleShardAction 0) { charFilterFactories = new CharFilterFactory[request.charFilters().size()]; @@ -468,19 +469,19 @@ public class TransportAnalyzeAction extends TransportSingleShardAction charFilterFactoryFactory; - if (analysisService == null) { + if (indexSettings == null) { charFilterFactoryFactory = analysisRegistry.getCharFilterProvider(charFilter.name); if (charFilterFactoryFactory == null) { throw new IllegalArgumentException("failed to find global char filter under [" + charFilter.name + "]"); } charFilterFactories[i] = charFilterFactoryFactory.get(environment, charFilter.name); } else { - charFilterFactoryFactory = analysisRegistry.getCharFilterProvider(charFilter.name, analysisService.getIndexSettings()); + charFilterFactoryFactory = analysisRegistry.getCharFilterProvider(charFilter.name, indexSettings); if (charFilterFactoryFactory == null) { throw new IllegalArgumentException("failed to find char filter under [" + charFilter.name + "]"); } - charFilterFactories[i] = charFilterFactoryFactory.get(analysisService.getIndexSettings(), environment, charFilter.name, - AnalysisRegistry.getSettingsFromIndexSettings(analysisService.getIndexSettings(), + charFilterFactories[i] = charFilterFactoryFactory.get(indexSettings, environment, charFilter.name, + AnalysisRegistry.getSettingsFromIndexSettings(indexSettings, AnalysisRegistry.INDEX_ANALYSIS_CHAR_FILTER + "." + charFilter.name)); } } @@ -492,7 +493,7 @@ public class TransportAnalyzeAction extends TransportSingleShardAction 0) { tokenFilterFactories = new TokenFilterFactory[request.tokenFilters().size()]; @@ -514,19 +515,19 @@ public class TransportAnalyzeAction extends TransportSingleShardAction tokenFilterFactoryFactory; - if (analysisService == null) { + if (indexSettings == null) { tokenFilterFactoryFactory = analysisRegistry.getTokenFilterProvider(tokenFilter.name); if (tokenFilterFactoryFactory == null) { throw new IllegalArgumentException("failed to find global token filter under [" + tokenFilter.name + "]"); } tokenFilterFactories[i] = tokenFilterFactoryFactory.get(environment, tokenFilter.name); } else { - tokenFilterFactoryFactory = analysisRegistry.getTokenFilterProvider(tokenFilter.name, analysisService.getIndexSettings()); + tokenFilterFactoryFactory = analysisRegistry.getTokenFilterProvider(tokenFilter.name, indexSettings); if (tokenFilterFactoryFactory == null) { throw new IllegalArgumentException("failed to find token filter under [" + tokenFilter.name + "]"); } - tokenFilterFactories[i] = tokenFilterFactoryFactory.get(analysisService.getIndexSettings(), environment, tokenFilter.name, - AnalysisRegistry.getSettingsFromIndexSettings(analysisService.getIndexSettings(), + tokenFilterFactories[i] = tokenFilterFactoryFactory.get(indexSettings, environment, tokenFilter.name, + AnalysisRegistry.getSettingsFromIndexSettings(indexSettings, AnalysisRegistry.INDEX_ANALYSIS_FILTER + "." + tokenFilter.name)); } } @@ -538,7 +539,7 @@ public class TransportAnalyzeAction extends TransportSingleShardAction tokenizerFactoryFactory; - if (analysisService == null) { + if (indexAnalzyers == null) { tokenizerFactoryFactory = analysisRegistry.getTokenizerProvider(tokenizer.name); if (tokenizerFactoryFactory == null) { throw new IllegalArgumentException("failed to find global tokenizer under [" + tokenizer.name + "]"); } tokenizerFactory = tokenizerFactoryFactory.get(environment, tokenizer.name); } else { - tokenizerFactoryFactory = analysisRegistry.getTokenizerProvider(tokenizer.name, analysisService.getIndexSettings()); + tokenizerFactoryFactory = analysisRegistry.getTokenizerProvider(tokenizer.name, indexAnalzyers.getIndexSettings()); if (tokenizerFactoryFactory == null) { throw new IllegalArgumentException("failed to find tokenizer under [" + tokenizer.name + "]"); } - tokenizerFactory = tokenizerFactoryFactory.get(analysisService.getIndexSettings(), environment, tokenizer.name, - AnalysisRegistry.getSettingsFromIndexSettings(analysisService.getIndexSettings(), + tokenizerFactory = tokenizerFactoryFactory.get(indexAnalzyers.getIndexSettings(), environment, tokenizer.name, + AnalysisRegistry.getSettingsFromIndexSettings(indexAnalzyers.getIndexSettings(), AnalysisRegistry.INDEX_ANALYSIS_TOKENIZER + "." + tokenizer.name)); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java index 085cc513f24..e592b5092b7 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -290,7 +290,7 @@ public class ClusterState implements ToXContent, Diffable { for (int shard = 0; shard < indexMetaData.getNumberOfShards(); shard++) { sb.append(TAB).append(TAB).append(shard).append(": "); sb.append("p_term [").append(indexMetaData.primaryTerm(shard)).append("], "); - sb.append("a_ids ").append(indexMetaData.inSyncAllocationIds(shard)).append("\n"); + sb.append("isa_ids ").append(indexMetaData.inSyncAllocationIds(shard)).append("\n"); } } sb.append(blocks().prettyPrint()); diff --git a/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java b/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java index a2c9a7b5f83..ce6473ecb42 100644 --- a/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java +++ b/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java @@ -37,7 +37,8 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingService; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; +import org.elasticsearch.cluster.routing.allocation.FailedShard; +import org.elasticsearch.cluster.routing.allocation.StaleShard; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Priority; @@ -251,8 +252,8 @@ public class ShardStateAction extends AbstractComponent { public BatchResult execute(ClusterState currentState, List tasks) throws Exception { BatchResult.Builder batchResultBuilder = BatchResult.builder(); List tasksToBeApplied = new ArrayList<>(); - List shardRoutingsToBeApplied = new ArrayList<>(); - List staleShardsToBeApplied = new ArrayList<>(); + List failedShardsToBeApplied = new ArrayList<>(); + List staleShardsToBeApplied = new ArrayList<>(); for (ShardEntry task : tasks) { IndexMetaData indexMetaData = currentState.metaData().index(task.shardId.getIndex()); @@ -292,7 +293,7 @@ public class ShardStateAction extends AbstractComponent { if (task.primaryTerm > 0 && inSyncAllocationIds.contains(task.allocationId)) { logger.debug("{} marking shard {} as stale (shard failed task: [{}])", task.shardId, task.allocationId, task); tasksToBeApplied.add(task); - staleShardsToBeApplied.add(new FailedRerouteAllocation.StaleShard(task.shardId, task.allocationId)); + staleShardsToBeApplied.add(new StaleShard(task.shardId, task.allocationId)); } else { // tasks that correspond to non-existent shards are marked as successful logger.debug("{} ignoring shard failed task [{}] (shard does not exist anymore)", task.shardId, task); @@ -302,18 +303,18 @@ public class ShardStateAction extends AbstractComponent { // failing a shard also possibly marks it as stale (see IndexMetaDataUpdater) logger.debug("{} failing shard {} (shard failed task: [{}])", task.shardId, matched, task); tasksToBeApplied.add(task); - shardRoutingsToBeApplied.add(new FailedRerouteAllocation.FailedShard(matched, task.message, task.failure)); + failedShardsToBeApplied.add(new FailedShard(matched, task.message, task.failure)); } } } - assert tasksToBeApplied.size() == shardRoutingsToBeApplied.size() + staleShardsToBeApplied.size(); + assert tasksToBeApplied.size() == failedShardsToBeApplied.size() + staleShardsToBeApplied.size(); ClusterState maybeUpdatedState = currentState; try { - maybeUpdatedState = applyFailedShards(currentState, shardRoutingsToBeApplied, staleShardsToBeApplied); + maybeUpdatedState = applyFailedShards(currentState, failedShardsToBeApplied, staleShardsToBeApplied); batchResultBuilder.successes(tasksToBeApplied); } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("failed to apply failed shards {}", shardRoutingsToBeApplied), e); + logger.warn((Supplier) () -> new ParameterizedMessage("failed to apply failed shards {}", failedShardsToBeApplied), e); // failures are communicated back to the requester // cluster state will not be updated in this case batchResultBuilder.failures(tasksToBeApplied, e); @@ -323,8 +324,7 @@ public class ShardStateAction extends AbstractComponent { } // visible for testing - ClusterState applyFailedShards(ClusterState currentState, List failedShards, - List staleShards) { + ClusterState applyFailedShards(ClusterState currentState, List failedShards, List staleShards) { return allocationService.applyFailedShards(currentState, failedShards, staleShards); } @@ -422,7 +422,7 @@ public class ShardStateAction extends AbstractComponent { ClusterState maybeUpdatedState = currentState; try { - maybeUpdatedState = allocationService.applyStartedShards(currentState, shardRoutingsToBeApplied, true); + maybeUpdatedState = allocationService.applyStartedShards(currentState, shardRoutingsToBeApplied); builder.successes(tasksToBeApplied); } catch (Exception e) { logger.warn((Supplier) () -> new ParameterizedMessage("failed to apply started shards {}", shardRoutingsToBeApplied), e); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java index fa55043f61a..d9faa520682 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java @@ -26,13 +26,16 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.mapper.MapperRegistry; +import java.util.AbstractMap; import java.util.Collections; +import java.util.Map; +import java.util.Set; /** * This service is responsible for upgrading legacy index metadata to the current version @@ -112,9 +115,30 @@ public class MetaDataIndexUpgradeService extends AbstractComponent { // been started yet. However, we don't really need real analyzers at this stage - so we can fake it IndexSettings indexSettings = new IndexSettings(indexMetaData, this.settings); SimilarityService similarityService = new SimilarityService(indexSettings, Collections.emptyMap()); + final NamedAnalyzer fakeDefault = new NamedAnalyzer("fake_default", new Analyzer() { + @Override + protected TokenStreamComponents createComponents(String fieldName) { + throw new UnsupportedOperationException("shouldn't be here"); + } + }); + // this is just a fake map that always returns the same value for any possible string key + // also the entrySet impl isn't fully correct but we implement it since internally + // IndexAnalyzers will iterate over all analyzers to close them. + final Map analyzerMap = new AbstractMap() { + @Override + public NamedAnalyzer get(Object key) { + assert key instanceof String : "key must be a string but was: " + key.getClass(); + return new NamedAnalyzer((String)key, fakeDefault.analyzer()); + } - try (AnalysisService analysisService = new FakeAnalysisService(indexSettings)) { - MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry, () -> null); + @Override + public Set> entrySet() { + // just to ensure we can iterate over this single analzyer + return Collections.singletonMap(fakeDefault.name(), fakeDefault).entrySet(); + } + }; + try (IndexAnalyzers fakeIndexAnalzyers = new IndexAnalyzers(indexSettings, fakeDefault, fakeDefault, fakeDefault, analyzerMap)) { + MapperService mapperService = new MapperService(indexSettings, fakeIndexAnalzyers, similarityService, mapperRegistry, () -> null); for (ObjectCursor cursor : indexMetaData.getMappings().values()) { MappingMetaData mappingMetaData = cursor.value; mapperService.merge(mappingMetaData.type(), mappingMetaData.source(), MapperService.MergeReason.MAPPING_RECOVERY, false); @@ -134,34 +158,6 @@ public class MetaDataIndexUpgradeService extends AbstractComponent { return IndexMetaData.builder(indexMetaData).settings(settings).build(); } - /** - * A fake analysis server that returns the same keyword analyzer for all requests - */ - private static class FakeAnalysisService extends AnalysisService { - - private Analyzer fakeAnalyzer = new Analyzer() { - @Override - protected TokenStreamComponents createComponents(String fieldName) { - throw new UnsupportedOperationException("shouldn't be here"); - } - }; - - public FakeAnalysisService(IndexSettings indexSettings) { - super(indexSettings, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); - } - - @Override - public NamedAnalyzer analyzer(String name) { - return new NamedAnalyzer(name, fakeAnalyzer); - } - - @Override - public void close() { - fakeAnalyzer.close(); - super.close(); - } - } - IndexMetaData archiveBrokenIndexSettings(IndexMetaData indexMetaData) { final Settings settings = indexMetaData.getSettings(); final Settings upgrade = indexScopedSettings.archiveUnknownOrBrokenSettings(settings); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java b/core/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java index fd8e7d02b28..58ee6d70f2c 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java @@ -31,7 +31,6 @@ import org.elasticsearch.cluster.routing.RecoverySource.LocalShardsRecoverySourc import org.elasticsearch.cluster.routing.RecoverySource.PeerRecoverySource; import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; import org.elasticsearch.cluster.routing.RecoverySource.StoreRecoverySource; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.collect.ImmutableOpenIntMap; import org.elasticsearch.common.io.stream.StreamInput; @@ -134,11 +133,22 @@ public class IndexRoutingTable extends AbstractDiffable imple throw new IllegalStateException("shard routing has an index [" + shardRouting.index() + "] that is different " + "from the routing table"); } + final Set inSyncAllocationIds = indexMetaData.inSyncAllocationIds(shardRouting.id()); if (shardRouting.active() && - indexMetaData.inSyncAllocationIds(shardRouting.id()).contains(shardRouting.allocationId().getId()) == false) { + inSyncAllocationIds.contains(shardRouting.allocationId().getId()) == false) { throw new IllegalStateException("active shard routing " + shardRouting + " has no corresponding entry in the in-sync " + - "allocation set " + indexMetaData.inSyncAllocationIds(shardRouting.id())); + "allocation set " + inSyncAllocationIds); } + + if (indexMetaData.getCreationVersion().onOrAfter(Version.V_5_0_0_alpha1) && + IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.getSettings()) == false && // see #20650 + shardRouting.primary() && shardRouting.initializing() && shardRouting.relocating() == false && + RecoverySource.isInitialRecovery(shardRouting.recoverySource().getType()) == false && + inSyncAllocationIds.contains(shardRouting.allocationId().getId()) == false) + throw new IllegalStateException("a primary shard routing " + shardRouting + " is a primary that is recovering from " + + "a known allocation id but has no corresponding entry in the in-sync " + + "allocation set " + inSyncAllocationIds); + } } return true; @@ -359,31 +369,28 @@ public class IndexRoutingTable extends AbstractDiffable imple * Initializes a new empty index, as if it was created from an API. */ public Builder initializeAsNew(IndexMetaData indexMetaData) { - RecoverySource primaryRecoverySource = indexMetaData.getMergeSourceIndex() != null ? - LocalShardsRecoverySource.INSTANCE : - StoreRecoverySource.EMPTY_STORE_INSTANCE; - return initializeEmpty(indexMetaData, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null), primaryRecoverySource); + return initializeEmpty(indexMetaData, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); } /** * Initializes an existing index. */ public Builder initializeAsRecovery(IndexMetaData indexMetaData) { - return initializeEmpty(indexMetaData, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null), null); + return initializeEmpty(indexMetaData, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null)); } /** * Initializes a new index caused by dangling index imported. */ public Builder initializeAsFromDangling(IndexMetaData indexMetaData) { - return initializeEmpty(indexMetaData, new UnassignedInfo(UnassignedInfo.Reason.DANGLING_INDEX_IMPORTED, null), null); + return initializeEmpty(indexMetaData, new UnassignedInfo(UnassignedInfo.Reason.DANGLING_INDEX_IMPORTED, null)); } /** * Initializes a new empty index, as as a result of opening a closed index. */ public Builder initializeAsFromCloseToOpen(IndexMetaData indexMetaData) { - return initializeEmpty(indexMetaData, new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, null), null); + return initializeEmpty(indexMetaData, new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, null)); } /** @@ -435,28 +442,36 @@ public class IndexRoutingTable extends AbstractDiffable imple /** * Initializes a new empty index, with an option to control if its from an API or not. - * - * @param primaryRecoverySource recovery source for primary shards. If null, it is automatically determined based on active - * allocation ids */ - private Builder initializeEmpty(IndexMetaData indexMetaData, UnassignedInfo unassignedInfo, @Nullable RecoverySource primaryRecoverySource) { + private Builder initializeEmpty(IndexMetaData indexMetaData, UnassignedInfo unassignedInfo) { assert indexMetaData.getIndex().equals(index); if (!shards.isEmpty()) { throw new IllegalStateException("trying to initialize an index with fresh shards, but already has shards created"); } for (int shardNumber = 0; shardNumber < indexMetaData.getNumberOfShards(); shardNumber++) { ShardId shardId = new ShardId(index, shardNumber); - if (primaryRecoverySource == null) { - if (indexMetaData.inSyncAllocationIds(shardNumber).isEmpty() && indexMetaData.getCreationVersion().onOrAfter(Version.V_5_0_0_alpha1)) { - primaryRecoverySource = indexMetaData.getMergeSourceIndex() != null ? LocalShardsRecoverySource.INSTANCE : StoreRecoverySource.EMPTY_STORE_INSTANCE; - } else { - primaryRecoverySource = StoreRecoverySource.EXISTING_STORE_INSTANCE; - } + final RecoverySource primaryRecoverySource; + if (indexMetaData.inSyncAllocationIds(shardNumber).isEmpty() == false) { + // we have previous valid copies for this shard. use them for recovery + primaryRecoverySource = StoreRecoverySource.EXISTING_STORE_INSTANCE; + } else if (indexMetaData.getCreationVersion().before(Version.V_5_0_0_alpha1) && + unassignedInfo.getReason() != UnassignedInfo.Reason.INDEX_CREATED // tests can create old indices + ) { + // the index is old and didn't maintain inSyncAllocationIds. Fall back to old behavior and require + // finding existing copies + primaryRecoverySource = StoreRecoverySource.EXISTING_STORE_INSTANCE; + } else if (indexMetaData.getMergeSourceIndex() != null) { + // this is a new index but the initial shards should merged from another index + primaryRecoverySource = LocalShardsRecoverySource.INSTANCE; + } else { + // a freshly created index with no restriction + primaryRecoverySource = StoreRecoverySource.EMPTY_STORE_INSTANCE; } IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); for (int i = 0; i <= indexMetaData.getNumberOfReplicas(); i++) { boolean primary = i == 0; - indexShardRoutingBuilder.addShard(ShardRouting.newUnassigned(shardId, primary, primary ? primaryRecoverySource : PeerRecoverySource.INSTANCE, unassignedInfo)); + indexShardRoutingBuilder.addShard(ShardRouting.newUnassigned(shardId, primary, + primary ? primaryRecoverySource : PeerRecoverySource.INSTANCE, unassignedInfo)); } shards.put(shardNumber, indexShardRoutingBuilder.build()); } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 77926fd7423..323adf78046 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -20,7 +20,6 @@ package org.elasticsearch.cluster.routing.allocation; import org.elasticsearch.cluster.ClusterInfoService; -import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterStateHealth; @@ -61,7 +60,6 @@ public class AllocationService extends AbstractComponent { private final GatewayAllocator gatewayAllocator; private final ShardsAllocator shardsAllocator; private final ClusterInfoService clusterInfoService; - private final ClusterName clusterName; @Inject public AllocationService(Settings settings, AllocationDeciders allocationDeciders, GatewayAllocator gatewayAllocator, @@ -71,7 +69,6 @@ public class AllocationService extends AbstractComponent { this.gatewayAllocator = gatewayAllocator; this.shardsAllocator = shardsAllocator; this.clusterInfoService = clusterInfoService; - clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings); } /** @@ -81,23 +78,17 @@ public class AllocationService extends AbstractComponent { * If the same instance of the {@link ClusterState} is returned, then no change has been made.

*/ public ClusterState applyStartedShards(ClusterState clusterState, List startedShards) { - return applyStartedShards(clusterState, startedShards, true); - } - - public ClusterState applyStartedShards(ClusterState clusterState, List startedShards, boolean withReroute) { if (startedShards.isEmpty()) { return clusterState; } RoutingNodes routingNodes = getMutableRoutingNodes(clusterState); // shuffle the unassigned nodes, just so we won't have things like poison failed shards routingNodes.unassigned().shuffle(); - StartedRerouteAllocation allocation = new StartedRerouteAllocation(allocationDeciders, routingNodes, clusterState, startedShards, - clusterInfoService.getClusterInfo(), currentNanoTime()); + RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState, + clusterInfoService.getClusterInfo(), currentNanoTime(), false); applyStartedShards(allocation, startedShards); - gatewayAllocator.applyStartedShards(allocation); - if (withReroute) { - reroute(allocation); - } + gatewayAllocator.applyStartedShards(allocation, startedShards); + reroute(allocation); String startedShardsAsString = firstListElementsToCommaDelimitedString(startedShards, s -> s.shardId().toString()); return buildResultAndLogHealthChange(clusterState, allocation, "shards started [" + startedShardsAsString + "] ..."); } @@ -123,11 +114,11 @@ public class AllocationService extends AbstractComponent { } public ClusterState applyFailedShard(ClusterState clusterState, ShardRouting failedShard) { - return applyFailedShards(clusterState, Collections.singletonList(new FailedRerouteAllocation.FailedShard(failedShard, null, null)), + return applyFailedShards(clusterState, Collections.singletonList(new FailedShard(failedShard, null, null)), Collections.emptyList()); } - public ClusterState applyFailedShards(ClusterState clusterState, List failedShards) { + public ClusterState applyFailedShards(ClusterState clusterState, List failedShards) { return applyFailedShards(clusterState, failedShards, Collections.emptyList()); } @@ -139,8 +130,8 @@ public class AllocationService extends AbstractComponent { *

* If the same instance of ClusterState is returned, then no change has been made.

*/ - public ClusterState applyFailedShards(final ClusterState clusterState, List failedShards, - List staleShards) { + public ClusterState applyFailedShards(final ClusterState clusterState, final List failedShards, + final List staleShards) { if (staleShards.isEmpty() && failedShards.isEmpty()) { return clusterState; } @@ -150,11 +141,11 @@ public class AllocationService extends AbstractComponent { // shuffle the unassigned nodes, just so we won't have things like poison failed shards routingNodes.unassigned().shuffle(); long currentNanoTime = currentNanoTime(); - FailedRerouteAllocation allocation = new FailedRerouteAllocation(allocationDeciders, routingNodes, tmpState, failedShards, - clusterInfoService.getClusterInfo(), currentNanoTime); + RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, tmpState, + clusterInfoService.getClusterInfo(), currentNanoTime, false); - for (FailedRerouteAllocation.FailedShard failedShardEntry : failedShards) { - ShardRouting shardToFail = failedShardEntry.routingEntry; + for (FailedShard failedShardEntry : failedShards) { + ShardRouting shardToFail = failedShardEntry.getRoutingEntry(); IndexMetaData indexMetaData = allocation.metaData().getIndexSafe(shardToFail.shardId().getIndex()); allocation.addIgnoreShardForNode(shardToFail.shardId(), shardToFail.currentNodeId()); // failing a primary also fails initializing replica shards, re-resolve ShardRouting @@ -165,18 +156,18 @@ public class AllocationService extends AbstractComponent { shardToFail.shardId(), shardToFail, failedShard); } int failedAllocations = failedShard.unassignedInfo() != null ? failedShard.unassignedInfo().getNumFailedAllocations() : 0; - UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, failedShardEntry.message, - failedShardEntry.failure, failedAllocations + 1, currentNanoTime, System.currentTimeMillis(), false, + UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, failedShardEntry.getMessage(), + failedShardEntry.getFailure(), failedAllocations + 1, currentNanoTime, System.currentTimeMillis(), false, AllocationStatus.NO_ATTEMPT); routingNodes.failShard(logger, failedShard, unassignedInfo, indexMetaData, allocation.changes()); } else { logger.trace("{} shard routing failed in an earlier iteration (routing: {})", shardToFail.shardId(), shardToFail); } } - gatewayAllocator.applyFailedShards(allocation); + gatewayAllocator.applyFailedShards(allocation, failedShards); reroute(allocation); - String failedShardsAsString = firstListElementsToCommaDelimitedString(failedShards, s -> s.routingEntry.shardId().toString()); + String failedShardsAsString = firstListElementsToCommaDelimitedString(failedShards, s -> s.getRoutingEntry().shardId().toString()); return buildResultAndLogHealthChange(clusterState, allocation, "shards failed [" + failedShardsAsString + "] ..."); } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedRerouteAllocation.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedRerouteAllocation.java deleted file mode 100644 index 8a31998a7dd..00000000000 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedRerouteAllocation.java +++ /dev/null @@ -1,87 +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.cluster.routing.allocation; - -import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.cluster.ClusterInfo; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.routing.RoutingNodes; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; -import org.elasticsearch.index.shard.ShardId; - -import java.util.List; - -/** - * This {@link RoutingAllocation} keeps a shard which routing - * allocation has failed. - */ -public class FailedRerouteAllocation extends RoutingAllocation { - - /** - * A failed shard with the shard routing itself and an optional - * details on why it failed. - */ - public static class FailedShard { - public final ShardRouting routingEntry; - public final String message; - public final Exception failure; - - public FailedShard(ShardRouting routingEntry, String message, Exception failure) { - assert routingEntry.assignedToNode() : "only assigned shards can be failed " + routingEntry; - this.routingEntry = routingEntry; - this.message = message; - this.failure = failure; - } - - @Override - public String toString() { - return "failed shard, shard " + routingEntry + ", message [" + message + "], failure [" + - ExceptionsHelper.detailedMessage(failure) + "]"; - } - } - - public static class StaleShard { - public final ShardId shardId; - public final String allocationId; - - public StaleShard(ShardId shardId, String allocationId) { - this.shardId = shardId; - this.allocationId = allocationId; - } - - @Override - public String toString() { - return "stale shard, shard " + shardId + ", alloc. id [" + allocationId + "]"; - } - } - - private final List failedShards; - - public FailedRerouteAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, ClusterState clusterState, - List failedShards, ClusterInfo clusterInfo, long currentNanoTime) { - super(deciders, routingNodes, clusterState, clusterInfo, currentNanoTime, false); - this.failedShards = failedShards; - } - - public List failedShards() { - return failedShards; - } -} diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedShard.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedShard.java new file mode 100644 index 00000000000..9bf9fa86d18 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedShard.java @@ -0,0 +1,69 @@ +/* + * 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.cluster.routing.allocation; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.Nullable; + +/** + * A class representing a failed shard. + */ +public class FailedShard { + private final ShardRouting routingEntry; + private final String message; + private final Exception failure; + + public FailedShard(ShardRouting routingEntry, String message, Exception failure) { + assert routingEntry.assignedToNode() : "only assigned shards can be failed " + routingEntry; + this.routingEntry = routingEntry; + this.message = message; + this.failure = failure; + } + + @Override + public String toString() { + return "failed shard, shard " + routingEntry + ", message [" + message + "], failure [" + + ExceptionsHelper.detailedMessage(failure) + "]"; + } + + /** + * The shard routing entry for the failed shard. + */ + public ShardRouting getRoutingEntry() { + return routingEntry; + } + + /** + * The failure message, if available, explaining why the shard failed. + */ + @Nullable + public String getMessage() { + return message; + } + + /** + * The exception, if present, causing the shard to fail. + */ + @Nullable + public Exception getFailure() { + return failure; + } +} diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java index edeeeea224a..f476972b216 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java @@ -27,7 +27,6 @@ import org.elasticsearch.cluster.routing.RoutingChangesObserver; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation.StaleShard; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; @@ -229,15 +228,15 @@ public class IndexMetaDataUpdater extends RoutingChangesObserver.AbstractRouting MetaData.Builder metaDataBuilder = null; // group staleShards entries by index for (Map.Entry> indexEntry : staleShards.stream().collect( - Collectors.groupingBy(fs -> fs.shardId.getIndex())).entrySet()) { + Collectors.groupingBy(fs -> fs.getShardId().getIndex())).entrySet()) { final IndexMetaData oldIndexMetaData = oldMetaData.getIndexSafe(indexEntry.getKey()); IndexMetaData.Builder indexMetaDataBuilder = null; // group staleShards entries by shard id for (Map.Entry> shardEntry : indexEntry.getValue().stream().collect( - Collectors.groupingBy(staleShard -> staleShard.shardId)).entrySet()) { + Collectors.groupingBy(staleShard -> staleShard.getShardId())).entrySet()) { int shardNumber = shardEntry.getKey().getId(); Set oldInSyncAllocations = oldIndexMetaData.inSyncAllocationIds(shardNumber); - Set idsToRemove = shardEntry.getValue().stream().map(e -> e.allocationId).collect(Collectors.toSet()); + Set idsToRemove = shardEntry.getValue().stream().map(e -> e.getAllocationId()).collect(Collectors.toSet()); assert idsToRemove.stream().allMatch(id -> oldRoutingTable.getByAllocationId(shardEntry.getKey(), id) == null) : "removing stale ids: " + idsToRemove + ", some of which have still a routing entry: " + oldRoutingTable.prettyPrint(); Set remainingInSyncAllocations = Sets.difference(oldInSyncAllocations, idsToRemove); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/StaleShard.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/StaleShard.java new file mode 100644 index 00000000000..9454f62db97 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/StaleShard.java @@ -0,0 +1,54 @@ +/* + * 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.cluster.routing.allocation; + +import org.elasticsearch.index.shard.ShardId; + +/** + * A class that represents a stale shard copy. + */ +public class StaleShard { + private final ShardId shardId; + private final String allocationId; + + public StaleShard(ShardId shardId, String allocationId) { + this.shardId = shardId; + this.allocationId = allocationId; + } + + @Override + public String toString() { + return "stale shard, shard " + shardId + ", alloc. id [" + allocationId + "]"; + } + + /** + * The shard id of the stale shard. + */ + public ShardId getShardId() { + return shardId; + } + + /** + * The allocation id of the stale shard. + */ + public String getAllocationId() { + return allocationId; + } +} diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/StartedRerouteAllocation.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/StartedRerouteAllocation.java deleted file mode 100644 index e63ce2b19e9..00000000000 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/StartedRerouteAllocation.java +++ /dev/null @@ -1,51 +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.cluster.routing.allocation; - -import org.elasticsearch.cluster.ClusterInfo; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.routing.RoutingNodes; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; - -import java.util.List; - -/** - * This {@link RoutingAllocation} holds a list of started shards within a - * cluster - */ -public class StartedRerouteAllocation extends RoutingAllocation { - - private final List startedShards; - - public StartedRerouteAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, ClusterState clusterState, - List startedShards, ClusterInfo clusterInfo, long currentNanoTime) { - super(deciders, routingNodes, clusterState, clusterInfo, currentNanoTime, false); - this.startedShards = startedShards; - } - - /** - * Get started shards - * @return list of started shards - */ - public List startedShards() { - return startedShards; - } -} diff --git a/core/src/main/java/org/elasticsearch/common/unit/ByteSizeUnit.java b/core/src/main/java/org/elasticsearch/common/unit/ByteSizeUnit.java index 4a159957d5e..7a412aac090 100644 --- a/core/src/main/java/org/elasticsearch/common/unit/ByteSizeUnit.java +++ b/core/src/main/java/org/elasticsearch/common/unit/ByteSizeUnit.java @@ -19,16 +19,20 @@ package org.elasticsearch.common.unit; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; + +import java.io.IOException; + /** * A SizeUnit represents size at a given unit of * granularity and provides utility methods to convert across units. * A SizeUnit does not maintain size information, but only * helps organize and use size representations that may be maintained * separately across various contexts. - * - * */ -public enum ByteSizeUnit { +public enum ByteSizeUnit implements Writeable { BYTES { @Override public long toBytes(long size) { @@ -225,6 +229,13 @@ public enum ByteSizeUnit { static final long MAX = Long.MAX_VALUE; + public static ByteSizeUnit fromId(int id) { + if (id < 0 || id >= values().length) { + throw new IllegalArgumentException("No byte size unit found for id [" + id + "]"); + } + return values()[id]; + } + /** * Scale d by m, checking for overflow. * This has a short name to make above code more readable. @@ -235,7 +246,6 @@ public enum ByteSizeUnit { return d * m; } - public abstract long toBytes(long size); public abstract long toKB(long size); @@ -247,4 +257,16 @@ public enum ByteSizeUnit { public abstract long toTB(long size); public abstract long toPB(long size); + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(this.ordinal()); + } + + /** + * Reads a {@link ByteSizeUnit} from a given {@link StreamInput} + */ + public static ByteSizeUnit readFrom(StreamInput in) throws IOException { + return ByteSizeUnit.fromId(in.readVInt()); + } } \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/common/unit/ByteSizeValue.java b/core/src/main/java/org/elasticsearch/common/unit/ByteSizeValue.java index db66885a974..7d2be6fee3e 100644 --- a/core/src/main/java/org/elasticsearch/common/unit/ByteSizeValue.java +++ b/core/src/main/java/org/elasticsearch/common/unit/ByteSizeValue.java @@ -32,11 +32,11 @@ import java.util.Objects; public class ByteSizeValue implements Writeable { private final long size; - private final ByteSizeUnit sizeUnit; + private final ByteSizeUnit unit; public ByteSizeValue(StreamInput in) throws IOException { size = in.readVLong(); - sizeUnit = ByteSizeUnit.BYTES; + unit = ByteSizeUnit.BYTES; } @Override @@ -48,9 +48,9 @@ public class ByteSizeValue implements Writeable { this(bytes, ByteSizeUnit.BYTES); } - public ByteSizeValue(long size, ByteSizeUnit sizeUnit) { + public ByteSizeValue(long size, ByteSizeUnit unit) { this.size = size; - this.sizeUnit = sizeUnit; + this.unit = unit; } public int bytesAsInt() { @@ -62,27 +62,27 @@ public class ByteSizeValue implements Writeable { } public long getBytes() { - return sizeUnit.toBytes(size); + return unit.toBytes(size); } public long getKb() { - return sizeUnit.toKB(size); + return unit.toKB(size); } public long getMb() { - return sizeUnit.toMB(size); + return unit.toMB(size); } public long getGb() { - return sizeUnit.toGB(size); + return unit.toGB(size); } public long getTb() { - return sizeUnit.toTB(size); + return unit.toTB(size); } public long getPb() { - return sizeUnit.toPB(size); + return unit.toPB(size); } public double getKbFrac() { @@ -199,7 +199,7 @@ public class ByteSizeValue implements Writeable { @Override public int hashCode() { int result = Long.hashCode(size); - result = 31 * result + (sizeUnit != null ? sizeUnit.hashCode() : 0); + result = 31 * result + (unit != null ? unit.hashCode() : 0); return result; } } diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/AbstractObjectParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/AbstractObjectParser.java index 6f8a606d9a6..a623a86c9b1 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/AbstractObjectParser.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/AbstractObjectParser.java @@ -60,6 +60,9 @@ public abstract class AbstractObjectParser void declareField(BiConsumer consumer, NoContextParser parser, ParseField parseField, ValueType type) { + if (parser == null) { + throw new IllegalArgumentException("[parser] is required"); + } declareField(consumer, (p, c) -> parser.parse(p), parseField, type); } diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/ConstructingObjectParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/ConstructingObjectParser.java index e1400463a72..b8a42cd1e13 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/ConstructingObjectParser.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/ConstructingObjectParser.java @@ -103,7 +103,7 @@ public final class ConstructingObjectParser builder) { - objectParser = new ObjectParser<>(name); + this(name, false, builder); + } + + /** + * Build the parser. + * + * @param name The name given to the delegate ObjectParser for error identification. Use what you'd use if the object worked with + * ObjectParser. + * @param ignoreUnknownFields Should this parser ignore unknown fields? This should generally be set to true only when parsing responses + * from external systems, never when parsing requests from users. + * @param builder A function that builds the object from an array of Objects. Declare this inline with the parser, casting the elements + * of the array to the arguments so they work with your favorite constructor. The objects in the array will be in the same order + * that you declared the {{@link #constructorArg()}s and none will be null. If any of the constructor arguments aren't defined in + * the XContent then parsing will throw an error. We use an array here rather than a {@code Map} to save on + * allocations. + */ + public ConstructingObjectParser(String name, boolean ignoreUnknownFields, Function builder) { + objectParser = new ObjectParser<>(name, ignoreUnknownFields, null); this.builder = builder; } @@ -153,6 +170,19 @@ public final class ConstructingObjectParser void declareField(BiConsumer consumer, ContextParser parser, ParseField parseField, ValueType type) { + if (consumer == null) { + throw new IllegalArgumentException("[consumer] is required"); + } + if (parser == null) { + throw new IllegalArgumentException("[parser] is required"); + } + if (parseField == null) { + throw new IllegalArgumentException("[parseField] is required"); + } + if (type == null) { + throw new IllegalArgumentException("[type] is required"); + } + if (consumer == REQUIRED_CONSTRUCTOR_ARG_MARKER || consumer == OPTIONAL_CONSTRUCTOR_ARG_MARKER) { /* * Constructor arguments are detected by this "marker" consumer. It keeps the API looking clean even if it is a bit sleezy. We diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java index 44d9e6e1993..2abd6e66df8 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.common.xcontent; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParseFieldMatcherSupplier; @@ -83,6 +84,11 @@ public final class ObjectParser fieldParserMap = new HashMap<>(); private final String name; private final Supplier valueSupplier; + /** + * Should this parser ignore unknown fields? This should generally be set to true only when parsing responses from external systems, + * never when parsing requests from users. + */ + private final boolean ignoreUnknownFields; /** * Creates a new ObjectParser instance with a name. This name is used to reference the parser in exceptions and messages. @@ -96,9 +102,21 @@ public final class ObjectParser valueSupplier) { + public ObjectParser(String name, @Nullable Supplier valueSupplier) { + this(name, false, valueSupplier); + } + + /** + * Creates a new ObjectParser instance which a name. + * @param name the parsers name, used to reference the parser in exceptions and messages. + * @param ignoreUnknownFields Should this parser ignore unknown fields? This should generally be set to true only when parsing + * responses from external systems, never when parsing requests from users. + * @param valueSupplier a supplier that creates a new Value instance used when the parser is used as an inner object parser. + */ + public ObjectParser(String name, boolean ignoreUnknownFields, @Nullable Supplier valueSupplier) { this.name = name; this.valueSupplier = valueSupplier; + this.ignoreUnknownFields = ignoreUnknownFields; } /** @@ -144,9 +162,13 @@ public final class ObjectParser p, ParseField parseField, ValueType type) { + if (parseField == null) { + throw new IllegalArgumentException("[parseField] is required"); + } + if (type == null) { + throw new IllegalArgumentException("[type] is required"); + } FieldParser fieldParser = new FieldParser(p, type.supportedTokens(), parseField, type); for (String fieldValue : parseField.getAllNamesIncludedDeprecated()) { fieldParserMap.putIfAbsent(fieldValue, fieldParser); @@ -178,6 +206,12 @@ public final class ObjectParser void declareField(BiConsumer consumer, ContextParser parser, ParseField parseField, ValueType type) { + if (consumer == null) { + throw new IllegalArgumentException("[consumer] is required"); + } + if (parser == null) { + throw new IllegalArgumentException("[parser] is required"); + } declareField((p, v, c) -> consumer.accept(v, parser.parse(p, c)), parseField, type); } @@ -362,7 +396,7 @@ public final class ObjectParser parser = fieldParserMap.get(fieldName); - if (parser == null) { + if (parser == null && false == ignoreUnknownFields) { throw new IllegalArgumentException("[" + name + "] unknown field [" + fieldName + "], parser not found"); } return parser; diff --git a/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java b/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java index c84a9c3378a..450255575d9 100644 --- a/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java +++ b/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java @@ -28,9 +28,8 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingService; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; +import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; -import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; @@ -40,6 +39,7 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData; +import java.util.List; import java.util.concurrent.ConcurrentMap; /** @@ -115,21 +115,28 @@ public class GatewayAllocator extends AbstractComponent { return count; } - public void applyStartedShards(StartedRerouteAllocation allocation) { - for (ShardRouting shard : allocation.startedShards()) { - Releasables.close(asyncFetchStarted.remove(shard.shardId())); - Releasables.close(asyncFetchStore.remove(shard.shardId())); + public void applyStartedShards(final RoutingAllocation allocation, final List startedShards) { + for (ShardRouting startedShard : startedShards) { + Releasables.close(asyncFetchStarted.remove(startedShard.shardId())); + Releasables.close(asyncFetchStore.remove(startedShard.shardId())); } } - public void applyFailedShards(FailedRerouteAllocation allocation) { - for (FailedRerouteAllocation.FailedShard shard : allocation.failedShards()) { - Releasables.close(asyncFetchStarted.remove(shard.routingEntry.shardId())); - Releasables.close(asyncFetchStore.remove(shard.routingEntry.shardId())); + public void applyFailedShards(final RoutingAllocation allocation, final List failedShards) { + for (FailedShard failedShard : failedShards) { + Releasables.close(asyncFetchStarted.remove(failedShard.getRoutingEntry().shardId())); + Releasables.close(asyncFetchStore.remove(failedShard.getRoutingEntry().shardId())); } } public void allocateUnassigned(final RoutingAllocation allocation) { + innerAllocatedUnassigned(allocation, primaryShardAllocator, replicaShardAllocator); + } + + // allow for testing infra to change shard allocators implementation + protected static void innerAllocatedUnassigned(RoutingAllocation allocation, + PrimaryShardAllocator primaryShardAllocator, + ReplicaShardAllocator replicaShardAllocator) { RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); unassigned.sort(PriorityComparator.getAllocationComparator(allocation)); // sort for priority ordering diff --git a/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java b/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java index 38afa11f5bd..25ae3b7cce9 100644 --- a/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java +++ b/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java @@ -138,7 +138,8 @@ public abstract class PrimaryShardAllocator extends BaseGatewayShardAllocator { if (inSyncAllocationIds.isEmpty()) { assert Version.indexCreated(indexMetaData.getSettings()).before(Version.V_5_0_0_alpha1) : - "trying to allocated a primary with an empty allocation id set, but index is new"; + "trying to allocate a primary with an empty in sync allocation id set, but index is new. index: " + + indexMetaData.getIndex(); // 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 @@ -257,11 +258,11 @@ public abstract class PrimaryShardAllocator extends BaseGatewayShardAllocator { /** * Builds a list of nodes. If matchAnyShard is set to false, only nodes that have an allocation id matching - * lastActiveAllocationIds are added to the list. Otherwise, any node that has a shard is added to the list, but + * inSyncAllocationIds 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 static NodeShardsResult buildAllocationIdBasedNodeShardsResult(ShardRouting shard, boolean matchAnyShard, - Set ignoreNodes, Set lastActiveAllocationIds, + Set ignoreNodes, Set inSyncAllocationIds, FetchResult shardState, Logger logger) { LinkedList matchingNodeShardStates = new LinkedList<>(); @@ -292,7 +293,7 @@ public abstract class PrimaryShardAllocator extends BaseGatewayShardAllocator { if (allocationId != null) { numberOfAllocationsFound++; - if (lastActiveAllocationIds.contains(allocationId)) { + if (inSyncAllocationIds.contains(allocationId)) { if (nodeShardState.primary()) { matchingNodeShardStates.addFirst(nodeShardState); } else { diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index e662e46c79d..58457417fa7 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -43,7 +43,7 @@ import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.ShardLock; import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.index.analysis.AnalysisRegistry; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.cache.query.QueryCache; @@ -97,7 +97,7 @@ import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder; public class IndexService extends AbstractIndexComponent implements IndicesClusterStateService.AllocatedIndex { private final IndexEventListener eventListener; - private final AnalysisService analysisService; + private final IndexAnalyzers indexAnalyzers; private final IndexFieldDataService indexFieldData; private final BitsetFilterCache bitsetFilterCache; private final NodeEnvironment nodeEnv; @@ -137,9 +137,9 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust List indexingOperationListeners) throws IOException { super(indexSettings); this.indexSettings = indexSettings; - this.analysisService = registry.build(indexSettings); + this.indexAnalyzers = registry.build(indexSettings); this.similarityService = similarityService; - this.mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry, + this.mapperService = new MapperService(indexSettings, indexAnalyzers, similarityService, mapperRegistry, IndexService.this::newQueryShardContext); this.indexFieldData = new IndexFieldDataService(indexSettings, indicesFieldDataCache, nodeServicesProvider.getCircuitBreakerService(), mapperService); @@ -214,8 +214,8 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust return indexFieldData; } - public AnalysisService analysisService() { - return this.analysisService; + public IndexAnalyzers getIndexAnalyzers() { + return this.indexAnalyzers; } public MapperService mapperService() { @@ -239,7 +239,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust } } } finally { - IOUtils.close(bitsetFilterCache, indexCache, indexFieldData, analysisService, refreshTask, fsyncTask); + IOUtils.close(bitsetFilterCache, indexCache, indexFieldData, indexAnalyzers, refreshTask, fsyncTask); } } } diff --git a/core/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java b/core/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java index 5ba11cd3542..1d2098d4390 100644 --- a/core/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java +++ b/core/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java @@ -18,14 +18,21 @@ */ package org.elasticsearch.index.analysis; +import org.apache.logging.log4j.Logger; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.util.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.logging.DeprecationLogger; +import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.env.Environment; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.indices.analysis.AnalysisModule; import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider; import org.elasticsearch.indices.analysis.PreBuiltAnalyzers; @@ -39,6 +46,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Locale; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; @@ -46,7 +54,7 @@ import static java.util.Collections.unmodifiableMap; /** * An internal registry for tokenizer, token filter, char filter and analyzer. - * This class exists per node and allows to create per-index {@link AnalysisService} via {@link #build(IndexSettings)} + * This class exists per node and allows to create per-index {@link IndexAnalyzers} via {@link #build(IndexSettings)} */ public final class AnalysisRegistry implements Closeable { public static final String INDEX_ANALYSIS_CHAR_FILTER = "index.analysis.char_filter"; @@ -136,17 +144,19 @@ public final class AnalysisRegistry implements Closeable { } /** - * Creates an index-level {@link AnalysisService} from this registry using the given index settings + * Creates an index-level {@link IndexAnalyzers} from this registry using the given index settings */ - public AnalysisService build(IndexSettings indexSettings) throws IOException { - final Map charFiltersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_CHAR_FILTER); + public IndexAnalyzers build(IndexSettings indexSettings) throws IOException { + + final Map charFilterFactories = buildCharFilterFactories(indexSettings); + final Map tokenizerFactories = buildTokenizerFactories(indexSettings); + final Map tokenFilterFactories = buildTokenFilterFactories(indexSettings); + final Map> analyzierFactories = buildAnalyzerFactories(indexSettings); + return build(indexSettings, analyzierFactories, tokenizerFactories, charFilterFactories, tokenFilterFactories); + } + + public Map buildTokenFilterFactories(IndexSettings indexSettings) throws IOException { final Map tokenFiltersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_FILTER); - final Map tokenizersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_TOKENIZER); - final Map analyzersSettings = indexSettings.getSettings().getGroups("index.analysis.analyzer"); - - final Map charFilterFactories = buildMapping(false, "charfilter", indexSettings, charFiltersSettings, charFilters, prebuiltAnalysis.charFilterFactories); - final Map tokenizerFactories = buildMapping(false, "tokenizer", indexSettings, tokenizersSettings, tokenizers, prebuiltAnalysis.tokenizerFactories); - Map> tokenFilters = new HashMap<>(this.tokenFilters); /* * synonym is different than everything else since it needs access to the tokenizer factories for this index. @@ -154,10 +164,22 @@ public final class AnalysisRegistry implements Closeable { * hide internal data-structures as much as possible. */ tokenFilters.put("synonym", requriesAnalysisSettings((is, env, name, settings) -> new SynonymTokenFilterFactory(is, env, this, name, settings))); - final Map tokenFilterFactories = buildMapping(false, "tokenfilter", indexSettings, tokenFiltersSettings, Collections.unmodifiableMap(tokenFilters), prebuiltAnalysis.tokenFilterFactories); - final Map> analyzierFactories = buildMapping(true, "analyzer", indexSettings, analyzersSettings, - analyzers, prebuiltAnalysis.analyzerProviderFactories); - return new AnalysisService(indexSettings, analyzierFactories, tokenizerFactories, charFilterFactories, tokenFilterFactories); + return buildMapping(false, "tokenfilter", indexSettings, tokenFiltersSettings, Collections.unmodifiableMap(tokenFilters), prebuiltAnalysis.tokenFilterFactories); + } + + public Map buildTokenizerFactories(IndexSettings indexSettings) throws IOException { + final Map tokenizersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_TOKENIZER); + return buildMapping(false, "tokenizer", indexSettings, tokenizersSettings, tokenizers, prebuiltAnalysis.tokenizerFactories); + } + + public Map buildCharFilterFactories(IndexSettings indexSettings) throws IOException { + final Map charFiltersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_CHAR_FILTER); + return buildMapping(false, "charfilter", indexSettings, charFiltersSettings, charFilters, prebuiltAnalysis.charFilterFactories); + } + + public Map> buildAnalyzerFactories(IndexSettings indexSettings) throws IOException { + final Map analyzersSettings = indexSettings.getSettings().getGroups("index.analysis.analyzer"); + return buildMapping(true, "analyzer", indexSettings, analyzersSettings, analyzers, prebuiltAnalysis.analyzerProviderFactories); } /** @@ -399,4 +421,132 @@ public final class AnalysisRegistry implements Closeable { IOUtils.close(analyzerProviderFactories.values().stream().map((a) -> ((PreBuiltAnalyzerProviderFactory)a).analyzer()).collect(Collectors.toList())); } } + + public IndexAnalyzers build(IndexSettings indexSettings, + Map> analyzerProviders, + Map tokenizerFactoryFactories, + Map charFilterFactoryFactories, + Map tokenFilterFactoryFactories) { + + Index index = indexSettings.getIndex(); + analyzerProviders = new HashMap<>(analyzerProviders); + Logger logger = Loggers.getLogger(getClass(), indexSettings.getSettings()); + DeprecationLogger deprecationLogger = new DeprecationLogger(logger); + Map analyzerAliases = new HashMap<>(); + Map analyzers = new HashMap<>(); + for (Map.Entry> entry : analyzerProviders.entrySet()) { + processAnalyzerFactory(deprecationLogger, indexSettings, entry.getKey(), entry.getValue(), analyzerAliases, analyzers, + tokenFilterFactoryFactories, charFilterFactoryFactories, tokenizerFactoryFactories); + } + for (Map.Entry entry : analyzerAliases.entrySet()) { + String key = entry.getKey(); + if (analyzers.containsKey(key) && + ("default".equals(key) || "default_search".equals(key) || "default_search_quoted".equals(key)) == false) { + throw new IllegalStateException("already registered analyzer with name: " + key); + } else { + NamedAnalyzer configured = entry.getValue(); + analyzers.put(key, configured); + } + } + + if (!analyzers.containsKey("default")) { + processAnalyzerFactory(deprecationLogger, indexSettings, "default", new StandardAnalyzerProvider(indexSettings, null, "default", Settings.Builder.EMPTY_SETTINGS), + analyzerAliases, analyzers, tokenFilterFactoryFactories, charFilterFactoryFactories, tokenizerFactoryFactories); + } + if (!analyzers.containsKey("default_search")) { + analyzers.put("default_search", analyzers.get("default")); + } + if (!analyzers.containsKey("default_search_quoted")) { + analyzers.put("default_search_quoted", analyzers.get("default_search")); + } + + + NamedAnalyzer defaultAnalyzer = analyzers.get("default"); + if (defaultAnalyzer == null) { + throw new IllegalArgumentException("no default analyzer configured"); + } + if (analyzers.containsKey("default_index")) { + final Version createdVersion = indexSettings.getIndexVersionCreated(); + if (createdVersion.onOrAfter(Version.V_5_0_0_alpha1)) { + throw new IllegalArgumentException("setting [index.analysis.analyzer.default_index] is not supported anymore, use [index.analysis.analyzer.default] instead for index [" + index.getName() + "]"); + } else { + deprecationLogger.deprecated("setting [index.analysis.analyzer.default_index] is deprecated, use [index.analysis.analyzer.default] instead for index [{}]", index.getName()); + } + } + NamedAnalyzer defaultIndexAnalyzer = analyzers.containsKey("default_index") ? analyzers.get("default_index") : defaultAnalyzer; + NamedAnalyzer defaultSearchAnalyzer = analyzers.containsKey("default_search") ? analyzers.get("default_search") : defaultAnalyzer; + NamedAnalyzer defaultSearchQuoteAnalyzer = analyzers.containsKey("default_search_quote") ? analyzers.get("default_search_quote") : defaultSearchAnalyzer; + + for (Map.Entry analyzer : analyzers.entrySet()) { + if (analyzer.getKey().startsWith("_")) { + throw new IllegalArgumentException("analyzer name must not start with '_'. got \"" + analyzer.getKey() + "\""); + } + } + return new IndexAnalyzers(indexSettings, defaultIndexAnalyzer, defaultSearchAnalyzer, defaultSearchQuoteAnalyzer, + unmodifiableMap(analyzers)); + } + + private void processAnalyzerFactory(DeprecationLogger deprecationLogger, + IndexSettings indexSettings, + String name, + AnalyzerProvider analyzerFactory, + Map analyzerAliases, + Map analyzers, Map tokenFilters, + Map charFilters, Map tokenizers) { + /* + * Lucene defaults positionIncrementGap to 0 in all analyzers but + * Elasticsearch defaults them to 0 only before version 2.0 + * and 100 afterwards so we override the positionIncrementGap if it + * doesn't match here. + */ + int overridePositionIncrementGap = TextFieldMapper.Defaults.POSITION_INCREMENT_GAP; + if (analyzerFactory instanceof CustomAnalyzerProvider) { + ((CustomAnalyzerProvider) analyzerFactory).build(tokenizers, charFilters, tokenFilters); + /* + * Custom analyzers already default to the correct, version + * dependent positionIncrementGap and the user is be able to + * configure the positionIncrementGap directly on the analyzer so + * we disable overriding the positionIncrementGap to preserve the + * user's setting. + */ + overridePositionIncrementGap = Integer.MIN_VALUE; + } + Analyzer analyzerF = analyzerFactory.get(); + if (analyzerF == null) { + throw new IllegalArgumentException("analyzer [" + analyzerFactory.name() + "] created null analyzer"); + } + NamedAnalyzer analyzer; + if (analyzerF instanceof NamedAnalyzer) { + // if we got a named analyzer back, use it... + analyzer = (NamedAnalyzer) analyzerF; + if (overridePositionIncrementGap >= 0 && analyzer.getPositionIncrementGap(analyzer.name()) != overridePositionIncrementGap) { + // unless the positionIncrementGap needs to be overridden + analyzer = new NamedAnalyzer(analyzer, overridePositionIncrementGap); + } + } else { + analyzer = new NamedAnalyzer(name, analyzerFactory.scope(), analyzerF, overridePositionIncrementGap); + } + if (analyzers.containsKey(name)) { + throw new IllegalStateException("already registered analyzer with name: " + name); + } + analyzers.put(name, analyzer); + // TODO: remove alias support completely when we no longer support pre 5.0 indices + final String analyzerAliasKey = "index.analysis.analyzer." + analyzerFactory.name() + ".alias"; + if (indexSettings.getSettings().get(analyzerAliasKey) != null) { + if (indexSettings.getIndexVersionCreated().onOrAfter(Version.V_5_0_0_alpha6)) { + // do not allow alias creation if the index was created on or after v5.0 alpha6 + throw new IllegalArgumentException("setting [" + analyzerAliasKey + "] is not supported"); + } + + // the setting is now removed but we only support it for loading indices created before v5.0 + deprecationLogger.deprecated("setting [{}] is only allowed on index [{}] because it was created before 5.x; " + + "analyzer aliases can no longer be created on new indices.", analyzerAliasKey, indexSettings.getIndex().getName()); + Set aliases = Sets.newHashSet(indexSettings.getSettings().getAsArray(analyzerAliasKey)); + for (String alias : aliases) { + if (analyzerAliases.putIfAbsent(alias, analyzer) != null) { + throw new IllegalStateException("alias [" + alias + "] is already used by [" + analyzerAliases.get(alias).name() + "]"); + } + } + } + } } diff --git a/core/src/main/java/org/elasticsearch/index/analysis/AnalysisService.java b/core/src/main/java/org/elasticsearch/index/analysis/AnalysisService.java deleted file mode 100644 index cb84e6c6d0a..00000000000 --- a/core/src/main/java/org/elasticsearch/index/analysis/AnalysisService.java +++ /dev/null @@ -1,218 +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.index.analysis; - -import org.apache.lucene.analysis.Analyzer; -import org.elasticsearch.Version; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.set.Sets; -import org.elasticsearch.index.AbstractIndexComponent; -import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.mapper.TextFieldMapper; - -import java.io.Closeable; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -import static java.util.Collections.unmodifiableMap; - -/** - * - */ -public class AnalysisService extends AbstractIndexComponent implements Closeable { - - private final Map analyzers; - private final Map tokenizers; - private final Map charFilters; - private final Map tokenFilters; - - private final NamedAnalyzer defaultIndexAnalyzer; - private final NamedAnalyzer defaultSearchAnalyzer; - private final NamedAnalyzer defaultSearchQuoteAnalyzer; - - public AnalysisService(IndexSettings indexSettings, - Map> analyzerProviders, - Map tokenizerFactoryFactories, - Map charFilterFactoryFactories, - Map tokenFilterFactoryFactories) { - super(indexSettings); - this.tokenizers = unmodifiableMap(tokenizerFactoryFactories); - this.charFilters = unmodifiableMap(charFilterFactoryFactories); - this.tokenFilters = unmodifiableMap(tokenFilterFactoryFactories); - analyzerProviders = new HashMap<>(analyzerProviders); - - Map analyzerAliases = new HashMap<>(); - Map analyzers = new HashMap<>(); - for (Map.Entry> entry : analyzerProviders.entrySet()) { - processAnalyzerFactory(entry.getKey(), entry.getValue(), analyzerAliases, analyzers); - } - for (Map.Entry entry : analyzerAliases.entrySet()) { - String key = entry.getKey(); - if (analyzers.containsKey(key) && - ("default".equals(key) || "default_search".equals(key) || "default_search_quoted".equals(key)) == false) { - throw new IllegalStateException("already registered analyzer with name: " + key); - } else { - NamedAnalyzer configured = entry.getValue(); - analyzers.put(key, configured); - } - } - - if (!analyzers.containsKey("default")) { - processAnalyzerFactory("default", new StandardAnalyzerProvider(indexSettings, null, "default", Settings.Builder.EMPTY_SETTINGS), - analyzerAliases, analyzers); - } - if (!analyzers.containsKey("default_search")) { - analyzers.put("default_search", analyzers.get("default")); - } - if (!analyzers.containsKey("default_search_quoted")) { - analyzers.put("default_search_quoted", analyzers.get("default_search")); - } - - - NamedAnalyzer defaultAnalyzer = analyzers.get("default"); - if (defaultAnalyzer == null) { - throw new IllegalArgumentException("no default analyzer configured"); - } - if (analyzers.containsKey("default_index")) { - final Version createdVersion = indexSettings.getIndexVersionCreated(); - if (createdVersion.onOrAfter(Version.V_5_0_0_alpha1)) { - throw new IllegalArgumentException("setting [index.analysis.analyzer.default_index] is not supported anymore, use [index.analysis.analyzer.default] instead for index [" + index().getName() + "]"); - } else { - deprecationLogger.deprecated("setting [index.analysis.analyzer.default_index] is deprecated, use [index.analysis.analyzer.default] instead for index [{}]", index().getName()); - } - } - defaultIndexAnalyzer = analyzers.containsKey("default_index") ? analyzers.get("default_index") : defaultAnalyzer; - defaultSearchAnalyzer = analyzers.containsKey("default_search") ? analyzers.get("default_search") : defaultAnalyzer; - defaultSearchQuoteAnalyzer = analyzers.containsKey("default_search_quote") ? analyzers.get("default_search_quote") : defaultSearchAnalyzer; - - for (Map.Entry analyzer : analyzers.entrySet()) { - if (analyzer.getKey().startsWith("_")) { - throw new IllegalArgumentException("analyzer name must not start with '_'. got \"" + analyzer.getKey() + "\""); - } - } - this.analyzers = unmodifiableMap(analyzers); - } - - private void processAnalyzerFactory(String name, AnalyzerProvider analyzerFactory, Map analyzerAliases, Map analyzers) { - /* - * Lucene defaults positionIncrementGap to 0 in all analyzers but - * Elasticsearch defaults them to 0 only before version 2.0 - * and 100 afterwards so we override the positionIncrementGap if it - * doesn't match here. - */ - int overridePositionIncrementGap = TextFieldMapper.Defaults.POSITION_INCREMENT_GAP; - if (analyzerFactory instanceof CustomAnalyzerProvider) { - ((CustomAnalyzerProvider) analyzerFactory).build(this); - /* - * Custom analyzers already default to the correct, version - * dependent positionIncrementGap and the user is be able to - * configure the positionIncrementGap directly on the analyzer so - * we disable overriding the positionIncrementGap to preserve the - * user's setting. - */ - overridePositionIncrementGap = Integer.MIN_VALUE; - } - Analyzer analyzerF = analyzerFactory.get(); - if (analyzerF == null) { - throw new IllegalArgumentException("analyzer [" + analyzerFactory.name() + "] created null analyzer"); - } - NamedAnalyzer analyzer; - if (analyzerF instanceof NamedAnalyzer) { - // if we got a named analyzer back, use it... - analyzer = (NamedAnalyzer) analyzerF; - if (overridePositionIncrementGap >= 0 && analyzer.getPositionIncrementGap(analyzer.name()) != overridePositionIncrementGap) { - // unless the positionIncrementGap needs to be overridden - analyzer = new NamedAnalyzer(analyzer, overridePositionIncrementGap); - } - } else { - analyzer = new NamedAnalyzer(name, analyzerFactory.scope(), analyzerF, overridePositionIncrementGap); - } - if (analyzers.containsKey(name)) { - throw new IllegalStateException("already registered analyzer with name: " + name); - } - analyzers.put(name, analyzer); - // TODO: remove alias support completely when we no longer support pre 5.0 indices - final String analyzerAliasKey = "index.analysis.analyzer." + analyzerFactory.name() + ".alias"; - if (indexSettings.getSettings().get(analyzerAliasKey) != null) { - if (indexSettings.getIndexVersionCreated().onOrAfter(Version.V_5_0_0_alpha6)) { - // do not allow alias creation if the index was created on or after v5.0 alpha6 - throw new IllegalArgumentException("setting [" + analyzerAliasKey + "] is not supported"); - } - - // the setting is now removed but we only support it for loading indices created before v5.0 - deprecationLogger.deprecated("setting [{}] is only allowed on index [{}] because it was created before 5.x; " + - "analyzer aliases can no longer be created on new indices.", analyzerAliasKey, index().getName()); - Set aliases = Sets.newHashSet(indexSettings.getSettings().getAsArray(analyzerAliasKey)); - for (String alias : aliases) { - if (analyzerAliases.putIfAbsent(alias, analyzer) != null) { - throw new IllegalStateException("alias [" + alias + "] is already used by [" + analyzerAliases.get(alias).name() + "]"); - } - } - } - } - - @Override - public void close() { - for (NamedAnalyzer analyzer : analyzers.values()) { - if (analyzer.scope() == AnalyzerScope.INDEX) { - try { - analyzer.close(); - } catch (NullPointerException e) { - // because analyzers are aliased, they might be closed several times - // an NPE is thrown in this case, so ignore.... - // TODO: Analyzer's can no longer have aliases in indices created in 5.x and beyond, - // so we only allow the aliases for analyzers on indices created pre 5.x for backwards - // compatibility. Once pre 5.0 indices are no longer supported, this check should be removed. - } catch (Exception e) { - logger.debug("failed to close analyzer {}", analyzer); - } - } - } - } - - public NamedAnalyzer analyzer(String name) { - return analyzers.get(name); - } - - public NamedAnalyzer defaultIndexAnalyzer() { - return defaultIndexAnalyzer; - } - - public NamedAnalyzer defaultSearchAnalyzer() { - return defaultSearchAnalyzer; - } - - public NamedAnalyzer defaultSearchQuoteAnalyzer() { - return defaultSearchQuoteAnalyzer; - } - - public TokenizerFactory tokenizer(String name) { - return tokenizers.get(name); - } - - public CharFilterFactory charFilter(String name) { - return charFilters.get(name); - } - - public TokenFilterFactory tokenFilter(String name) { - return tokenFilters.get(name); - } -} diff --git a/core/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java b/core/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java index 144cbe81743..63861e80849 100644 --- a/core/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java +++ b/core/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java @@ -26,6 +26,7 @@ import org.elasticsearch.index.mapper.TextFieldMapper; import java.util.ArrayList; import java.util.List; +import java.util.Map; /** * A custom analyzer that is built out of a single {@link org.apache.lucene.analysis.Tokenizer} and a list @@ -43,35 +44,36 @@ public class CustomAnalyzerProvider extends AbstractIndexAnalyzerProvider tokenizers, final Map charFilters, + final Map tokenFilters) { String tokenizerName = analyzerSettings.get("tokenizer"); if (tokenizerName == null) { throw new IllegalArgumentException("Custom Analyzer [" + name() + "] must be configured with a tokenizer"); } - TokenizerFactory tokenizer = analysisService.tokenizer(tokenizerName); + TokenizerFactory tokenizer = tokenizers.get(tokenizerName); if (tokenizer == null) { throw new IllegalArgumentException("Custom Analyzer [" + name() + "] failed to find tokenizer under name [" + tokenizerName + "]"); } - List charFilters = new ArrayList<>(); + List charFiltersList = new ArrayList<>(); String[] charFilterNames = analyzerSettings.getAsArray("char_filter"); for (String charFilterName : charFilterNames) { - CharFilterFactory charFilter = analysisService.charFilter(charFilterName); + CharFilterFactory charFilter = charFilters.get(charFilterName); if (charFilter == null) { throw new IllegalArgumentException("Custom Analyzer [" + name() + "] failed to find char_filter under name [" + charFilterName + "]"); } - charFilters.add(charFilter); + charFiltersList.add(charFilter); } - List tokenFilters = new ArrayList<>(); + List tokenFilterList = new ArrayList<>(); String[] tokenFilterNames = analyzerSettings.getAsArray("filter"); for (String tokenFilterName : tokenFilterNames) { - TokenFilterFactory tokenFilter = analysisService.tokenFilter(tokenFilterName); + TokenFilterFactory tokenFilter = tokenFilters.get(tokenFilterName); if (tokenFilter == null) { throw new IllegalArgumentException("Custom Analyzer [" + name() + "] failed to find filter under name [" + tokenFilterName + "]"); } - tokenFilters.add(tokenFilter); + tokenFilterList.add(tokenFilter); } int positionIncrementGap = TextFieldMapper.Defaults.POSITION_INCREMENT_GAP; @@ -93,8 +95,8 @@ public class CustomAnalyzerProvider extends AbstractIndexAnalyzerProvider analyzers; + private final IndexSettings indexSettings; + + public IndexAnalyzers(IndexSettings indexSettings, NamedAnalyzer defaultIndexAnalyzer, NamedAnalyzer defaultSearchAnalyzer, + NamedAnalyzer defaultSearchQuoteAnalyzer, Map analyzers) { + super(indexSettings); + this.defaultIndexAnalyzer = defaultIndexAnalyzer; + this.defaultSearchAnalyzer = defaultSearchAnalyzer; + this.defaultSearchQuoteAnalyzer = defaultSearchQuoteAnalyzer; + this.analyzers = analyzers; + this.indexSettings = indexSettings; + } + + /** + * Returns an analyzer mapped to the given name or null if not present + */ + public NamedAnalyzer get(String name) { + return analyzers.get(name); + } + + + /** + * Returns the default index analyzer for this index + */ + public NamedAnalyzer getDefaultIndexAnalyzer() { + return defaultIndexAnalyzer; + } + + /** + * Returns the default search analyzer for this index + */ + public NamedAnalyzer getDefaultSearchAnalyzer() { + return defaultSearchAnalyzer; + } + + /** + * Returns the default search quote analyzer for this index + */ + public NamedAnalyzer getDefaultSearchQuoteAnalyzer() { + return defaultSearchQuoteAnalyzer; + } + + @Override + public void close() throws IOException { + IOUtils.close(() -> analyzers.values().stream() + .filter(a -> a.scope() == AnalyzerScope.INDEX) + .iterator()); + } + + /** + * Returns the indices settings + */ + public IndexSettings getIndexSettings() { + return indexSettings; + } + +} diff --git a/core/src/main/java/org/elasticsearch/index/mapper/AllFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/AllFieldMapper.java index c418dd5e6e2..6c1477d8d07 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/AllFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/AllFieldMapper.java @@ -106,9 +106,9 @@ public class AllFieldMapper extends MetadataFieldMapper { public MetadataFieldMapper.Builder parse(String name, Map node, ParserContext parserContext) throws MapperParsingException { Builder builder = new Builder(parserContext.mapperService().fullName(NAME)); - builder.fieldType().setIndexAnalyzer(parserContext.analysisService().defaultIndexAnalyzer()); - builder.fieldType().setSearchAnalyzer(parserContext.analysisService().defaultSearchAnalyzer()); - builder.fieldType().setSearchQuoteAnalyzer(parserContext.analysisService().defaultSearchQuoteAnalyzer()); + builder.fieldType().setIndexAnalyzer(parserContext.getIndexAnalyzers().getDefaultIndexAnalyzer()); + builder.fieldType().setSearchAnalyzer(parserContext.getIndexAnalyzers().getDefaultSearchAnalyzer()); + builder.fieldType().setSearchQuoteAnalyzer(parserContext.getIndexAnalyzers().getDefaultSearchQuoteAnalyzer()); // parseField below will happily parse the doc_values setting, but it is then never passed to // the AllFieldMapper ctor in the builder since it is not valid. Here we validate diff --git a/core/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper.java index 13bb7d255a8..09035bfa3ce 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper.java @@ -153,7 +153,7 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp if (searchAnalyzer != null) { throw new MapperParsingException("analyzer on completion field [" + name + "] must be set when search_analyzer is set"); } - indexAnalyzer = searchAnalyzer = parserContext.analysisService().analyzer("simple"); + indexAnalyzer = searchAnalyzer = parserContext.getIndexAnalyzers().get("simple"); } else if (searchAnalyzer == null) { searchAnalyzer = indexAnalyzer; } @@ -164,7 +164,7 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp } private NamedAnalyzer getNamedAnalyzer(ParserContext parserContext, String name) { - NamedAnalyzer analyzer = parserContext.analysisService().analyzer(name); + NamedAnalyzer analyzer = parserContext.getIndexAnalyzers().get(name); if (analyzer == null) { throw new IllegalArgumentException("Can't find default or mapped analyzer with name [" + name + "]"); } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper2x.java b/core/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper2x.java index 655af43710f..590ca0f8615 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper2x.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper2x.java @@ -206,7 +206,7 @@ public class CompletionFieldMapper2x extends FieldMapper { throw new MapperParsingException( "analyzer on completion field [" + name + "] must be set when search_analyzer is set"); } - indexAnalyzer = searchAnalyzer = parserContext.analysisService().analyzer("simple"); + indexAnalyzer = searchAnalyzer = parserContext.getIndexAnalyzers().get("simple"); } else if (searchAnalyzer == null) { searchAnalyzer = indexAnalyzer; } @@ -217,7 +217,7 @@ public class CompletionFieldMapper2x extends FieldMapper { } private NamedAnalyzer getNamedAnalyzer(ParserContext parserContext, String name) { - NamedAnalyzer analyzer = parserContext.analysisService().analyzer(name); + NamedAnalyzer analyzer = parserContext.getIndexAnalyzers().get(name); if (analyzer == null) { throw new IllegalArgumentException("Can't find default or mapped analyzer with name [" + name + "]"); } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index a4d1a0c5e4b..eb6d6a9a3e8 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -32,7 +32,7 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.mapper.MetadataFieldMapper.TypeParser; import org.elasticsearch.search.internal.SearchContext; @@ -147,11 +147,11 @@ public class DocumentMapper implements ToXContent { } MapperUtils.collect(this.mapping.root, newObjectMappers, newFieldMappers); - final AnalysisService analysisService = mapperService.analysisService(); + final IndexAnalyzers indexAnalyzers = mapperService.getIndexAnalyzers(); this.fieldMappers = new DocumentFieldMappers(newFieldMappers, - analysisService.defaultIndexAnalyzer(), - analysisService.defaultSearchAnalyzer(), - analysisService.defaultSearchQuoteAnalyzer()); + indexAnalyzers.getDefaultIndexAnalyzer(), + indexAnalyzers.getDefaultSearchAnalyzer(), + indexAnalyzers.getDefaultSearchQuoteAnalyzer()); Map builder = new HashMap<>(); for (ObjectMapper objectMapper : newObjectMappers) { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java index f336fbb01ac..2cdeed9f040 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java @@ -22,14 +22,13 @@ package org.elasticsearch.index.mapper; import org.elasticsearch.Version; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseFieldMatcher; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.mapper.MapperRegistry; @@ -44,7 +43,7 @@ import static java.util.Collections.unmodifiableMap; public class DocumentMapperParser { final MapperService mapperService; - final AnalysisService analysisService; + final IndexAnalyzers indexAnalyzers; private final SimilarityService similarityService; private final Supplier queryShardContextSupplier; @@ -56,12 +55,12 @@ public class DocumentMapperParser { private final Map typeParsers; private final Map rootTypeParsers; - public DocumentMapperParser(IndexSettings indexSettings, MapperService mapperService, AnalysisService analysisService, + public DocumentMapperParser(IndexSettings indexSettings, MapperService mapperService, IndexAnalyzers indexAnalyzers, SimilarityService similarityService, MapperRegistry mapperRegistry, Supplier queryShardContextSupplier) { this.parseFieldMatcher = new ParseFieldMatcher(indexSettings.getSettings()); this.mapperService = mapperService; - this.analysisService = analysisService; + this.indexAnalyzers = indexAnalyzers; this.similarityService = similarityService; this.queryShardContextSupplier = queryShardContextSupplier; this.typeParsers = mapperRegistry.getMapperParsers(); @@ -70,7 +69,7 @@ public class DocumentMapperParser { } public Mapper.TypeParser.ParserContext parserContext(String type) { - return new Mapper.TypeParser.ParserContext(type, analysisService, similarityService::getSimilarity, mapperService, typeParsers::get, indexVersionCreated, parseFieldMatcher, queryShardContextSupplier.get()); + return new Mapper.TypeParser.ParserContext(type, indexAnalyzers, similarityService::getSimilarity, mapperService, typeParsers::get, indexVersionCreated, parseFieldMatcher, queryShardContextSupplier.get()); } public DocumentMapper parse(@Nullable String type, CompressedXContent source) throws MapperParsingException { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyTokenCountFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyTokenCountFieldMapper.java index 2ed1b544a02..7981b400214 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyTokenCountFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyTokenCountFieldMapper.java @@ -97,7 +97,7 @@ public class LegacyTokenCountFieldMapper extends LegacyIntegerFieldMapper { builder.nullValue(nodeIntegerValue(propNode)); iterator.remove(); } else if (propName.equals("analyzer")) { - NamedAnalyzer analyzer = parserContext.analysisService().analyzer(propNode.toString()); + NamedAnalyzer analyzer = parserContext.getIndexAnalyzers().get(propNode.toString()); if (analyzer == null) { throw new MapperParsingException("Analyzer [" + propNode.toString() + "] not found for field [" + name + "]"); } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java b/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java index d1341c8f7d4..06928566424 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java @@ -24,7 +24,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.similarity.SimilarityProvider; @@ -85,7 +85,7 @@ public abstract class Mapper implements ToXContent, Iterable { private final String type; - private final AnalysisService analysisService; + private final IndexAnalyzers indexAnalyzers; private final Function similarityLookupService; @@ -99,11 +99,11 @@ public abstract class Mapper implements ToXContent, Iterable { private final QueryShardContext queryShardContext; - public ParserContext(String type, AnalysisService analysisService, Function similarityLookupService, + public ParserContext(String type, IndexAnalyzers indexAnalyzers, Function similarityLookupService, MapperService mapperService, Function typeParsers, Version indexVersionCreated, ParseFieldMatcher parseFieldMatcher, QueryShardContext queryShardContext) { this.type = type; - this.analysisService = analysisService; + this.indexAnalyzers = indexAnalyzers; this.similarityLookupService = similarityLookupService; this.mapperService = mapperService; this.typeParsers = typeParsers; @@ -116,8 +116,8 @@ public abstract class Mapper implements ToXContent, Iterable { return type; } - public AnalysisService analysisService() { - return analysisService; + public IndexAnalyzers getIndexAnalyzers() { + return indexAnalyzers; } public SimilarityProvider getSimilarity(String name) { @@ -159,7 +159,7 @@ public abstract class Mapper implements ToXContent, Iterable { static class MultiFieldParserContext extends ParserContext { MultiFieldParserContext(ParserContext in) { - super(in.type(), in.analysisService, in.similarityLookupService(), in.mapperService(), in.typeParsers(), in.indexVersionCreated(), in.parseFieldMatcher(), in.queryShardContext()); + super(in.type(), in.indexAnalyzers, in.similarityLookupService(), in.mapperService(), in.typeParsers(), in.indexVersionCreated(), in.parseFieldMatcher(), in.queryShardContext()); } } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 148fde7b648..2da082cba75 100755 --- a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -36,7 +36,7 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.AbstractIndexComponent; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.mapper.Mapper.BuilderContext; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.similarity.SimilarityService; @@ -100,7 +100,7 @@ public class MapperService extends AbstractIndexComponent { @Deprecated public static final String PERCOLATOR_LEGACY_TYPE_NAME = ".percolator"; - private final AnalysisService analysisService; + private final IndexAnalyzers indexAnalyzers; /** * Will create types automatically if they do not exists in the mapping definition yet @@ -127,16 +127,16 @@ public class MapperService extends AbstractIndexComponent { final MapperRegistry mapperRegistry; - public MapperService(IndexSettings indexSettings, AnalysisService analysisService, + public MapperService(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers, SimilarityService similarityService, MapperRegistry mapperRegistry, Supplier queryShardContextSupplier) { super(indexSettings); - this.analysisService = analysisService; + this.indexAnalyzers = indexAnalyzers; this.fieldTypes = new FieldTypeLookup(); - this.documentParser = new DocumentMapperParser(indexSettings, this, analysisService, similarityService, mapperRegistry, queryShardContextSupplier); - this.indexAnalyzer = new MapperAnalyzerWrapper(analysisService.defaultIndexAnalyzer(), p -> p.indexAnalyzer()); - this.searchAnalyzer = new MapperAnalyzerWrapper(analysisService.defaultSearchAnalyzer(), p -> p.searchAnalyzer()); - this.searchQuoteAnalyzer = new MapperAnalyzerWrapper(analysisService.defaultSearchQuoteAnalyzer(), p -> p.searchQuoteAnalyzer()); + this.documentParser = new DocumentMapperParser(indexSettings, this, indexAnalyzers, similarityService, mapperRegistry, queryShardContextSupplier); + this.indexAnalyzer = new MapperAnalyzerWrapper(indexAnalyzers.getDefaultIndexAnalyzer(), p -> p.indexAnalyzer()); + this.searchAnalyzer = new MapperAnalyzerWrapper(indexAnalyzers.getDefaultSearchAnalyzer(), p -> p.searchAnalyzer()); + this.searchQuoteAnalyzer = new MapperAnalyzerWrapper(indexAnalyzers.getDefaultSearchQuoteAnalyzer(), p -> p.searchQuoteAnalyzer()); this.mapperRegistry = mapperRegistry; this.dynamic = this.indexSettings.getValue(INDEX_MAPPER_DYNAMIC_SETTING); @@ -171,8 +171,8 @@ public class MapperService extends AbstractIndexComponent { }; } - public AnalysisService analysisService() { - return this.analysisService; + public IndexAnalyzers getIndexAnalyzers() { + return this.indexAnalyzers; } public DocumentMapperParser documentMapperParser() { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index 8a2aca97e68..7ff5c4a37fe 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -29,15 +29,11 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lucene.all.AllEntries; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.analysis.AnalysisService; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -/** - * - */ public abstract class ParseContext { /** Fork of {@link org.apache.lucene.document.Document} with additional functionality. */ @@ -242,11 +238,6 @@ public abstract class ParseContext { return in.docMapper(); } - @Override - public AnalysisService analysisService() { - return in.analysisService(); - } - @Override public MapperService mapperService() { return in.mapperService(); @@ -385,11 +376,6 @@ public abstract class ParseContext { return this.docMapper; } - @Override - public AnalysisService analysisService() { - return docMapperParser.analysisService; - } - @Override public MapperService mapperService() { return docMapperParser.mapperService; @@ -525,8 +511,6 @@ public abstract class ParseContext { public abstract DocumentMapper docMapper(); - public abstract AnalysisService analysisService(); - public abstract MapperService mapperService(); public abstract Field version(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java index ec7a90148ad..d290ef0fb51 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java @@ -318,13 +318,13 @@ public class StringFieldMapper extends FieldMapper { // we need to update to actual analyzers if they are not set in this case... // so we can inject the position increment gap... if (builder.fieldType().indexAnalyzer() == null) { - builder.fieldType().setIndexAnalyzer(parserContext.analysisService().defaultIndexAnalyzer()); + builder.fieldType().setIndexAnalyzer(parserContext.getIndexAnalyzers().getDefaultIndexAnalyzer()); } if (builder.fieldType().searchAnalyzer() == null) { - builder.fieldType().setSearchAnalyzer(parserContext.analysisService().defaultSearchAnalyzer()); + builder.fieldType().setSearchAnalyzer(parserContext.getIndexAnalyzers().getDefaultSearchAnalyzer()); } if (builder.fieldType().searchQuoteAnalyzer() == null) { - builder.fieldType().setSearchQuoteAnalyzer(parserContext.analysisService().defaultSearchQuoteAnalyzer()); + builder.fieldType().setSearchQuoteAnalyzer(parserContext.getIndexAnalyzers().getDefaultSearchQuoteAnalyzer()); } iterator.remove(); } else if (propName.equals("ignore_above")) { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java index 53b02717cc1..63febfcaf01 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java @@ -174,13 +174,13 @@ public class TextFieldMapper extends FieldMapper { } node.put("fielddata", fielddata); } - + return new StringFieldMapper.TypeParser().parse(fieldName, node, parserContext); } TextFieldMapper.Builder builder = new TextFieldMapper.Builder(fieldName); - builder.fieldType().setIndexAnalyzer(parserContext.analysisService().defaultIndexAnalyzer()); - builder.fieldType().setSearchAnalyzer(parserContext.analysisService().defaultSearchAnalyzer()); - builder.fieldType().setSearchQuoteAnalyzer(parserContext.analysisService().defaultSearchQuoteAnalyzer()); + builder.fieldType().setIndexAnalyzer(parserContext.getIndexAnalyzers().getDefaultIndexAnalyzer()); + builder.fieldType().setSearchAnalyzer(parserContext.getIndexAnalyzers().getDefaultSearchAnalyzer()); + builder.fieldType().setSearchQuoteAnalyzer(parserContext.getIndexAnalyzers().getDefaultSearchQuoteAnalyzer()); parseTextField(builder, fieldName, node, parserContext); for (Iterator> iterator = node.entrySet().iterator(); iterator.hasNext();) { Map.Entry entry = iterator.next(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/TokenCountFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/TokenCountFieldMapper.java index 9eeaf4012fa..3b6026d1b21 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/TokenCountFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/TokenCountFieldMapper.java @@ -89,7 +89,7 @@ public class TokenCountFieldMapper extends FieldMapper { builder.nullValue(nodeIntegerValue(propNode)); iterator.remove(); } else if (propName.equals("analyzer")) { - NamedAnalyzer analyzer = parserContext.analysisService().analyzer(propNode.toString()); + NamedAnalyzer analyzer = parserContext.getIndexAnalyzers().get(propNode.toString()); if (analyzer == null) { throw new MapperParsingException("Analyzer [" + propNode.toString() + "] not found for field [" + name + "]"); } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/TypeParsers.java b/core/src/main/java/org/elasticsearch/index/mapper/TypeParsers.java index eaa97ac5100..f192efc24a6 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/TypeParsers.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/TypeParsers.java @@ -125,21 +125,21 @@ public class TypeParsers { builder.storeTermVectorPayloads(nodeBooleanValue("store_term_vector_payloads", propNode, parserContext)); iterator.remove(); } else if (propName.equals("analyzer")) { - NamedAnalyzer analyzer = parserContext.analysisService().analyzer(propNode.toString()); + NamedAnalyzer analyzer = parserContext.getIndexAnalyzers().get(propNode.toString()); if (analyzer == null) { throw new MapperParsingException("analyzer [" + propNode.toString() + "] not found for field [" + name + "]"); } indexAnalyzer = analyzer; iterator.remove(); } else if (propName.equals("search_analyzer")) { - NamedAnalyzer analyzer = parserContext.analysisService().analyzer(propNode.toString()); + NamedAnalyzer analyzer = parserContext.getIndexAnalyzers().get(propNode.toString()); if (analyzer == null) { throw new MapperParsingException("analyzer [" + propNode.toString() + "] not found for field [" + name + "]"); } searchAnalyzer = analyzer; iterator.remove(); } else if (propName.equals("search_quote_analyzer")) { - NamedAnalyzer analyzer = parserContext.analysisService().analyzer(propNode.toString()); + NamedAnalyzer analyzer = parserContext.getIndexAnalyzers().get(propNode.toString()); if (analyzer == null) { throw new MapperParsingException("analyzer [" + propNode.toString() + "] not found for field [" + name + "]"); } diff --git a/core/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java index 21328ff8fc4..3bd7a8abc12 100644 --- a/core/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java @@ -383,7 +383,7 @@ public class CommonTermsQueryBuilder extends AbstractQueryBuilder { @Override protected Query doToQuery(QueryShardContext context) throws IOException { // validate context specific fields - if (analyzer != null && context.getAnalysisService().analyzer(analyzer) == null) { + if (analyzer != null && context.getIndexAnalyzers().get(analyzer) == null) { throw new QueryShardException(context, "[" + NAME + "] analyzer [" + analyzer + "] not found"); } diff --git a/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java index 666422f9dc7..7ba39d7b348 100644 --- a/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java @@ -147,7 +147,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder settings) throws Exception { + if (pluginId == null) { + throw new UserException(ExitCodes.USAGE, "plugin id is required"); + } final Environment env = InternalSettingsPreparer.prepareEnvironment(Settings.EMPTY, terminal, settings); // TODO: remove this leniency!! is it needed anymore? if (Files.exists(env.pluginsFile()) == false) { diff --git a/core/src/main/java/org/elasticsearch/search/DefaultSearchContext.java b/core/src/main/java/org/elasticsearch/search/DefaultSearchContext.java index 20789e015a9..9aab786aa34 100644 --- a/core/src/main/java/org/elasticsearch/search/DefaultSearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/DefaultSearchContext.java @@ -39,7 +39,6 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.fielddata.IndexFieldDataService; @@ -497,11 +496,6 @@ final class DefaultSearchContext extends SearchContext { return indexService.mapperService(); } - @Override - public AnalysisService analysisService() { - return indexService.analysisService(); - } - @Override public SimilarityService similarityService() { return indexService.similarityService(); diff --git a/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java b/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java index 04156bfac2d..5084f9ecad4 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java @@ -27,7 +27,6 @@ import org.elasticsearch.action.search.SearchType; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MappedFieldType; @@ -259,11 +258,6 @@ public abstract class FilteredSearchContext extends SearchContext { return in.mapperService(); } - @Override - public AnalysisService analysisService() { - return in.analysisService(); - } - @Override public SimilarityService similarityService() { return in.similarityService(); diff --git a/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java index 459df14a357..63a1995b08f 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -33,7 +33,6 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.RefCounted; import org.elasticsearch.common.util.iterable.Iterables; -import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MappedFieldType; @@ -237,8 +236,6 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas public abstract MapperService mapperService(); - public abstract AnalysisService analysisService(); - public abstract SimilarityService similarityService(); public abstract ScriptService scriptService(); diff --git a/core/src/main/java/org/elasticsearch/search/suggest/SuggestionBuilder.java b/core/src/main/java/org/elasticsearch/search/suggest/SuggestionBuilder.java index 59555e049c2..ca1d7a2306f 100644 --- a/core/src/main/java/org/elasticsearch/search/suggest/SuggestionBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/suggest/SuggestionBuilder.java @@ -319,7 +319,7 @@ public abstract class SuggestionBuilder> extends suggestionContext.setAnalyzer(fieldType.searchAnalyzer()); } } else { - Analyzer luceneAnalyzer = mapperService.analysisService().analyzer(analyzer); + Analyzer luceneAnalyzer = mapperService.getIndexAnalyzers().get(analyzer); if (luceneAnalyzer == null) { throw new IllegalArgumentException("analyzer [" + analyzer + "] doesn't exists"); } diff --git a/core/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java b/core/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java index 9e3beb2ccf1..bf9158f9b87 100644 --- a/core/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java @@ -395,13 +395,13 @@ public final class DirectCandidateGeneratorBuilder implements CandidateGenerator generator.setField(this.field); transferIfNotNull(this.size, generator::size); if (this.preFilter != null) { - generator.preFilter(mapperService.analysisService().analyzer(this.preFilter)); + generator.preFilter(mapperService.getIndexAnalyzers().get(this.preFilter)); if (generator.preFilter() == null) { throw new IllegalArgumentException("Analyzer [" + this.preFilter + "] doesn't exists"); } } if (this.postFilter != null) { - generator.postFilter(mapperService.analysisService().analyzer(this.postFilter)); + generator.postFilter(mapperService.getIndexAnalyzers().get(this.postFilter)); if (generator.postFilter() == null) { throw new IllegalArgumentException("Analyzer [" + this.postFilter + "] doesn't exists"); } diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeClientNode.java b/core/src/main/java/org/elasticsearch/tribe/TribeClientNode.java index 02957ab99a4..d9520aef768 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeClientNode.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeClientNode.java @@ -24,13 +24,13 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.node.Node; import org.elasticsearch.plugins.Plugin; -import java.util.Collections; +import java.util.Collection; /** * An internal node that connects to a remove cluster, as part of a tribe node. */ class TribeClientNode extends Node { - TribeClientNode(Settings settings) { - super(new Environment(settings), Collections.>emptyList()); + TribeClientNode(Settings settings, Collection> classpathPlugins) { + super(new Environment(settings), classpathPlugins); } } diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index 99f6c696e66..fd697340cd7 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -58,10 +58,12 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.node.Node; +import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.transport.TransportSettings; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; @@ -182,7 +184,8 @@ public class TribeService extends AbstractLifecycleComponent { private final List nodes = new CopyOnWriteArrayList<>(); - public TribeService(Settings settings, ClusterService clusterService, final String tribeNodeId) { + public TribeService(Settings settings, ClusterService clusterService, final String tribeNodeId, + Collection> classpathPlugins) { super(settings); this.clusterService = clusterService; Map nodesSettings = new HashMap<>(settings.getGroups("tribe", true)); @@ -190,7 +193,7 @@ public class TribeService extends AbstractLifecycleComponent { nodesSettings.remove("on_conflict"); // remove prefix settings that don't indicate a client for (Map.Entry entry : nodesSettings.entrySet()) { Settings clientSettings = buildClientSettings(entry.getKey(), tribeNodeId, settings, entry.getValue()); - nodes.add(new TribeClientNode(clientSettings)); + nodes.add(new TribeClientNode(clientSettings, classpathPlugins)); } this.blockIndicesMetadata = BLOCKS_METADATA_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY); diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java index 3b263451487..97c1a20c33f 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.junit.annotations.TestLogging; import java.util.List; import java.util.Map; @@ -43,6 +44,7 @@ import static org.hamcrest.Matchers.greaterThan; */ @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0) public final class ClusterAllocationExplainIT extends ESIntegTestCase { + @TestLogging("_root:DEBUG") public void testDelayShards() throws Exception { logger.info("--> starting 3 nodes"); List nodes = internalCluster().startNodesAsync(3).get(); diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteTests.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteTests.java index c4e95889745..a9054879941 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteTests.java @@ -28,7 +28,7 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; +import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.command.AllocateEmptyPrimaryAllocationCommand; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; @@ -41,7 +41,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.io.IOException; import java.util.Collections; @@ -82,7 +82,7 @@ public class ClusterRerouteTests extends ESAllocationTestCase { public void testClusterStateUpdateTask() { AllocationService allocationService = new AllocationService(Settings.builder().build(), new AllocationDeciders(Settings.EMPTY, Collections.singleton(new MaxRetryAllocationDecider(Settings.EMPTY))), - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); + new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); ClusterState clusterState = createInitialClusterState(allocationService); ClusterRerouteRequest req = new ClusterRerouteRequest(); req.dryRun(true); @@ -118,8 +118,8 @@ public class ClusterRerouteTests extends ESAllocationTestCase { assertEquals(routingTable.index("idx").shards().size(), 1); assertEquals(routingTable.index("idx").shard(0).shards().get(0).state(), INITIALIZING); assertEquals(routingTable.index("idx").shard(0).shards().get(0).unassignedInfo().getNumFailedAllocations(), i); - List failedShards = Collections.singletonList( - new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom" + i, + List failedShards = Collections.singletonList( + new FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom" + i, new UnsupportedOperationException())); newState = allocationService.applyFailedShards(clusterState, failedShards); assertThat(newState, not(equalTo(clusterState))); diff --git a/core/src/test/java/org/elasticsearch/action/admin/indices/TransportAnalyzeActionTests.java b/core/src/test/java/org/elasticsearch/action/admin/indices/TransportAnalyzeActionTests.java index 9933d514056..bcd7bba8d38 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/indices/TransportAnalyzeActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/indices/TransportAnalyzeActionTests.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.action.admin.indices; -import org.apache.lucene.analysis.minhash.MinHashFilter; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse; @@ -29,7 +28,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.AnalysisRegistry; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.mapper.AllFieldMapper; import org.elasticsearch.indices.analysis.AnalysisModule; import org.elasticsearch.test.ESTestCase; @@ -42,7 +41,7 @@ import static java.util.Collections.emptyList; public class TransportAnalyzeActionTests extends ESTestCase { - private AnalysisService analysisService; + private IndexAnalyzers indexAnalyzers; private AnalysisRegistry registry; private Environment environment; @@ -71,10 +70,10 @@ public class TransportAnalyzeActionTests extends ESTestCase { IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings); environment = new Environment(settings); registry = new AnalysisModule(environment, emptyList()).getAnalysisRegistry(); - analysisService = registry.build(idxSettings); + indexAnalyzers = registry.build(idxSettings); } - public void testNoAnalysisService() throws IOException { + public void testNoIndexAnalyzers() throws IOException { AnalyzeRequest request = new AnalyzeRequest(); request.analyzer("standard"); request.text("the quick brown fox"); @@ -87,7 +86,7 @@ public class TransportAnalyzeActionTests extends ESTestCase { request.addTokenFilter("lowercase"); request.addTokenFilter("word_delimiter"); request.text("the qu1ck brown fox"); - analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, randomBoolean() ? analysisService : null, registry, environment); + analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, randomBoolean() ? indexAnalyzers : null, registry, environment); tokens = analyze.getTokens(); assertEquals(6, tokens.size()); assertEquals("qu", tokens.get(1).getTerm()); @@ -100,7 +99,7 @@ public class TransportAnalyzeActionTests extends ESTestCase { request.addTokenFilter("lowercase"); request.addTokenFilter("word_delimiter"); request.text("

the qu1ck brown fox

"); - analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, randomBoolean() ? analysisService : null, registry, environment); + analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, randomBoolean() ? indexAnalyzers : null, registry, environment); tokens = analyze.getTokens(); assertEquals(6, tokens.size()); assertEquals("the", tokens.get(0).getTerm()); @@ -143,26 +142,26 @@ public class TransportAnalyzeActionTests extends ESTestCase { assertEquals("", tokens.get(3).getType()); } - public void testWithAnalysisService() throws IOException { + public void testWithIndexAnalyzers() throws IOException { AnalyzeRequest request = new AnalyzeRequest(); request.analyzer("standard"); request.text("the quick brown fox"); request.analyzer("custom_analyzer"); request.text("the qu1ck brown fox"); - AnalyzeResponse analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, analysisService, registry, environment); + AnalyzeResponse analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, indexAnalyzers, registry, environment); List tokens = analyze.getTokens(); assertEquals(4, tokens.size()); request.analyzer("whitespace"); request.text("the qu1ck brown fox-dog"); - analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, analysisService, registry, environment); + analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, indexAnalyzers, registry, environment); tokens = analyze.getTokens(); assertEquals(4, tokens.size()); request.analyzer("custom_analyzer"); request.text("the qu1ck brown fox-dog"); - analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, analysisService, registry, environment); + analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, indexAnalyzers, registry, environment); tokens = analyze.getTokens(); assertEquals(5, tokens.size()); @@ -171,7 +170,7 @@ public class TransportAnalyzeActionTests extends ESTestCase { request.addTokenFilter("lowercase"); request.addTokenFilter("wordDelimiter"); request.text("the qu1ck brown fox-dog"); - analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, analysisService, registry, environment); + analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, indexAnalyzers, registry, environment); tokens = analyze.getTokens(); assertEquals(5, tokens.size()); assertEquals("the", tokens.get(0).getTerm()); @@ -184,14 +183,14 @@ public class TransportAnalyzeActionTests extends ESTestCase { request.tokenizer("trigram"); request.addTokenFilter("synonym"); request.text("kimchy"); - analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, analysisService, registry, environment); + analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, indexAnalyzers, registry, environment); tokens = analyze.getTokens(); assertEquals(2, tokens.size()); assertEquals("sha", tokens.get(0).getTerm()); assertEquals("hay", tokens.get(1).getTerm()); } - public void testGetIndexAnalyserWithoutAnalysisService() throws IOException { + public void testGetIndexAnalyserWithoutIndexAnalyzers() throws IOException { IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> TransportAnalyzeAction.analyze( new AnalyzeRequest() @@ -208,7 +207,7 @@ public class TransportAnalyzeActionTests extends ESTestCase { new AnalyzeRequest() .analyzer("foobar") .text("the qu1ck brown fox"), - AllFieldMapper.NAME, null, notGlobal ? analysisService : null, registry, environment)); + AllFieldMapper.NAME, null, notGlobal ? indexAnalyzers : null, registry, environment)); if (notGlobal) { assertEquals(e.getMessage(), "failed to find analyzer [foobar]"); } else { @@ -220,7 +219,7 @@ public class TransportAnalyzeActionTests extends ESTestCase { new AnalyzeRequest() .tokenizer("foobar") .text("the qu1ck brown fox"), - AllFieldMapper.NAME, null, notGlobal ? analysisService : null, registry, environment)); + AllFieldMapper.NAME, null, notGlobal ? indexAnalyzers : null, registry, environment)); if (notGlobal) { assertEquals(e.getMessage(), "failed to find tokenizer under [foobar]"); } else { @@ -233,7 +232,7 @@ public class TransportAnalyzeActionTests extends ESTestCase { .tokenizer("whitespace") .addTokenFilter("foobar") .text("the qu1ck brown fox"), - AllFieldMapper.NAME, null, notGlobal ? analysisService : null, registry, environment)); + AllFieldMapper.NAME, null, notGlobal ? indexAnalyzers : null, registry, environment)); if (notGlobal) { assertEquals(e.getMessage(), "failed to find token filter under [foobar]"); } else { @@ -247,7 +246,7 @@ public class TransportAnalyzeActionTests extends ESTestCase { .addTokenFilter("lowercase") .addCharFilter("foobar") .text("the qu1ck brown fox"), - AllFieldMapper.NAME, null, notGlobal ? analysisService : null, registry, environment)); + AllFieldMapper.NAME, null, notGlobal ? indexAnalyzers : null, registry, environment)); if (notGlobal) { assertEquals(e.getMessage(), "failed to find char filter under [foobar]"); } else { @@ -260,7 +259,7 @@ public class TransportAnalyzeActionTests extends ESTestCase { request.tokenizer("whitespace"); request.addTokenFilter("min_hash"); request.text("the quick brown fox"); - AnalyzeResponse analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, analysisService, registry, environment); + AnalyzeResponse analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, indexAnalyzers, registry, environment); List tokens = analyze.getTokens(); int default_hash_count = 1; int default_bucket_size = 512; diff --git a/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java b/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java index 49d0ce447ba..0c5164aec5b 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java @@ -42,7 +42,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.DocsStats; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.util.Arrays; import java.util.Collections; @@ -97,7 +97,7 @@ public class TransportShrinkActionTests extends ESTestCase { .build(); AllocationService service = new AllocationService(Settings.builder().build(), new AllocationDeciders(Settings.EMPTY, Collections.singleton(new MaxRetryAllocationDecider(Settings.EMPTY))), - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); + new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); @@ -120,7 +120,7 @@ public class TransportShrinkActionTests extends ESTestCase { .build(); AllocationService service = new AllocationService(Settings.builder().build(), new AllocationDeciders(Settings.EMPTY, Collections.singleton(new MaxRetryAllocationDecider(Settings.EMPTY))), - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); + new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java b/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java index 13ec55a41c9..00edd536258 100644 --- a/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java @@ -37,7 +37,8 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; +import org.elasticsearch.cluster.routing.allocation.FailedShard; +import org.elasticsearch.cluster.routing.allocation.StaleShard; import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; @@ -115,8 +116,7 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa List nonExistentTasks = createNonExistentShards(currentState, reason); ShardStateAction.ShardFailedClusterStateTaskExecutor failingExecutor = new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocationService, null, logger) { @Override - ClusterState applyFailedShards(ClusterState currentState, List failedShards, - List staleShards) { + ClusterState applyFailedShards(ClusterState currentState, List failedShards, List staleShards) { throw new RuntimeException("simulated applyFailedShards failure"); } }; diff --git a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java index 93a0c7f9e5c..d80e16397ac 100644 --- a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java @@ -48,7 +48,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import org.elasticsearch.test.transport.CapturingTransport; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -139,7 +139,7 @@ public class ClusterStateHealthTests extends ESTestCase { listenerCalled.await(); TransportClusterHealthAction action = new TransportClusterHealthAction(Settings.EMPTY, transportService, - clusterService, threadPool, new ActionFilters(new HashSet<>()), indexNameExpressionResolver, NoopGatewayAllocator.INSTANCE); + clusterService, threadPool, new ActionFilters(new HashSet<>()), indexNameExpressionResolver, new TestGatewayAllocator()); PlainActionFuture listener = new PlainActionFuture<>(); action.execute(new ClusterHealthRequest(), listener); @@ -277,9 +277,9 @@ public class ClusterStateHealthTests extends ESTestCase { // if the inactive primaries are due solely to recovery (not failed allocation or previously being allocated) // then cluster health is YELLOW, otherwise RED if (primaryInactiveDueToRecovery(indexName, clusterState)) { - assertThat(health.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + assertThat("clusterState is:\n" + clusterState.prettyPrint(), health.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); } else { - assertThat(health.getStatus(), equalTo(ClusterHealthStatus.RED)); + assertThat("clusterState is:\n" + clusterState.prettyPrint(), health.getStatus(), equalTo(ClusterHealthStatus.RED)); } } } @@ -532,6 +532,12 @@ public class ClusterStateHealthTests extends ESTestCase { primaryShard.recoverySource().getType() == RecoverySource.Type.EXISTING_STORE) { return false; } + if (primaryShard.unassignedInfo().getNumFailedAllocations() > 0) { + return false; + } + if (primaryShard.unassignedInfo().getLastAllocationStatus() == UnassignedInfo.AllocationStatus.DECIDERS_NO) { + return false; + } } } return true; diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java index de82ce8d073..d74b450f5bf 100644 --- a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java @@ -39,7 +39,7 @@ import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.InvalidIndexNameException; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.util.Arrays; import java.util.Collections; @@ -133,7 +133,7 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { .build(); AllocationService service = new AllocationService(Settings.builder().build(), new AllocationDeciders(Settings.EMPTY, Collections.singleton(new MaxRetryAllocationDecider(Settings.EMPTY))), - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); + new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); @@ -161,7 +161,7 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { .build(); AllocationService service = new AllocationService(Settings.builder().build(), new AllocationDeciders(Settings.EMPTY, Collections.singleton(new MaxRetryAllocationDecider(Settings.EMPTY))), - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); + new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryTermsTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryTermsTests.java index cf22462865f..863a33b1327 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryTermsTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryTermsTests.java @@ -28,7 +28,7 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes.Builder; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; +import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.common.settings.Settings; import java.util.ArrayList; @@ -63,7 +63,7 @@ public class PrimaryTermsTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_initial_primaries_recoveries", Integer.MAX_VALUE) .build()); this.numberOfShards = randomIntBetween(1, 5); - this.numberOfReplicas = randomIntBetween(1, 5); + this.numberOfReplicas = randomIntBetween(0, 5); logger.info("Setup test with {} shards and {} replicas.", this.numberOfShards, this.numberOfReplicas); this.primaryTermsPerIndex.clear(); MetaData metaData = MetaData.builder() @@ -142,9 +142,9 @@ public class PrimaryTermsTests extends ESAllocationTestCase { shardIdsToFail.add(randomInt(numberOfShards - 1)); } logger.info("failing primary shards {} for index [{}]", shardIdsToFail, index); - List failedShards = new ArrayList<>(); + List failedShards = new ArrayList<>(); for (int shard : shardIdsToFail) { - failedShards.add(new FailedRerouteAllocation.FailedShard(indexShardRoutingTable.shard(shard).primaryShard(), "test", null)); + failedShards.add(new FailedShard(indexShardRoutingTable.shard(shard).primaryShard(), "test", null)); incrementPrimaryTerm(index, shard); // the primary failure should increment the primary term; } applyRerouteResult(allocationService.applyFailedShards(this.clusterState, failedShards,Collections.emptyList())); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java index f0514794a5e..6ed42ee45aa 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java @@ -328,13 +328,20 @@ public class RoutingTableTests extends ESAllocationTestCase { expectThrows(IllegalStateException.class, () -> indexRoutingTable.validate(metaData4)); } + /** reverse engineer the in sync aid based on the given indexRoutingTable **/ public static IndexMetaData updateActiveAllocations(IndexRoutingTable indexRoutingTable, IndexMetaData indexMetaData) { IndexMetaData.Builder imdBuilder = IndexMetaData.builder(indexMetaData); for (IndexShardRoutingTable shardTable : indexRoutingTable) { for (ShardRouting shardRouting : shardTable) { - Set activeAllocations = shardTable.activeShards().stream().map( + Set insyncAids = shardTable.activeShards().stream().map( shr -> shr.allocationId().getId()).collect(Collectors.toSet()); - imdBuilder.putInSyncAllocationIds(shardRouting.id(), activeAllocations); + final ShardRouting primaryShard = shardTable.primaryShard(); + if (primaryShard.initializing() && primaryShard.relocating() == false && + RecoverySource.isInitialRecovery(primaryShard.recoverySource().getType()) == false ) { + // simulate a primary was initialized based on aid + insyncAids.add(primaryShard.allocationId().getId()); + } + imdBuilder.putInSyncAllocationIds(shardRouting.id(), insyncAids); } } return imdBuilder.build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 4b7e81df380..5eff8a0a53d 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -31,7 +31,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; +import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.io.stream.ByteBufferStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; @@ -255,7 +255,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase { assertThat(clusterState.getRoutingNodes().unassigned().size() > 0, equalTo(false)); // fail shard ShardRouting shardToFail = clusterState.getRoutingNodes().shardsWithState(STARTED).get(0); - clusterState = allocation.applyFailedShards(clusterState, Collections.singletonList(new FailedRerouteAllocation.FailedShard(shardToFail, "test fail", null))); + clusterState = allocation.applyFailedShards(clusterState, Collections.singletonList(new FailedShard(shardToFail, "test fail", null))); // verify the reason and details assertThat(clusterState.getRoutingNodes().unassigned().size() > 0, equalTo(true)); assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(1)); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java index b2dd5ae0d26..e2a360b93fe 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java @@ -41,10 +41,12 @@ import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllo import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.gateway.GatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import org.hamcrest.Matchers; import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; @@ -71,7 +73,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase { settings.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), replicaBalance); settings.put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), balanceTreshold); - AllocationService strategy = createAllocationService(settings.build()); + AllocationService strategy = createAllocationService(settings.build(), new NoopGatewayAllocator()); ClusterState clusterState = initCluster(strategy); assertIndexBalance(clusterState.getRoutingTable(), clusterState.getRoutingNodes(), numberOfNodes, numberOfIndices, numberOfReplicas, numberOfShards, balanceTreshold); @@ -95,7 +97,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase { settings.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), replicaBalance); settings.put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), balanceTreshold); - AllocationService strategy = createAllocationService(settings.build()); + AllocationService strategy = createAllocationService(settings.build(), new NoopGatewayAllocator()); ClusterState clusterState = initCluster(strategy); assertReplicaBalance(logger, clusterState.getRoutingNodes(), numberOfNodes, numberOfIndices, numberOfReplicas, numberOfShards, balanceTreshold); @@ -254,7 +256,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase { Settings.Builder settings = Settings.builder(); AllocationService strategy = new AllocationService(settings.build(), randomAllocationDeciders(settings.build(), new ClusterSettings(Settings.Builder.EMPTY_SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), random()), - NoopGatewayAllocator.INSTANCE, new ShardsAllocator() { + new TestGatewayAllocator(), new ShardsAllocator() { public Map weighShard(RoutingAllocation allocation, ShardRouting shard) { return new HashMap(); @@ -351,7 +353,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase { assertThat(shardRouting.state(), Matchers.equalTo(ShardRoutingState.INITIALIZING)); } } - strategy = createAllocationService(settings.build()); + strategy = createAllocationService(settings.build(), new NoopGatewayAllocator()); logger.info("use the new allocator and check if it moves shards"); routingNodes = clusterState.getRoutingNodes(); @@ -385,7 +387,26 @@ public class BalanceConfigurationTests extends ESAllocationTestCase { assertThat(shardRouting.state(), Matchers.equalTo(ShardRoutingState.STARTED)); } } - } + private class NoopGatewayAllocator extends GatewayAllocator { + + public NoopGatewayAllocator() { + super(Settings.EMPTY, null, null); + } + + @Override + public void applyStartedShards(RoutingAllocation allocation, List startedShards) { + // noop + } + + @Override + public void applyFailedShards(RoutingAllocation allocation, List failedShards) { + // noop + } + @Override + public void allocateUnassigned(RoutingAllocation allocation) { + // noop + } + } } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java index 953ad1535e8..8cccdb08fb5 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java @@ -33,7 +33,7 @@ import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.util.concurrent.atomic.AtomicBoolean; @@ -577,7 +577,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase { public void testRebalanceWithIgnoredUnassignedShards() { final AtomicBoolean allocateTest1 = new AtomicBoolean(false); - AllocationService strategy = createAllocationService(Settings.EMPTY, new NoopGatewayAllocator() { + AllocationService strategy = createAllocationService(Settings.EMPTY, new TestGatewayAllocator() { @Override public void allocateUnassigned(RoutingAllocation allocation) { if (allocateTest1.get() == false) { @@ -677,7 +677,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase { public void testRebalanceWhileShardFetching() { final AtomicBoolean hasFetches = new AtomicBoolean(true); AllocationService strategy = createAllocationService(Settings.builder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), - ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString()).build(), new NoopGatewayAllocator() { + ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString()).build(), new TestGatewayAllocator() { @Override public void allocateUnassigned(RoutingAllocation allocation) { if (hasFetches.get()) { diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/DecisionsImpactOnClusterHealthTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/DecisionsImpactOnClusterHealthTests.java index a6897d972c4..bee2275743b 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/DecisionsImpactOnClusterHealthTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/DecisionsImpactOnClusterHealthTests.java @@ -22,6 +22,7 @@ package org.elasticsearch.cluster.routing.allocation; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.EmptyClusterInfoService; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterStateHealth; @@ -38,8 +39,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; -import org.elasticsearch.cluster.ESAllocationTestCase; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.io.IOException; import java.util.Collections; @@ -161,7 +161,7 @@ public class DecisionsImpactOnClusterHealthTests extends ESAllocationTestCase { private static AllocationService newAllocationService(Settings settings, Set deciders) { return new AllocationService(settings, new AllocationDeciders(settings, deciders), - NoopGatewayAllocator.INSTANCE, + new TestGatewayAllocator(), new BalancedShardsAllocator(settings), EmptyClusterInfoService.INSTANCE); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedShardsRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedShardsRoutingTests.java index da6f103f51c..667ae850bfa 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedShardsRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedShardsRoutingTests.java @@ -34,12 +34,14 @@ import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationComman import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.shard.ShardId; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.Set; +import static org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; @@ -69,7 +71,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { RoutingTable routingTable = RoutingTable.builder() .addAsNew(metaData.index("test")) .build(); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build(); + ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build(); logger.info("--> adding 2 nodes on same rack and do rerouting"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder() @@ -154,7 +156,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { .addAsNew(metaData.index("test")) .build(); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); + ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); logger.info("Adding two nodes and performing rerouting"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build(); @@ -227,7 +229,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { .addAsNew(metaData.index("test")) .build(); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); + ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); logger.info("Adding single node and performing rerouting"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build(); @@ -278,7 +280,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { .addAsNew(metaData.index("test")) .build(); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); + ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); logger.info("Adding {} nodes and performing rerouting", numberOfReplicas + 1); DiscoveryNodes.Builder nodeBuilder = DiscoveryNodes.builder(); @@ -294,7 +296,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { } int shardsToFail = randomIntBetween(1, numberOfReplicas); - ArrayList failedShards = new ArrayList<>(); + ArrayList failedShards = new ArrayList<>(); RoutingNodes routingNodes = clusterState.getRoutingNodes(); Set failedNodes = new HashSet<>(); Set shardRoutingsToFail = new HashSet<>(); @@ -303,7 +305,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { logger.info("failing shard on node [{}]", failedNode); ShardRouting shardToFail = routingNodes.node(failedNode).iterator().next(); if (shardRoutingsToFail.contains(shardToFail) == false) { - failedShards.add(new FailedRerouteAllocation.FailedShard(shardToFail, null, null)); + failedShards.add(new FailedShard(shardToFail, null, null)); failedNodes.add(failedNode); shardRoutingsToFail.add(shardToFail); } @@ -311,8 +313,9 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { clusterState = strategy.applyFailedShards(clusterState, failedShards); routingNodes = clusterState.getRoutingNodes(); - for (FailedRerouteAllocation.FailedShard failedShard : failedShards) { - if (routingNodes.getByAllocationId(failedShard.routingEntry.shardId(), failedShard.routingEntry.allocationId().getId()) != null) { + for (FailedShard failedShard : failedShards) { + if (routingNodes.getByAllocationId(failedShard.getRoutingEntry().shardId(), + failedShard.getRoutingEntry().allocationId().getId()) != null) { fail("shard " + failedShard + " was not failed"); } } @@ -340,7 +343,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { .addAsNew(metaData.index("test")) .build(); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); + ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); logger.info("Adding two nodes and performing rerouting"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build(); @@ -395,7 +398,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { .addAsNew(metaData.index("test")) .build(); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); + ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); logger.info("Adding two nodes and performing rerouting"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build(); @@ -479,7 +482,10 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { .addAsNew(metaData.index("test")) .build(); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build(); + ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metaData(metaData).routingTable(routingTable).build(); + + ShardId shardId = new ShardId(metaData.index("test").getIndex(), 0); // add 4 nodes clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2")).add(newNode("node3")).add(newNode("node4"))).build(); @@ -491,22 +497,26 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2)); + // start one replica so it can take over. + clusterState = allocation.applyStartedShards(clusterState, + Collections.singletonList(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0))); + assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2)); + assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); + ShardRouting startedReplica = clusterState.getRoutingNodes().activeReplica(shardId); + + // fail the primary shard, check replicas get removed as well... ShardRouting primaryShardToFail = clusterState.routingTable().index("test").shard(0).primaryShard(); ClusterState newState = allocation.applyFailedShard(clusterState, primaryShardToFail); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; - // the primary gets allocated on another node, replicas are unassigned - assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); - assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2)); + // the primary gets allocated on another node, replicas are initializing + assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); + assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2)); ShardRouting newPrimaryShard = clusterState.routingTable().index("test").shard(0).primaryShard(); assertThat(newPrimaryShard, not(equalTo(primaryShardToFail))); - - // start the primary shard - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); - assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); - assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2)); + assertThat(newPrimaryShard.allocationId(), equalTo(startedReplica.allocationId())); } public void testFailAllReplicasInitializingOnPrimaryFailWhileHavingAReplicaToElect() { @@ -521,7 +531,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase { .addAsNew(metaData.index("test")) .build(); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build(); + ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build(); // add 4 nodes clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2")).add(newNode("node3")).add(newNode("node4"))).build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/FilterAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/FilterAllocationDeciderTests.java index 88ac089c7d8..3a792ae991c 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/FilterAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/FilterAllocationDeciderTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.EmptyClusterInfoService; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.RoutingTable; @@ -32,13 +33,15 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ReplicaAfterPrimaryActiveAllocationDecider; +import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.util.Arrays; +import java.util.Collections; import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_SHRINK_SOURCE_NAME; import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_SHRINK_SOURCE_UUID; @@ -52,9 +55,10 @@ public class FilterAllocationDeciderTests extends ESAllocationTestCase { FilterAllocationDecider filterAllocationDecider = new FilterAllocationDecider(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); AllocationDeciders allocationDeciders = new AllocationDeciders(Settings.EMPTY, - Arrays.asList(filterAllocationDecider, new ReplicaAfterPrimaryActiveAllocationDecider(Settings.EMPTY))); + Arrays.asList(filterAllocationDecider, + new SameShardAllocationDecider(Settings.EMPTY), new ReplicaAfterPrimaryActiveAllocationDecider(Settings.EMPTY))); AllocationService service = new AllocationService(Settings.builder().build(), allocationDeciders, - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); + new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); ClusterState state = createInitialClusterState(service, Settings.builder().put("index.routing.allocation.initial_recovery._id", "node2").build()); RoutingTable routingTable = state.routingTable(); @@ -72,10 +76,10 @@ public class FilterAllocationDeciderTests extends ESAllocationTestCase { null, 0, false); assertEquals(filterAllocationDecider.canAllocate(routingTable.index("idx").shard(0).primaryShard(), state.getRoutingNodes().node("node2") - ,allocation), Decision.YES); + , allocation), Decision.YES); assertEquals(filterAllocationDecider.canAllocate(routingTable.index("idx").shard(0).primaryShard(), state.getRoutingNodes().node("node1") - ,allocation), Decision.NO); + , allocation), Decision.NO); state = service.reroute(state, "try allocate again"); routingTable = state.routingTable(); @@ -86,57 +90,80 @@ public class FilterAllocationDeciderTests extends ESAllocationTestCase { routingTable = state.routingTable(); // ok now we are started and can be allocated anywhere!! lets see... - assertEquals(routingTable.index("idx").shard(0).primaryShard().state(), STARTED); - assertEquals(routingTable.index("idx").shard(0).primaryShard().currentNodeId(), "node2"); - - // replicas should be initializing + // first create another copy assertEquals(routingTable.index("idx").shard(0).replicaShards().get(0).state(), INITIALIZING); assertEquals(routingTable.index("idx").shard(0).replicaShards().get(0).currentNodeId(), "node1"); - - // we fail it again to check if we are initializing immediately on the other node - state = service.applyFailedShard(state, routingTable.index("idx").shard(0).shards().get(0)); + state = service.applyStartedShards(state, routingTable.index("idx").shard(0).replicaShardsWithState(INITIALIZING)); routingTable = state.routingTable(); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).state(), INITIALIZING); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).currentNodeId(), "node1"); + assertEquals(routingTable.index("idx").shard(0).replicaShards().get(0).state(), STARTED); + assertEquals(routingTable.index("idx").shard(0).replicaShards().get(0).currentNodeId(), "node1"); + + // now remove the node of the other copy and fail the current + DiscoveryNode node1 = state.nodes().resolveNode("node1"); + state = service.deassociateDeadNodes( + ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).remove("node1")).build(), + true, "test"); + state = service.applyFailedShard(state, routingTable.index("idx").shard(0).primaryShard()); + + // now bring back node1 and see it's assigned + state = service.reroute( + ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).add(node1)).build(), "test"); + routingTable = state.routingTable(); + assertEquals(routingTable.index("idx").shard(0).primaryShard().state(), INITIALIZING); + assertEquals(routingTable.index("idx").shard(0).primaryShard().currentNodeId(), "node1"); allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state, null, 0, false); assertEquals(filterAllocationDecider.canAllocate(routingTable.index("idx").shard(0).shards().get(0), state.getRoutingNodes().node("node2") - ,allocation), Decision.YES); + , allocation), Decision.YES); assertEquals(filterAllocationDecider.canAllocate(routingTable.index("idx").shard(0).shards().get(0), state.getRoutingNodes().node("node1") - ,allocation), Decision.YES); + , allocation), Decision.YES); } private ClusterState createInitialClusterState(AllocationService service, Settings settings) { - boolean shrinkIndex = randomBoolean(); + RecoverySource.Type recoveryType = randomFrom(RecoverySource.Type.EMPTY_STORE, + RecoverySource.Type.LOCAL_SHARDS, RecoverySource.Type.SNAPSHOT); MetaData.Builder metaData = MetaData.builder(); final Settings.Builder indexSettings = settings(Version.CURRENT).put(settings); final IndexMetaData sourceIndex; - if (shrinkIndex) { + if (recoveryType == RecoverySource.Type.LOCAL_SHARDS) { //put a fake closed source index sourceIndex = IndexMetaData.builder("sourceIndex") - .settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0).build(); + .settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0) + .putInSyncAllocationIds(0, Collections.singleton("aid0")) + .putInSyncAllocationIds(1, Collections.singleton("aid1")) + .build(); metaData.put(sourceIndex, false); indexSettings.put(INDEX_SHRINK_SOURCE_UUID.getKey(), sourceIndex.getIndexUUID()); indexSettings.put(INDEX_SHRINK_SOURCE_NAME.getKey(), sourceIndex.getIndex().getName()); } else { sourceIndex = null; } - final IndexMetaData indexMetaData = IndexMetaData.builder("idx").settings(indexSettings) - .numberOfShards(1).numberOfReplicas(1).build(); + final IndexMetaData.Builder indexMetaDataBuilder = IndexMetaData.builder("idx").settings(indexSettings) + .numberOfShards(1).numberOfReplicas(1); + if (recoveryType == RecoverySource.Type.SNAPSHOT) { + indexMetaDataBuilder.putInSyncAllocationIds(0, Collections.singleton("_snapshot_restore")); + } + final IndexMetaData indexMetaData = indexMetaDataBuilder.build(); metaData.put(indexMetaData, false); RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); - if (shrinkIndex) { - routingTableBuilder.addAsFromCloseToOpen(sourceIndex); - routingTableBuilder.addAsNew(indexMetaData); - } if (randomBoolean()) { - routingTableBuilder.addAsNew(indexMetaData); - } else { - routingTableBuilder.addAsRestore(indexMetaData, new RecoverySource.SnapshotRecoverySource( - new Snapshot("repository", new SnapshotId("snapshot_name", "snapshot_uuid")), - Version.CURRENT, indexMetaData.getIndex().getName())); + switch (recoveryType) { + case EMPTY_STORE: + routingTableBuilder.addAsNew(indexMetaData); + break; + case SNAPSHOT: + routingTableBuilder.addAsRestore(indexMetaData, new RecoverySource.SnapshotRecoverySource( + new Snapshot("repository", new SnapshotId("snapshot_name", "snapshot_uuid")), + Version.CURRENT, indexMetaData.getIndex().getName())); + break; + case LOCAL_SHARDS: + routingTableBuilder.addAsFromCloseToOpen(sourceIndex); + routingTableBuilder.addAsNew(indexMetaData); + break; + default: + throw new UnsupportedOperationException(recoveryType + " is not supported"); } RoutingTable routingTable = routingTableBuilder.build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java index fe131e924cd..31e2330a600 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java @@ -35,7 +35,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.util.Collections; import java.util.List; @@ -55,7 +55,7 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { super.setUp(); strategy = new AllocationService(Settings.builder().build(), new AllocationDeciders(Settings.EMPTY, Collections.singleton(new MaxRetryAllocationDecider(Settings.EMPTY))), - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); + new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); } private ClusterState createInitialClusterState() { @@ -88,8 +88,8 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { final int retries = MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY.get(Settings.EMPTY); // now fail it N-1 times for (int i = 0; i < retries-1; i++) { - List failedShards = Collections.singletonList( - new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom" + i, + List failedShards = Collections.singletonList( + new FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom" + i, new UnsupportedOperationException())); ClusterState newState = strategy.applyFailedShards(clusterState, failedShards); assertThat(newState, not(equalTo(clusterState))); @@ -101,8 +101,8 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { assertEquals(routingTable.index("idx").shard(0).shards().get(0).unassignedInfo().getMessage(), "boom" + i); } // now we go and check that we are actually stick to unassigned on the next failure - List failedShards = Collections.singletonList( - new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom", + List failedShards = Collections.singletonList( + new FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom", new UnsupportedOperationException())); ClusterState newState = strategy.applyFailedShards(clusterState, failedShards); assertThat(newState, not(equalTo(clusterState))); @@ -127,7 +127,7 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { // now we go and check that we are actually stick to unassigned on the next failure ie. no retry failedShards = Collections.singletonList( - new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom", + new FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom", new UnsupportedOperationException())); newState = strategy.applyFailedShards(clusterState, failedShards); @@ -147,8 +147,8 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { final int retries = MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY.get(Settings.EMPTY); // now fail it N-1 times for (int i = 0; i < retries-1; i++) { - List failedShards = Collections.singletonList( - new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom" + i, + List failedShards = Collections.singletonList( + new FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom" + i, new UnsupportedOperationException())); ClusterState newState = strategy.applyFailedShards(clusterState, failedShards); assertThat(newState, not(equalTo(clusterState))); @@ -165,8 +165,8 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { } // now we go and check that we are actually stick to unassigned on the next failure { - List failedShards = Collections.singletonList( - new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom", + List failedShards = Collections.singletonList( + new FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom", new UnsupportedOperationException())); ClusterState newState = strategy.applyFailedShards(clusterState, failedShards); assertThat(newState, not(equalTo(clusterState))); @@ -204,9 +204,9 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { routingTable.index("idx").shard(0).shards().get(0), null, new RoutingAllocation(null, null, clusterState, null, 0, false))); // now we start the shard - routingTable = strategy.applyStartedShards(clusterState, Collections.singletonList( - routingTable.index("idx").shard(0).shards().get(0))).routingTable(); - clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); + clusterState = strategy.applyStartedShards(clusterState, Collections.singletonList( + routingTable.index("idx").shard(0).shards().get(0))); + routingTable = clusterState.routingTable(); // all counters have been reset to 0 ie. no unassigned info assertEquals(routingTable.index("idx").shards().size(), 1); @@ -214,8 +214,8 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { assertEquals(routingTable.index("idx").shard(0).shards().get(0).state(), STARTED); // now fail again and see if it has a new counter - List failedShards = Collections.singletonList( - new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "ZOOOMG", + List failedShards = Collections.singletonList( + new FailedShard(routingTable.index("idx").shard(0).shards().get(0), "ZOOOMG", new UnsupportedOperationException())); newState = strategy.applyFailedShards(clusterState, failedShards); assertThat(newState, not(equalTo(clusterState))); @@ -224,7 +224,7 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { assertEquals(routingTable.index("idx").shards().size(), 1); unassignedPrimary = routingTable.index("idx").shard(0).shards().get(0); assertEquals(unassignedPrimary.unassignedInfo().getNumFailedAllocations(), 1); - assertEquals(unassignedPrimary.state(), INITIALIZING); + assertEquals(unassignedPrimary.state(), UNASSIGNED); assertEquals(unassignedPrimary.unassignedInfo().getMessage(), "ZOOOMG"); // Counter reset, so MaxRetryAllocationDecider#canForceAllocatePrimary should return a YES decision assertEquals(Decision.YES, new MaxRetryAllocationDecider(Settings.EMPTY).canForceAllocatePrimary( diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java index ab7e31ba431..b472cc5e0d2 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java @@ -53,7 +53,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.VersionUtils; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.util.ArrayList; import java.util.Arrays; @@ -326,7 +326,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { AllocationDeciders allocationDeciders = new AllocationDeciders(Settings.EMPTY, Collections.singleton(new NodeVersionAllocationDecider(Settings.EMPTY))); AllocationService strategy = new MockAllocationService(Settings.EMPTY, allocationDeciders, - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); + new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); state = strategy.reroute(state, new AllocationCommands(), true, false).getClusterState(); // the two indices must stay as is, the replicas cannot move to oldNode2 because versions don't match assertThat(state.routingTable().index(shard2.getIndex()).shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(0)); @@ -342,10 +342,12 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { MASTER_DATA_ROLES, VersionUtils.getPreviousVersion()); int numberOfShards = randomIntBetween(1, 3); - MetaData metaData = MetaData.builder() - .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(numberOfShards).numberOfReplicas - (randomIntBetween(0, 3))) - .build(); + final IndexMetaData.Builder indexMetaData = IndexMetaData.builder("test").settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards).numberOfReplicas(randomIntBetween(0, 3)); + for (int i = 0; i < numberOfShards; i++) { + indexMetaData.putInSyncAllocationIds(i, Collections.singleton("_test_")); + } + MetaData metaData = MetaData.builder().put(indexMetaData).build(); ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) .metaData(metaData) @@ -358,7 +360,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { new NodeVersionAllocationDecider(Settings.EMPTY))); AllocationService strategy = new MockAllocationService(Settings.EMPTY, allocationDeciders, - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); + new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); state = strategy.reroute(state, new AllocationCommands(), true, false).getClusterState(); // Make sure that primary shards are only allocated on the new node diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java index 10f1cf69355..7e528e601d3 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java @@ -38,6 +38,7 @@ import static org.hamcrest.Matchers.equalTo; /** */ public class PreferLocalPrimariesToRelocatingPrimariesTests extends ESAllocationTestCase { + public void testPreferLocalPrimaryAllocationOverFiltered() { int concurrentRecoveries = randomIntBetween(1, 10); int primaryRecoveries = randomIntBetween(1, 10); @@ -66,8 +67,7 @@ public class PreferLocalPrimariesToRelocatingPrimariesTests extends ESAllocation logger.info("adding two nodes and performing rerouting till all are allocated"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder() - .add(newNode("node1", singletonMap("tag1", "value1"))) - .add(newNode("node2", singletonMap("tag1", "value2")))).build(); + .add(newNode("node1")).add(newNode("node2"))).build(); clusterState = strategy.reroute(clusterState, "reroute"); @@ -82,12 +82,12 @@ public class PreferLocalPrimariesToRelocatingPrimariesTests extends ESAllocation .put(IndexMetaData.builder(clusterState.metaData().index("test1")).settings(settings(Version.CURRENT) .put("index.number_of_shards", numberOfShards) .put("index.number_of_replicas", 0) - .put("index.routing.allocation.exclude.tag1", "value2") + .put("index.routing.allocation.exclude._name", "node2") .build())) .put(IndexMetaData.builder(clusterState.metaData().index("test2")).settings(settings(Version.CURRENT) .put("index.number_of_shards", numberOfShards) .put("index.number_of_replicas", 0) - .put("index.routing.allocation.exclude.tag1", "value2") + .put("index.routing.allocation.exclude._name", "node2") .build())) .build(); clusterState = ClusterState.builder(clusterState).metaData(metaData).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node1")).build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java index ca9d61997b0..d789e6c4ec6 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java @@ -33,6 +33,7 @@ import org.elasticsearch.common.settings.Settings; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; +import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; @@ -128,8 +129,9 @@ public class PrimaryElectionRoutingTests extends ESAllocationTestCase { routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.shardsWithState(STARTED).size(), equalTo(1)); - assertThat(routingNodes.shardsWithState(INITIALIZING).size(), equalTo(1)); - assertThat(routingNodes.node(nodeIdRemaining).shardsWithState(INITIALIZING).get(0).primary(), equalTo(true)); + assertThat(routingNodes.shardsWithState(INITIALIZING).size(), equalTo(0)); + assertThat(routingNodes.shardsWithState(UNASSIGNED).size(), equalTo(3)); // 2 replicas and one primary + assertThat(routingNodes.node(nodeIdRemaining).shardsWithState(STARTED).get(0).primary(), equalTo(true)); assertThat(clusterState.metaData().index("test").primaryTerm(0), equalTo(2L)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java index 53d54c1835d..6722e048030 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java @@ -38,10 +38,11 @@ import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.routing.allocation.decider.ReplicaAfterPrimaryActiveAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import org.hamcrest.Matchers; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.Random; @@ -58,7 +59,7 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase { RandomAllocationDecider randomAllocationDecider = new RandomAllocationDecider(random()); AllocationService strategy = new AllocationService(Settings.builder().build(), new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList(new SameShardAllocationDecider(Settings.EMPTY), new ReplicaAfterPrimaryActiveAllocationDecider(Settings.EMPTY), - randomAllocationDecider))), NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); + randomAllocationDecider))), new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE); int indices = scaledRandomIntBetween(1, 20); Builder metaBuilder = MetaData.builder(); int maxNumReplicas = 1; @@ -101,9 +102,25 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase { boolean nodesRemoved = false; if (nodeIdCounter > 1 && rarely()) { int nodeId = scaledRandomIntBetween(0, nodeIdCounter - 2); - logger.info("removing node [{}]", nodeId); - newNodesBuilder.remove("NODE_" + nodeId); - nodesRemoved = true; + final String node = "NODE_" + nodeId; + boolean safeToRemove = true; + RoutingNode routingNode = clusterState.getRoutingNodes().node(node); + for (ShardRouting shard: routingNode != null ? routingNode : Collections.emptyList()) { + if (shard.active() && shard.primary()) { + // make sure there is an active replica to prevent from going red + if (clusterState.routingTable().shardRoutingTable(shard.shardId()).activeShards().size() <= 1) { + safeToRemove = false; + break; + } + } + } + if (safeToRemove) { + logger.info("removing node [{}]", nodeId); + newNodesBuilder.remove(node); + nodesRemoved = true; + } else { + logger.debug("not removing node [{}] as it holds a primary with no replacement", nodeId); + } } stateBuilder.nodes(newNodesBuilder.build()); @@ -142,7 +159,7 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase { } while (clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size() != 0 || clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size() != 0 && iterations < 200); - logger.info("Done Balancing after [{}] iterations", iterations); + logger.info("Done Balancing after [{}] iterations. State:\n{}", iterations, clusterState.prettyPrint()); // we stop after 200 iterations if it didn't stabelize by then something is likely to be wrong assertThat("max num iteration exceeded", iterations, Matchers.lessThan(200)); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(0)); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java index 33088bcce99..dd89d6b6a52 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java @@ -117,27 +117,41 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase { assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(STARTED)); assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1")); - logger.info("Killing node1 where the shard is, checking the shard is relocated"); + logger.info("Killing node1 where the shard is, checking the shard is unassigned"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node1")).build(); newState = strategy.deassociateDeadNodes(clusterState, true, "reroute"); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; + assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1)); + assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1)); + assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1)); + assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED)); + assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), nullValue()); + + logger.info("Bring node1 back, and see it's assinged"); + + clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node1"))).build(); + newState = strategy.reroute(clusterState, "reroute"); + assertThat(newState, not(equalTo(clusterState))); + clusterState = newState; + assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1)); assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1)); assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1)); assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(INITIALIZING)); - assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node2")); + assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1")); + logger.info("Start another node, make sure that things remain the same (shard is in node2 and initializing)"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build(); newState = strategy.reroute(clusterState, "reroute"); assertThat(newState, equalTo(clusterState)); - logger.info("Start the shard on node 2"); + logger.info("Start the shard on node 1"); routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)); + newState = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -145,7 +159,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase { assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1)); assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1)); assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(STARTED)); - assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node2")); + assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1")); } public void testSingleIndexShardFailed() { diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java index d8d00e3915f..454e8410484 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java @@ -69,14 +69,15 @@ public class StartedShardsRoutingTests extends ESAllocationTestCase { logger.info("--> test starting of shard"); - ClusterState newState = allocation.applyStartedShards(state, Arrays.asList(initShard), false); + ClusterState newState = allocation.applyStartedShards(state, Arrays.asList(initShard)); assertThat("failed to start " + initShard + "\ncurrent routing table:" + newState.routingTable().prettyPrint(), newState, not(equalTo(state))); assertTrue(initShard + "isn't started \ncurrent routing table:" + newState.routingTable().prettyPrint(), newState.routingTable().index("test").shard(initShard.id()).allShardsStarted()); + state = newState; logger.info("--> testing starting of relocating shards"); - newState = allocation.applyStartedShards(state, Arrays.asList(relocatingShard.getTargetRelocatingShard()), false); + newState = allocation.applyStartedShards(state, Arrays.asList(relocatingShard.getTargetRelocatingShard())); assertThat("failed to start " + relocatingShard + "\ncurrent routing table:" + newState.routingTable().prettyPrint(), newState, not(equalTo(state))); ShardRouting shardRouting = newState.routingTable().index("test").shard(relocatingShard.id()).getShards().get(0); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java index d6aafbda34a..894b5b42f0c 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java @@ -20,23 +20,35 @@ package org.elasticsearch.cluster.routing.allocation; import com.carrotsearch.hppc.IntHashSet; +import com.carrotsearch.hppc.cursors.ObjectCursor; import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; import org.elasticsearch.cluster.routing.RoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingHelper; +import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.test.gateway.TestGatewayAllocator; +import java.util.Collections; + +import static org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; @@ -50,10 +62,12 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase { private final Logger logger = Loggers.getLogger(ThrottlingAllocationTests.class); public void testPrimaryRecoveryThrottling() { + + TestGatewayAllocator gatewayAllocator = new TestGatewayAllocator(); AllocationService strategy = createAllocationService(Settings.builder() .put("cluster.routing.allocation.node_concurrent_recoveries", 3) .put("cluster.routing.allocation.node_initial_primaries_recoveries", 3) - .build()); + .build(), gatewayAllocator); logger.info("Building initial routing table"); @@ -61,9 +75,7 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase { .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(10).numberOfReplicas(1)) .build(); - RoutingTable initialRoutingTable = createRecoveryRoutingTable(metaData.index("test")); - - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); + ClusterState clusterState = createRecoveryStateAndInitalizeAllocations(metaData, gatewayAllocator); logger.info("start one node, do reroute, only 3 should initialize"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build(); @@ -103,11 +115,13 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase { } public void testReplicaAndPrimaryRecoveryThrottling() { + TestGatewayAllocator gatewayAllocator = new TestGatewayAllocator(); AllocationService strategy = createAllocationService(Settings.builder() .put("cluster.routing.allocation.node_concurrent_recoveries", 3) .put("cluster.routing.allocation.concurrent_source_recoveries", 3) .put("cluster.routing.allocation.node_initial_primaries_recoveries", 3) - .build()); + .build(), + gatewayAllocator); logger.info("Building initial routing table"); @@ -115,12 +129,9 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase { .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1)) .build(); - RoutingTable initialRoutingTable = createRecoveryRoutingTable(metaData.index("test")); + ClusterState clusterState = createRecoveryStateAndInitalizeAllocations(metaData, gatewayAllocator); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); - - logger.info("start one node, do reroute, only 3 should initialize"); - clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build(); + logger.info("with one node, do reroute, only 3 should initialize"); clusterState = strategy.reroute(clusterState, "reroute"); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(0)); @@ -165,24 +176,22 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase { } public void testThrottleIncomingAndOutgoing() { + TestGatewayAllocator gatewayAllocator = new TestGatewayAllocator(); Settings settings = Settings.builder() .put("cluster.routing.allocation.node_concurrent_recoveries", 5) .put("cluster.routing.allocation.node_initial_primaries_recoveries", 5) .put("cluster.routing.allocation.cluster_concurrent_rebalance", 5) .build(); - AllocationService strategy = createAllocationService(settings); + AllocationService strategy = createAllocationService(settings, gatewayAllocator); logger.info("Building initial routing table"); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(9).numberOfReplicas(0)) .build(); - RoutingTable initialRoutingTable = createRecoveryRoutingTable(metaData.index("test")); + ClusterState clusterState = createRecoveryStateAndInitalizeAllocations(metaData, gatewayAllocator); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); - - logger.info("start one node, do reroute, only 5 should initialize"); - clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build(); + logger.info("with one node, do reroute, only 5 should initialize"); clusterState = strategy.reroute(clusterState, "reroute"); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(0)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(5)); @@ -225,9 +234,10 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase { } public void testOutgoingThrottlesAllocation() { + TestGatewayAllocator gatewayAllocator = new TestGatewayAllocator(); AllocationService strategy = createAllocationService(Settings.builder() .put("cluster.routing.allocation.node_concurrent_outgoing_recoveries", 1) - .build()); + .build(), gatewayAllocator); logger.info("Building initial routing table"); @@ -235,12 +245,9 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase { .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2)) .build(); - RoutingTable initialRoutingTable = createRecoveryRoutingTable(metaData.index("test")); + ClusterState clusterState = createRecoveryStateAndInitalizeAllocations(metaData, gatewayAllocator); - ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build(); - - logger.info("start one node, do reroute, only 1 should initialize"); - clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build(); + logger.info("with one node, do reroute, only 1 should initialize"); clusterState = strategy.reroute(clusterState, "reroute"); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(0)); @@ -301,23 +308,66 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase { assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node2"), 0); } - private RoutingTable createRecoveryRoutingTable(IndexMetaData indexMetaData) { + private ClusterState createRecoveryStateAndInitalizeAllocations(MetaData metaData, TestGatewayAllocator gatewayAllocator) { + DiscoveryNode node1 = newNode("node1"); + MetaData.Builder metaDataBuilder = new MetaData.Builder(metaData); RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); - switch (randomInt(5)) { - case 0: routingTableBuilder.addAsRecovery(indexMetaData); break; - case 1: routingTableBuilder.addAsFromCloseToOpen(indexMetaData); break; - case 2: routingTableBuilder.addAsFromDangling(indexMetaData); break; - case 3: routingTableBuilder.addAsNewRestore(indexMetaData, - new SnapshotRecoverySource(new Snapshot("repo", new SnapshotId("snap", "randomId")), Version.CURRENT, - indexMetaData.getIndex().getName()), new IntHashSet()); break; - case 4: routingTableBuilder.addAsRestore(indexMetaData, - new SnapshotRecoverySource(new Snapshot("repo", new SnapshotId("snap", "randomId")), Version.CURRENT, - indexMetaData.getIndex().getName())); break; - case 5: routingTableBuilder.addAsNew(indexMetaData); break; - default: throw new IndexOutOfBoundsException(); + for (ObjectCursor cursor: metaData.indices().values()) { + Index index = cursor.value.getIndex(); + IndexMetaData.Builder indexMetaDataBuilder = IndexMetaData.builder(cursor.value); + final int recoveryType = randomInt(5); + if (recoveryType <= 4) { + addInSyncAllocationIds(index, indexMetaDataBuilder, gatewayAllocator, node1); + } + IndexMetaData indexMetaData = indexMetaDataBuilder.build(); + metaDataBuilder.put(indexMetaData, false); + switch (recoveryType) { + case 0: + routingTableBuilder.addAsRecovery(indexMetaData); + break; + case 1: + routingTableBuilder.addAsFromCloseToOpen(indexMetaData); + break; + case 2: + routingTableBuilder.addAsFromDangling(indexMetaData); + break; + case 3: + routingTableBuilder.addAsNewRestore(indexMetaData, + new SnapshotRecoverySource(new Snapshot("repo", new SnapshotId("snap", "randomId")), Version.CURRENT, + indexMetaData.getIndex().getName()), new IntHashSet()); + break; + case 4: + routingTableBuilder.addAsRestore(indexMetaData, + new SnapshotRecoverySource(new Snapshot("repo", new SnapshotId("snap", "randomId")), Version.CURRENT, + indexMetaData.getIndex().getName())); + break; + case 5: + routingTableBuilder.addAsNew(indexMetaData); + break; + default: + throw new IndexOutOfBoundsException(); + } } - - return routingTableBuilder.build(); + return ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .nodes(DiscoveryNodes.builder().add(node1)) + .metaData(metaDataBuilder.build()) + .routingTable(routingTableBuilder.build()).build(); } + private void addInSyncAllocationIds(Index index, IndexMetaData.Builder indexMetaData, + TestGatewayAllocator gatewayAllocator, DiscoveryNode node1) { + for (int shard = 0; shard < indexMetaData.numberOfShards(); shard++) { + + final boolean primary = randomBoolean(); + final ShardRouting unassigned = ShardRouting.newUnassigned(new ShardId(index, shard), primary, + primary ? + RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : + RecoverySource.PeerRecoverySource.INSTANCE, + new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "test") + ); + ShardRouting started = ShardRoutingHelper.moveToStarted(ShardRoutingHelper.initialize(unassigned, node1.getId())); + indexMetaData.putInSyncAllocationIds(shard, Collections.singleton(started.allocationId().getId())); + gatewayAllocator.addKnownAllocation(started); + } + } } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index 65fb1ac0020..062a018a82d 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -50,7 +50,7 @@ import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.util.Arrays; import java.util.HashMap; @@ -113,7 +113,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1)) @@ -194,7 +194,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); clusterState = strategy.reroute(clusterState, "reroute"); logShardStates(clusterState); @@ -224,7 +224,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); clusterState = strategy.reroute(clusterState, "reroute"); @@ -301,7 +301,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2)) @@ -358,7 +358,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); clusterState = strategy.reroute(clusterState, "reroute"); logShardStates(clusterState); @@ -421,7 +421,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); clusterState = strategy.reroute(clusterState, "reroute"); logShardStates(clusterState); @@ -451,7 +451,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); clusterState = strategy.reroute(clusterState, "reroute"); @@ -555,7 +555,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0)) @@ -625,7 +625,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0)) @@ -729,7 +729,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1)) @@ -900,7 +900,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put("cluster.routing.allocation.node_concurrent_recoveries", 10) .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); // Ensure that the reroute call doesn't alter the routing table, since the first primary is relocating away // and therefor we will have sufficient disk space on node1. ClusterState result = strategy.reroute(clusterState, "reroute"); @@ -998,7 +998,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always") .put("cluster.routing.allocation.cluster_concurrent_rebalance", -1) - .build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis); + .build(), deciders, new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), cis); ClusterState result = strategy.reroute(clusterState, "reroute"); assertThat(result.routingTable().index("test").getShards().get(0).primaryShard().state(), equalTo(STARTED)); diff --git a/core/src/test/java/org/elasticsearch/common/unit/ByteSizeUnitTests.java b/core/src/test/java/org/elasticsearch/common/unit/ByteSizeUnitTests.java index ab6d2818946..719313d1c86 100644 --- a/core/src/test/java/org/elasticsearch/common/unit/ByteSizeUnitTests.java +++ b/core/src/test/java/org/elasticsearch/common/unit/ByteSizeUnitTests.java @@ -19,20 +19,23 @@ package org.elasticsearch.common.unit; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ESTestCase; +import java.io.IOException; + import static org.elasticsearch.common.unit.ByteSizeUnit.BYTES; import static org.elasticsearch.common.unit.ByteSizeUnit.GB; import static org.elasticsearch.common.unit.ByteSizeUnit.KB; import static org.elasticsearch.common.unit.ByteSizeUnit.MB; import static org.elasticsearch.common.unit.ByteSizeUnit.PB; import static org.elasticsearch.common.unit.ByteSizeUnit.TB; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; -/** - * - */ public class ByteSizeUnitTests extends ESTestCase { + public void testBytes() { assertThat(BYTES.toBytes(1), equalTo(1L)); assertThat(BYTES.toKB(1024), equalTo(1L)); @@ -77,4 +80,23 @@ public class ByteSizeUnitTests extends ESTestCase { assertThat(PB.toTB(1), equalTo(1024L)); assertThat(PB.toPB(1), equalTo(1L)); } + + public void testSerialization() throws IOException { + for (ByteSizeUnit unit : ByteSizeUnit.values()) { + try (BytesStreamOutput out = new BytesStreamOutput()) { + unit.writeTo(out); + + try (StreamInput in = out.bytes().streamInput()) { + ByteSizeUnit deserialized = ByteSizeUnit.readFrom(in); + assertEquals(unit, deserialized); + } + } + } + } + + public void testFromUnknownId() throws IOException { + final byte randomId = (byte) randomIntBetween(ByteSizeUnit.values().length + 1, 100); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> ByteSizeUnit.fromId(randomId)); + assertThat(e.getMessage(), containsString("No byte size unit found for id [" + String.valueOf(randomId) + "]")); + } } diff --git a/core/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java b/core/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java index bef4a047ef5..cad712951d9 100644 --- a/core/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java +++ b/core/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java @@ -25,6 +25,8 @@ import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParseFieldMatcherSupplier; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.xcontent.AbstractObjectParser.ContextParser; +import org.elasticsearch.common.xcontent.AbstractObjectParser.NoContextParser; import org.elasticsearch.test.ESTestCase; import org.hamcrest.Matcher; @@ -43,6 +45,27 @@ import static org.hamcrest.Matchers.nullValue; public class ConstructingObjectParserTests extends ESTestCase { private static final ParseFieldMatcherSupplier MATCHER = () -> ParseFieldMatcher.STRICT; + public void testNullDeclares() { + ConstructingObjectParser objectParser = new ConstructingObjectParser<>("foo", a -> null); + Exception e = expectThrows(IllegalArgumentException.class, + () -> objectParser.declareField(null, (r, c) -> null, new ParseField("test"), ObjectParser.ValueType.STRING)); + assertEquals("[consumer] is required", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> objectParser.declareField( + (o, v) -> {}, (ContextParser) null, + new ParseField("test"), ObjectParser.ValueType.STRING)); + assertEquals("[parser] is required", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> objectParser.declareField( + (o, v) -> {}, (NoContextParser) null, + new ParseField("test"), ObjectParser.ValueType.STRING)); + assertEquals("[parser] is required", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> objectParser.declareField( + (o, v) -> {}, (r, c) -> null, null, ObjectParser.ValueType.STRING)); + assertEquals("[parseField] is required", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> objectParser.declareField( + (o, v) -> {}, (r, c) -> null, new ParseField("test"), null)); + assertEquals("[type] is required", e.getMessage()); + } + /** * Builds the object in random order and parses it. */ @@ -261,6 +284,25 @@ public class ConstructingObjectParserTests extends ESTestCase { assertTrue(result.fooSet); } + public void testIgnoreUnknownFields() throws IOException { + XContentParser parser = XContentType.JSON.xContent().createParser( + "{\n" + + " \"test\" : \"foo\",\n" + + " \"junk\" : 2\n" + + "}"); + class TestStruct { + public final String test; + public TestStruct(String test) { + this.test = test; + } + } + ConstructingObjectParser objectParser = new ConstructingObjectParser<>("foo", true, a -> + new TestStruct((String) a[0])); + objectParser.declareString(constructorArg(), new ParseField("test")); + TestStruct s = objectParser.apply(parser, MATCHER); + assertEquals(s.test, "foo"); + } + private static class HasCtorArguments implements ToXContent { @Nullable final String animal; diff --git a/core/src/test/java/org/elasticsearch/common/xcontent/ObjectParserTests.java b/core/src/test/java/org/elasticsearch/common/xcontent/ObjectParserTests.java index a8d26e87ecf..2cc4889be9d 100644 --- a/core/src/test/java/org/elasticsearch/common/xcontent/ObjectParserTests.java +++ b/core/src/test/java/org/elasticsearch/common/xcontent/ObjectParserTests.java @@ -18,20 +18,22 @@ */ package org.elasticsearch.common.xcontent; -import static org.hamcrest.Matchers.hasSize; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.ParseFieldMatcher; +import org.elasticsearch.common.ParseFieldMatcherSupplier; +import org.elasticsearch.common.ParsingException; +import org.elasticsearch.common.xcontent.AbstractObjectParser.ContextParser; +import org.elasticsearch.common.xcontent.AbstractObjectParser.NoContextParser; +import org.elasticsearch.common.xcontent.ObjectParser.NamedObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser.ValueType; +import org.elasticsearch.test.ESTestCase; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.ParseFieldMatcher; -import org.elasticsearch.common.ParseFieldMatcherSupplier; -import org.elasticsearch.common.ParsingException; -import org.elasticsearch.common.xcontent.ObjectParser.NamedObjectParser; -import org.elasticsearch.common.xcontent.ObjectParser.ValueType; -import org.elasticsearch.test.ESTestCase; +import static org.hamcrest.Matchers.hasSize; public class ObjectParserTests extends ESTestCase { @@ -72,6 +74,27 @@ public class ObjectParserTests extends ESTestCase { + "FieldParser{preferred_name=test_number, supportedTokens=[VALUE_STRING, VALUE_NUMBER], type=INT}]}"); } + public void testNullDeclares() { + ObjectParser objectParser = new ObjectParser<>("foo"); + Exception e = expectThrows(IllegalArgumentException.class, + () -> objectParser.declareField(null, (r, c) -> null, new ParseField("test"), ObjectParser.ValueType.STRING)); + assertEquals("[consumer] is required", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> objectParser.declareField( + (o, v) -> {}, (ContextParser) null, + new ParseField("test"), ObjectParser.ValueType.STRING)); + assertEquals("[parser] is required", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> objectParser.declareField( + (o, v) -> {}, (NoContextParser) null, + new ParseField("test"), ObjectParser.ValueType.STRING)); + assertEquals("[parser] is required", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> objectParser.declareField( + (o, v) -> {}, (r, c) -> null, null, ObjectParser.ValueType.STRING)); + assertEquals("[parseField] is required", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> objectParser.declareField( + (o, v) -> {}, (r, c) -> null, new ParseField("test"), null)); + assertEquals("[type] is required", e.getMessage()); + } + public void testObjectOrDefault() throws IOException { XContentParser parser = XContentType.JSON.xContent().createParser("{\"object\" : { \"test\": 2}}"); ObjectParser objectParser = new ObjectParser<>("foo", StaticTestStruct::new); @@ -440,6 +463,77 @@ public class ObjectParserTests extends ESTestCase { assertEquals("[named] doesn't support arrays. Use a single object with multiple fields.", e.getCause().getMessage()); } + public void testIgnoreUnknownFields() throws IOException { + XContentBuilder b = XContentBuilder.builder(XContentType.JSON.xContent()); + b.startObject(); + { + b.field("test", "foo"); + b.field("junk", 2); + } + b.endObject(); + b = shuffleXContent(b); + XContentParser parser = XContentType.JSON.xContent().createParser(b.bytes()); + + class TestStruct { + public String test; + } + ObjectParser objectParser = new ObjectParser<>("foo", true, null); + objectParser.declareField((i, c, x) -> c.test = i.text(), new ParseField("test"), ObjectParser.ValueType.STRING); + TestStruct s = objectParser.parse(parser, new TestStruct(), STRICT_PARSING); + assertEquals(s.test, "foo"); + } + + public void testIgnoreUnknownObjects() throws IOException { + XContentBuilder b = XContentBuilder.builder(XContentType.JSON.xContent()); + b.startObject(); + { + b.field("test", "foo"); + b.startObject("junk"); + { + b.field("really", "junk"); + } + b.endObject(); + } + b.endObject(); + b = shuffleXContent(b); + XContentParser parser = XContentType.JSON.xContent().createParser(b.bytes()); + + class TestStruct { + public String test; + } + ObjectParser objectParser = new ObjectParser<>("foo", true, null); + objectParser.declareField((i, c, x) -> c.test = i.text(), new ParseField("test"), ObjectParser.ValueType.STRING); + TestStruct s = objectParser.parse(parser, new TestStruct(), STRICT_PARSING); + assertEquals(s.test, "foo"); + } + + public void testIgnoreUnknownArrays() throws IOException { + XContentBuilder b = XContentBuilder.builder(XContentType.JSON.xContent()); + b.startObject(); + { + b.field("test", "foo"); + b.startArray("junk"); + { + b.startObject(); + { + b.field("really", "junk"); + } + b.endObject(); + } + b.endArray(); + } + b.endObject(); + b = shuffleXContent(b); + XContentParser parser = XContentType.JSON.xContent().createParser(b.bytes()); + class TestStruct { + public String test; + } + ObjectParser objectParser = new ObjectParser<>("foo", true, null); + objectParser.declareField((i, c, x) -> c.test = i.text(), new ParseField("test"), ObjectParser.ValueType.STRING); + TestStruct s = objectParser.parse(parser, new TestStruct(), STRICT_PARSING); + assertEquals(s.test, "foo"); + } + static class NamedObjectHolder { public static final ObjectParser PARSER = new ObjectParser<>("named_object_holder", NamedObjectHolder::new); diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index ca75ea960ad..907d3786992 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -90,7 +90,7 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; -@TestLogging("org.elasticsearch.discovery.zen:TRACE") +@TestLogging("org.elasticsearch.discovery.zen:TRACE,org.elasticsearch.cluster.service:TRACE") public class NodeJoinControllerTests extends ESTestCase { private static ThreadPool threadPool; diff --git a/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java b/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java index 080b1f0a00e..0fd89ec8898 100644 --- a/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java @@ -23,18 +23,20 @@ import org.apache.lucene.index.CorruptIndexException; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterStateHealth; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; +import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; @@ -48,7 +50,6 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardStateMetaData; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; -import org.elasticsearch.cluster.ESAllocationTestCase; import org.junit.Before; import java.util.Arrays; @@ -57,6 +58,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import static org.elasticsearch.cluster.routing.UnassignedInfo.Reason.CLUSTER_RECOVERED; +import static org.elasticsearch.cluster.routing.UnassignedInfo.Reason.INDEX_CREATED; +import static org.elasticsearch.cluster.routing.UnassignedInfo.Reason.INDEX_REOPENED; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -78,11 +82,9 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testNoProcessPrimaryNotAllocatedBefore() { final RoutingAllocation allocation; - if (randomBoolean()) { - allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), randomBoolean(), Version.CURRENT); - } else { - allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), true, Version.V_2_1_0); - } + // with old version, we can't know if a shard was allocated before or not + allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), + randomFrom(INDEX_CREATED, CLUSTER_RECOVERED, INDEX_REOPENED), Version.CURRENT); testAllocator.allocateUnassigned(allocation); assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().unassigned().size(), equalTo(1)); @@ -96,9 +98,9 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testNoAsyncFetchData() { final RoutingAllocation allocation; if (randomBoolean()) { - allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.CURRENT, "allocId"); + allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), CLUSTER_RECOVERED, Version.CURRENT, "allocId"); } else { - allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_0); + allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), CLUSTER_RECOVERED, Version.V_2_1_0); } testAllocator.allocateUnassigned(allocation); assertThat(allocation.routingNodesChanged(), equalTo(true)); @@ -114,9 +116,9 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testNoAllocationFound() { final RoutingAllocation allocation; if (randomBoolean()) { - allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.CURRENT, "allocId"); + allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), CLUSTER_RECOVERED, Version.CURRENT, "allocId"); } else { - allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_0); + allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), CLUSTER_RECOVERED, Version.V_2_1_0); } testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, null, randomBoolean()); testAllocator.allocateUnassigned(allocation); @@ -130,7 +132,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { * Tests when the node returns data with a shard allocation id that does not match active allocation ids, it will be moved to ignore unassigned. */ public void testNoMatchingAllocationIdFound() { - RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.CURRENT, "id2"); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), CLUSTER_RECOVERED, Version.CURRENT, "id2"); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, "id1", randomBoolean()); testAllocator.allocateUnassigned(allocation); assertThat(allocation.routingNodesChanged(), equalTo(true)); @@ -144,7 +146,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { * This is the case when we have old shards from pre-3.0 days. */ public void testNoActiveAllocationIds() { - RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_1); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), CLUSTER_RECOVERED, Version.V_2_1_1); testAllocator.addData(node1, 1, null, randomBoolean()); testAllocator.allocateUnassigned(allocation); assertThat(allocation.routingNodesChanged(), equalTo(true)); @@ -160,10 +162,11 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testStoreException() { final RoutingAllocation allocation; if (randomBoolean()) { - allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1"); + allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), CLUSTER_RECOVERED, + randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1"); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, "allocId1", randomBoolean(), new CorruptIndexException("test", "test")); } else { - allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_1); + allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), CLUSTER_RECOVERED, Version.V_2_1_1); testAllocator.addData(node1, 3, null, randomBoolean(), new CorruptIndexException("test", "test")); } testAllocator.allocateUnassigned(allocation); @@ -180,10 +183,12 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { final RoutingAllocation allocation; boolean useAllocationIds = randomBoolean(); if (useAllocationIds) { - allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1"); + allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), randomFrom(CLUSTER_RECOVERED, INDEX_REOPENED), + randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1"); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, "allocId1", randomBoolean()); } else { - allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_2_0); + allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), randomFrom(CLUSTER_RECOVERED, INDEX_REOPENED), + Version.V_2_2_0); testAllocator.addData(node1, 3, null, randomBoolean()); } testAllocator.allocateUnassigned(allocation); @@ -210,7 +215,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { // the allocator will see if it can force assign the primary, where the decision will be YES new TestAllocateDecision(randomBoolean() ? Decision.YES : Decision.NO), getNoDeciderThatAllowsForceAllocate() )); - RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(deciders, false, Version.CURRENT, "allocId1"); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(deciders, CLUSTER_RECOVERED, Version.CURRENT, "allocId1"); testAllocator.allocateUnassigned(allocation); assertThat(allocation.routingNodesChanged(), equalTo(true)); assertTrue(allocation.routingNodes().unassigned().ignored().isEmpty()); @@ -233,7 +238,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { new TestAllocateDecision(Decision.NO), forceDecisionNo ? getNoDeciderThatDeniesForceAllocate() : getNoDeciderThatThrottlesForceAllocate() )); - RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(deciders, false, Version.CURRENT, "allocId1"); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(deciders, CLUSTER_RECOVERED, Version.CURRENT, "allocId1"); testAllocator.allocateUnassigned(allocation); assertThat(allocation.routingNodesChanged(), equalTo(true)); List ignored = allocation.routingNodes().unassigned().ignored(); @@ -257,7 +262,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { // force allocating the shard, we still THROTTLE due to the decision from TestAllocateDecision new TestAllocateDecision(Decision.THROTTLE), getNoDeciderThatAllowsForceAllocate() )); - RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(deciders, false, Version.CURRENT, "allocId1"); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(deciders, CLUSTER_RECOVERED, Version.CURRENT, "allocId1"); testAllocator.allocateUnassigned(allocation); assertThat(allocation.routingNodesChanged(), equalTo(true)); List ignored = allocation.routingNodes().unassigned().ignored(); @@ -272,7 +277,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testPreferAllocatingPreviousPrimary() { String primaryAllocId = UUIDs.randomBase64UUID(); String replicaAllocId = UUIDs.randomBase64UUID(); - RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), primaryAllocId, replicaAllocId); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), + randomFrom(CLUSTER_RECOVERED, INDEX_REOPENED), randomFrom(Version.V_2_0_0, Version.CURRENT), primaryAllocId, replicaAllocId); boolean node1HasPrimaryShard = randomBoolean(); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, node1HasPrimaryShard ? primaryAllocId : replicaAllocId, node1HasPrimaryShard); testAllocator.addData(node2, ShardStateMetaData.NO_VERSION, node1HasPrimaryShard ? replicaAllocId : primaryAllocId, !node1HasPrimaryShard); @@ -292,10 +298,11 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testFoundAllocationButThrottlingDecider() { final RoutingAllocation allocation; if (randomBoolean()) { - allocation = routingAllocationWithOnePrimaryNoReplicas(throttleAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1"); + allocation = routingAllocationWithOnePrimaryNoReplicas(throttleAllocationDeciders(), CLUSTER_RECOVERED, + randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1"); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, "allocId1", randomBoolean()); } else { - allocation = routingAllocationWithOnePrimaryNoReplicas(throttleAllocationDeciders(), false, Version.V_2_2_0); + allocation = routingAllocationWithOnePrimaryNoReplicas(throttleAllocationDeciders(), CLUSTER_RECOVERED, Version.V_2_2_0); testAllocator.addData(node1, 3, null, randomBoolean()); } testAllocator.allocateUnassigned(allocation); @@ -312,10 +319,11 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testFoundAllocationButNoDecider() { final RoutingAllocation allocation; if (randomBoolean()) { - allocation = routingAllocationWithOnePrimaryNoReplicas(noAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1"); + allocation = routingAllocationWithOnePrimaryNoReplicas(noAllocationDeciders(), CLUSTER_RECOVERED, + randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1"); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, "allocId1", randomBoolean()); } else { - allocation = routingAllocationWithOnePrimaryNoReplicas(noAllocationDeciders(), false, Version.V_2_0_0); + allocation = routingAllocationWithOnePrimaryNoReplicas(noAllocationDeciders(), CLUSTER_RECOVERED, Version.V_2_0_0); testAllocator.addData(node1, 3, null, randomBoolean()); } testAllocator.allocateUnassigned(allocation); @@ -330,7 +338,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { * Tests that the highest version node is chosen for allocation. */ public void testAllocateToTheHighestVersionOnLegacyIndex() { - RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_0_0); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), CLUSTER_RECOVERED, Version.V_2_0_0); testAllocator.addData(node1, 10, null, randomBoolean()).addData(node2, 12, null, randomBoolean()); testAllocator.allocateUnassigned(allocation); assertThat(allocation.routingNodesChanged(), equalTo(true)); @@ -347,7 +355,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { * allocation mode would be chosen). */ public void testVersionBasedAllocationPrefersShardWithAllocationId() { - RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_0_0); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), CLUSTER_RECOVERED, Version.V_2_0_0); testAllocator.addData(node1, 10, null, randomBoolean()); testAllocator.addData(node2, ShardStateMetaData.NO_VERSION, "some allocId", randomBoolean()); testAllocator.addData(node3, 12, null, randomBoolean()); @@ -616,17 +624,27 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { assertClusterHealthStatus(allocation, ClusterHealthStatus.RED); } - private RoutingAllocation routingAllocationWithOnePrimaryNoReplicas(AllocationDeciders deciders, boolean asNew, Version version, + private RoutingAllocation routingAllocationWithOnePrimaryNoReplicas(AllocationDeciders deciders, + UnassignedInfo.Reason reason, Version version, String... activeAllocationIds) { MetaData metaData = MetaData.builder() .put(IndexMetaData.builder(shardId.getIndexName()).settings(settings(version)) .numberOfShards(1).numberOfReplicas(0).putInSyncAllocationIds(shardId.id(), Sets.newHashSet(activeAllocationIds))) .build(); RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); - if (asNew) { - routingTableBuilder.addAsNew(metaData.index(shardId.getIndex())); - } else { - routingTableBuilder.addAsRecovery(metaData.index(shardId.getIndex())); + switch (reason) { + + case INDEX_CREATED: + routingTableBuilder.addAsNew(metaData.index(shardId.getIndex())); + break; + case CLUSTER_RECOVERED: + routingTableBuilder.addAsRecovery(metaData.index(shardId.getIndex())); + break; + case INDEX_REOPENED: + routingTableBuilder.addAsFromCloseToOpen(metaData.index(shardId.getIndex())); + break; + default: + throw new IllegalArgumentException("can't do " + reason + " for you. teach me"); } ClusterState state = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) .metaData(metaData) diff --git a/core/src/test/java/org/elasticsearch/index/analysis/ASCIIFoldingTokenFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/ASCIIFoldingTokenFilterFactoryTests.java index 7b6d74a418c..d68cbaa9d30 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/ASCIIFoldingTokenFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/ASCIIFoldingTokenFilterFactoryTests.java @@ -23,6 +23,7 @@ import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.core.WhitespaceTokenizer; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import java.io.IOException; @@ -30,11 +31,11 @@ import java.io.StringReader; public class ASCIIFoldingTokenFilterFactoryTests extends ESTokenStreamTestCase { public void testDefault() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(Settings.builder() + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put("index.analysis.filter.my_ascii_folding.type", "asciifolding") .build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_ascii_folding"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_ascii_folding"); String source = "Ansprüche"; String[] expected = new String[]{"Anspruche"}; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -43,12 +44,12 @@ public class ASCIIFoldingTokenFilterFactoryTests extends ESTokenStreamTestCase { } public void testPreserveOriginal() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(Settings.builder() + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put("index.analysis.filter.my_ascii_folding.type", "asciifolding") .put("index.analysis.filter.my_ascii_folding.preserve_original", true) .build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_ascii_folding"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_ascii_folding"); String source = "Ansprüche"; String[] expected = new String[]{"Anspruche", "Ansprüche"}; Tokenizer tokenizer = new WhitespaceTokenizer(); diff --git a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisServiceTests.java b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisRegistryTests.java similarity index 68% rename from core/src/test/java/org/elasticsearch/index/analysis/AnalysisServiceTests.java rename to core/src/test/java/org/elasticsearch/index/analysis/AnalysisRegistryTests.java index 52fcdd4bb2e..4a5a0b95672 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisServiceTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisRegistryTests.java @@ -49,12 +49,25 @@ import static java.util.Collections.singletonMap; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; -public class AnalysisServiceTests extends ESTestCase { +public class AnalysisRegistryTests extends ESTestCase { + + private AnalysisRegistry registry; private static AnalyzerProvider analyzerProvider(final String name) { return new PreBuiltAnalyzerProvider(name, AnalyzerScope.INDEX, new EnglishAnalyzer()); } + @Override + public void setUp() throws Exception { + super.setUp(); + Settings settings = Settings + .builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .build(); + registry = new AnalysisRegistry(new Environment(settings), + emptyMap(), emptyMap(), emptyMap(), emptyMap()); + } + public void testDefaultAnalyzers() throws IOException { Version version = VersionUtils.randomVersion(random()); Settings settings = Settings @@ -63,29 +76,30 @@ public class AnalysisServiceTests extends ESTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", settings); - AnalysisService analysisService = new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap()) - .build(idxSettings); - assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); - assertThat(analysisService.defaultSearchAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); - assertThat(analysisService.defaultSearchQuoteAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); + IndexAnalyzers indexAnalyzers = new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap()) + .build(idxSettings); + assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultSearchAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultSearchQuoteAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); } public void testOverrideDefaultAnalyzer() throws IOException { Version version = VersionUtils.randomVersion(random()); Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build(); - AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings("index", settings), - singletonMap("default", analyzerProvider("default")), emptyMap(), emptyMap(), emptyMap()); - assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); - assertThat(analysisService.defaultSearchAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); - assertThat(analysisService.defaultSearchQuoteAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); + IndexAnalyzers indexAnalyzers = registry.build(IndexSettingsModule.newIndexSettings("index", settings), + singletonMap("default", analyzerProvider("default")) + , emptyMap(), emptyMap(), emptyMap()); + assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultSearchAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultSearchQuoteAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); } public void testOverrideDefaultIndexAnalyzerIsUnsupported() { Version version = VersionUtils.randomVersionBetween(random(), Version.V_5_0_0_alpha1, Version.CURRENT); Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build(); - AnalyzerProvider defaultIndex = new PreBuiltAnalyzerProvider("default_index", AnalyzerScope.INDEX, new EnglishAnalyzer()); + AnalyzerProvider defaultIndex = new PreBuiltAnalyzerProvider("default_index", AnalyzerScope.INDEX, new EnglishAnalyzer()); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> new AnalysisService(IndexSettingsModule.newIndexSettings("index", settings), + () -> registry.build(IndexSettingsModule.newIndexSettings("index", settings), singletonMap("default_index", defaultIndex), emptyMap(), emptyMap(), emptyMap())); assertTrue(e.getMessage().contains("[index.analysis.analyzer.default_index] is not supported")); } @@ -94,21 +108,21 @@ public class AnalysisServiceTests extends ESTestCase { Version version = VersionUtils.randomVersionBetween(random(), VersionUtils.getFirstVersion(), VersionUtils.getPreviousVersion(Version.V_5_0_0_alpha1)); Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build(); - AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings("index", settings), + IndexAnalyzers indexAnalyzers = registry.build(IndexSettingsModule.newIndexSettings("index", settings), singletonMap("default_index", analyzerProvider("default_index")), emptyMap(), emptyMap(), emptyMap()); - assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); - assertThat(analysisService.defaultSearchAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); - assertThat(analysisService.defaultSearchQuoteAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultSearchAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultSearchQuoteAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); } public void testOverrideDefaultSearchAnalyzer() { Version version = VersionUtils.randomVersion(random()); Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build(); - AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings("index", settings), + IndexAnalyzers indexAnalyzers = registry.build(IndexSettingsModule.newIndexSettings("index", settings), singletonMap("default_search", analyzerProvider("default_search")), emptyMap(), emptyMap(), emptyMap()); - assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); - assertThat(analysisService.defaultSearchAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); - assertThat(analysisService.defaultSearchQuoteAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultSearchAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultSearchQuoteAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); } public void testBackCompatOverrideDefaultIndexAndSearchAnalyzer() { @@ -118,11 +132,11 @@ public class AnalysisServiceTests extends ESTestCase { Map> analyzers = new HashMap<>(); analyzers.put("default_index", analyzerProvider("default_index")); analyzers.put("default_search", analyzerProvider("default_search")); - AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings("index", settings), + IndexAnalyzers indexAnalyzers = registry.build(IndexSettingsModule.newIndexSettings("index", settings), analyzers, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); - assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); - assertThat(analysisService.defaultSearchAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); - assertThat(analysisService.defaultSearchQuoteAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultSearchAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); + assertThat(indexAnalyzers.getDefaultSearchQuoteAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); } public void testConfigureCamelCaseTokenFilter() throws IOException { @@ -137,10 +151,10 @@ public class AnalysisServiceTests extends ESTestCase { .putArray("index.analysis.analyzer.custom_analyzer_1.filter", "lowercase", "word_delimiter").build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings); - - AnalysisService analysisService = new AnalysisModule(new Environment(settings), emptyList()).getAnalysisRegistry() + + IndexAnalyzers indexAnalyzers = new AnalysisModule(new Environment(settings), emptyList()).getAnalysisRegistry() .build(idxSettings); - try (NamedAnalyzer custom_analyser = analysisService.analyzer("custom_analyzer")) { + try (NamedAnalyzer custom_analyser = indexAnalyzers.get("custom_analyzer")) { assertNotNull(custom_analyser); TokenStream tokenStream = custom_analyser.tokenStream("foo", "J2SE j2ee"); tokenStream.reset(); @@ -154,7 +168,7 @@ public class AnalysisServiceTests extends ESTestCase { assertEquals("j2ee", token.get(1)); } - try (NamedAnalyzer custom_analyser = analysisService.analyzer("custom_analyzer_1")) { + try (NamedAnalyzer custom_analyser = indexAnalyzers.get("custom_analyzer_1")) { assertNotNull(custom_analyser); TokenStream tokenStream = custom_analyser.tokenStream("foo", "J2SE j2ee"); tokenStream.reset(); @@ -178,14 +192,14 @@ public class AnalysisServiceTests extends ESTestCase { Settings indexSettings = Settings.builder() .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings); - AnalysisService analysisService = new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap()) + IndexAnalyzers indexAnalyzers = new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap()) .build(idxSettings); - AnalysisService otherAnalysisSergice = new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), + IndexAnalyzers otherIndexAnalyzers = new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap()).build(idxSettings); final int numIters = randomIntBetween(5, 20); for (int i = 0; i < numIters; i++) { PreBuiltAnalyzers preBuiltAnalyzers = RandomPicks.randomFrom(random(), PreBuiltAnalyzers.values()); - assertSame(analysisService.analyzer(preBuiltAnalyzers.name()), otherAnalysisSergice.analyzer(preBuiltAnalyzers.name())); + assertSame(indexAnalyzers.get(preBuiltAnalyzers.name()), otherIndexAnalyzers.get(preBuiltAnalyzers.name())); } } @@ -204,4 +218,15 @@ public class AnalysisServiceTests extends ESTestCase { () -> new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap()).build(idxSettings)); assertThat(e.getMessage(), equalTo("analyzer [test_analyzer] must specify either an analyzer type, or a tokenizer")); } + + public void testCloseIndexAnalyzersMultipleTimes() throws IOException { + Settings settings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()).build(); + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); + IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings); + IndexAnalyzers indexAnalyzers = new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap()) + .build(idxSettings); + indexAnalyzers.close(); + indexAnalyzers.close(); + } } diff --git a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisTestsHelper.java b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisTestsHelper.java index 40ec2b412ff..a60c21c1a7e 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisTestsHelper.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisTestsHelper.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.indices.analysis.AnalysisModule; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import java.io.IOException; @@ -34,21 +35,25 @@ import static java.util.Collections.emptyList; public class AnalysisTestsHelper { - public static AnalysisService createAnalysisServiceFromClassPath(Path baseDir, String resource) throws IOException { + public static ESTestCase.TestAnalysis createTestAnalysisFromClassPath(Path baseDir, String resource) throws IOException { Settings settings = Settings.builder() .loadFromStream(resource, AnalysisTestsHelper.class.getResourceAsStream(resource)) .put(Environment.PATH_HOME_SETTING.getKey(), baseDir.toString()) .build(); - return createAnalysisServiceFromSettings(settings); + return createTestAnalysisFromSettings(settings); } - public static AnalysisService createAnalysisServiceFromSettings( + public static ESTestCase.TestAnalysis createTestAnalysisFromSettings( Settings settings) throws IOException { if (settings.get(IndexMetaData.SETTING_VERSION_CREATED) == null) { settings = Settings.builder().put(settings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); } - IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("test", settings); - return new AnalysisModule(new Environment(settings), emptyList()).getAnalysisRegistry().build(idxSettings); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", settings); + AnalysisRegistry analysisRegistry = new AnalysisModule(new Environment(settings), emptyList()).getAnalysisRegistry(); + return new ESTestCase.TestAnalysis(analysisRegistry.build(indexSettings), + analysisRegistry.buildTokenFilterFactories(indexSettings), + analysisRegistry.buildTokenizerFactories(indexSettings), + analysisRegistry.buildCharFilterFactories(indexSettings)); } } diff --git a/core/src/test/java/org/elasticsearch/index/analysis/CJKFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/CJKFilterFactoryTests.java index d2e2d4cc6e2..5ae2fbbb1cb 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/CJKFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/CJKFilterFactoryTests.java @@ -21,6 +21,7 @@ package org.elasticsearch.index.analysis; import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.standard.StandardTokenizer; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import java.io.IOException; @@ -30,8 +31,8 @@ public class CJKFilterFactoryTests extends ESTokenStreamTestCase { private static final String RESOURCE = "/org/elasticsearch/index/analysis/cjk_analysis.json"; public void testDefault() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("cjk_bigram"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("cjk_bigram"); String source = "多くの学生が試験に落ちた。"; String[] expected = new String[]{"多く", "くの", "の学", "学生", "生が", "が試", "試験", "験に", "に落", "落ち", "ちた" }; Tokenizer tokenizer = new StandardTokenizer(); @@ -40,8 +41,8 @@ public class CJKFilterFactoryTests extends ESTokenStreamTestCase { } public void testNoFlags() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("cjk_no_flags"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("cjk_no_flags"); String source = "多くの学生が試験に落ちた。"; String[] expected = new String[]{"多く", "くの", "の学", "学生", "生が", "が試", "試験", "験に", "に落", "落ち", "ちた" }; Tokenizer tokenizer = new StandardTokenizer(); @@ -50,8 +51,8 @@ public class CJKFilterFactoryTests extends ESTokenStreamTestCase { } public void testHanOnly() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("cjk_han_only"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("cjk_han_only"); String source = "多くの学生が試験に落ちた。"; String[] expected = new String[]{"多", "く", "の", "学生", "が", "試験", "に", "落", "ち", "た" }; Tokenizer tokenizer = new StandardTokenizer(); @@ -60,8 +61,8 @@ public class CJKFilterFactoryTests extends ESTokenStreamTestCase { } public void testHanUnigramOnly() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("cjk_han_unigram_only"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("cjk_han_unigram_only"); String source = "多くの学生が試験に落ちた。"; String[] expected = new String[]{"多", "く", "の", "学", "学生", "生", "が", "試", "試験", "験", "に", "落", "ち", "た" }; Tokenizer tokenizer = new StandardTokenizer(); diff --git a/core/src/test/java/org/elasticsearch/index/analysis/CharFilterTests.java b/core/src/test/java/org/elasticsearch/index/analysis/CharFilterTests.java index 3f2b1461ef3..206dffd0fb7 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/CharFilterTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/CharFilterTests.java @@ -26,7 +26,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.test.ESTokenStreamTestCase; import org.elasticsearch.test.IndexSettingsModule; -import static org.elasticsearch.test.ESTestCase.createAnalysisService; +import static org.elasticsearch.test.ESTestCase.createTestAnalysis; /** */ @@ -41,8 +41,8 @@ public class CharFilterTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("test", settings); - AnalysisService analysisService = createAnalysisService(idxSettings, settings); - NamedAnalyzer analyzer1 = analysisService.analyzer("custom_with_char_filter"); + IndexAnalyzers indexAnalyzers = createTestAnalysis(idxSettings, settings).indexAnalyzers; + NamedAnalyzer analyzer1 = indexAnalyzers.get("custom_with_char_filter"); assertTokenStreamContents(analyzer1.tokenStream("test", "jeff quit phish"), new String[]{"jeff", "qit", "fish"}); @@ -58,9 +58,9 @@ public class CharFilterTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("test", settings); - AnalysisService analysisService = createAnalysisService(idxSettings, settings); - NamedAnalyzer analyzer1 = analysisService.analyzer("custom_with_char_filter"); + IndexAnalyzers indexAnalyzers = createTestAnalysis(idxSettings, settings).indexAnalyzers; + NamedAnalyzer analyzer1 = indexAnalyzers.get("custom_with_char_filter"); assertTokenStreamContents(analyzer1.tokenStream("test", "hello!"), new String[]{"hello"}); @@ -80,8 +80,8 @@ public class CharFilterTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("test", settings); - AnalysisService analysisService = createAnalysisService(idxSettings, settings); - NamedAnalyzer analyzer1 = analysisService.analyzer("custom_with_char_filter"); + IndexAnalyzers indexAnalyzers = createTestAnalysis(idxSettings, settings).indexAnalyzers; + NamedAnalyzer analyzer1 = indexAnalyzers.get("custom_with_char_filter"); assertTokenStreamContents(analyzer1.tokenStream("test", "faBBbBB aBbbbBf"), new String[]{"foo", "oof"}); } diff --git a/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java b/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java index 0c9010b2c9b..ede42404558 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java @@ -61,9 +61,7 @@ public class CompoundAnalysisTests extends ESTestCase { return singletonMap("myfilter", MyFilterTokenFilterFactory::new); } })); - AnalysisService analysisService = analysisModule.getAnalysisRegistry().build(idxSettings); - - TokenFilterFactory filterFactory = analysisService.tokenFilter("dict_dec"); + TokenFilterFactory filterFactory = analysisModule.getAnalysisRegistry().buildTokenFilterFactories(idxSettings).get("dict_dec"); MatcherAssert.assertThat(filterFactory, instanceOf(DictionaryCompoundWordTokenFilterFactory.class)); } @@ -85,9 +83,8 @@ public class CompoundAnalysisTests extends ESTestCase { return singletonMap("myfilter", MyFilterTokenFilterFactory::new); } })); - AnalysisService analysisService = analysisModule.getAnalysisRegistry().build(idxSettings); - - Analyzer analyzer = analysisService.analyzer(analyzerName).analyzer(); + IndexAnalyzers indexAnalyzers = analysisModule.getAnalysisRegistry().build(idxSettings); + Analyzer analyzer = indexAnalyzers.get(analyzerName).analyzer(); AllEntries allEntries = new AllEntries(); allEntries.addText("field1", text, 1.0f); diff --git a/core/src/test/java/org/elasticsearch/index/analysis/HunspellTokenFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/HunspellTokenFilterFactoryTests.java index 45e4dd24f94..2708387da12 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/HunspellTokenFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/HunspellTokenFilterFactoryTests.java @@ -36,8 +36,8 @@ public class HunspellTokenFilterFactoryTests extends ESTestCase { .put("index.analysis.filter.en_US.locale", "en_US") .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("en_US"); + TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("en_US"); assertThat(tokenFilter, instanceOf(HunspellTokenFilterFactory.class)); HunspellTokenFilterFactory hunspellTokenFilter = (HunspellTokenFilterFactory) tokenFilter; assertThat(hunspellTokenFilter.dedup(), is(true)); @@ -50,8 +50,8 @@ public class HunspellTokenFilterFactoryTests extends ESTestCase { .put("index.analysis.filter.en_US.locale", "en_US") .build(); - analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - tokenFilter = analysisService.tokenFilter("en_US"); + analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + tokenFilter = analysis.tokenFilter.get("en_US"); assertThat(tokenFilter, instanceOf(HunspellTokenFilterFactory.class)); hunspellTokenFilter = (HunspellTokenFilterFactory) tokenFilter; assertThat(hunspellTokenFilter.dedup(), is(false)); diff --git a/core/src/test/java/org/elasticsearch/index/analysis/KeepFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/KeepFilterFactoryTests.java index 35148874e18..d5a6a590e78 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/KeepFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/KeepFilterFactoryTests.java @@ -23,6 +23,7 @@ import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.core.WhitespaceTokenizer; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import org.junit.Assert; @@ -35,8 +36,8 @@ public class KeepFilterFactoryTests extends ESTokenStreamTestCase { private static final String RESOURCE = "/org/elasticsearch/index/analysis/keep_analysis.json"; public void testLoadWithoutSettings() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("keep"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("keep"); Assert.assertNull(tokenFilter); } @@ -48,7 +49,7 @@ public class KeepFilterFactoryTests extends ESTokenStreamTestCase { .put("index.analysis.filter.broken_keep_filter.keep_words", "[\"Hello\", \"worlD\"]") .build(); try { - AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); + AnalysisTestsHelper.createTestAnalysisFromSettings(settings); Assert.fail("path and array are configured"); } catch (IllegalArgumentException e) { } catch (IOException e) { @@ -64,7 +65,7 @@ public class KeepFilterFactoryTests extends ESTokenStreamTestCase { .build(); try { // test our none existing setup is picked up - AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); + AnalysisTestsHelper.createTestAnalysisFromSettings(settings); fail("expected an exception due to non existent keep_words_path"); } catch (IllegalArgumentException e) { } catch (IOException e) { @@ -76,7 +77,7 @@ public class KeepFilterFactoryTests extends ESTokenStreamTestCase { .build(); try { // test our none existing setup is picked up - AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); + AnalysisTestsHelper.createTestAnalysisFromSettings(settings); fail("expected an exception indicating that you can't use [keep_words_path] with [keep_words] "); } catch (IllegalArgumentException e) { } catch (IOException e) { @@ -86,8 +87,8 @@ public class KeepFilterFactoryTests extends ESTokenStreamTestCase { } public void testCaseInsensitiveMapping() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_keep_filter"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_keep_filter"); assertThat(tokenFilter, instanceOf(KeepWordFilterFactory.class)); String source = "hello small world"; String[] expected = new String[]{"hello", "world"}; @@ -97,8 +98,8 @@ public class KeepFilterFactoryTests extends ESTokenStreamTestCase { } public void testCaseSensitiveMapping() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_case_sensitive_keep_filter"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_case_sensitive_keep_filter"); assertThat(tokenFilter, instanceOf(KeepWordFilterFactory.class)); String source = "Hello small world"; String[] expected = new String[]{"Hello"}; diff --git a/core/src/test/java/org/elasticsearch/index/analysis/KeepTypesFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/KeepTypesFilterFactoryTests.java index 986b79fad20..48ce1139d8f 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/KeepTypesFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/KeepTypesFilterFactoryTests.java @@ -23,6 +23,7 @@ import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.standard.StandardTokenizer; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import java.io.IOException; @@ -37,8 +38,8 @@ public class KeepTypesFilterFactoryTests extends ESTokenStreamTestCase { .put("index.analysis.filter.keep_numbers.type", "keep_types") .putArray("index.analysis.filter.keep_numbers.types", new String[] {"", ""}) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("keep_numbers"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("keep_numbers"); assertThat(tokenFilter, instanceOf(KeepTypesFilterFactory.class)); String source = "Hello 123 world"; String[] expected = new String[]{"123"}; diff --git a/core/src/test/java/org/elasticsearch/index/analysis/LimitTokenCountFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/LimitTokenCountFilterFactoryTests.java index bf17e5c7bf0..f1d810505bf 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/LimitTokenCountFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/LimitTokenCountFilterFactoryTests.java @@ -23,6 +23,7 @@ import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.core.WhitespaceTokenizer; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import java.io.IOException; @@ -34,9 +35,9 @@ public class LimitTokenCountFilterFactoryTests extends ESTokenStreamTestCase { .put("index.analysis.filter.limit_default.type", "limit") .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); { - TokenFilterFactory tokenFilter = analysisService.tokenFilter("limit_default"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("limit_default"); String source = "the quick brown fox"; String[] expected = new String[] { "the" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -44,7 +45,7 @@ public class LimitTokenCountFilterFactoryTests extends ESTokenStreamTestCase { assertTokenStreamContents(tokenFilter.create(tokenizer), expected); } { - TokenFilterFactory tokenFilter = analysisService.tokenFilter("limit"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("limit"); String source = "the quick brown fox"; String[] expected = new String[] { "the" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -61,8 +62,8 @@ public class LimitTokenCountFilterFactoryTests extends ESTokenStreamTestCase { .put("index.analysis.filter.limit_1.consume_all_tokens", true) .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("limit_1"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("limit_1"); String source = "the quick brown fox"; String[] expected = new String[] { "the", "quick", "brown" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -76,8 +77,8 @@ public class LimitTokenCountFilterFactoryTests extends ESTokenStreamTestCase { .put("index.analysis.filter.limit_1.consume_all_tokens", false) .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("limit_1"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("limit_1"); String source = "the quick brown fox"; String[] expected = new String[] { "the", "quick", "brown" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -92,8 +93,8 @@ public class LimitTokenCountFilterFactoryTests extends ESTokenStreamTestCase { .put("index.analysis.filter.limit_1.consume_all_tokens", true) .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("limit_1"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("limit_1"); String source = "the quick brown fox"; String[] expected = new String[] { "the", "quick", "brown", "fox" }; Tokenizer tokenizer = new WhitespaceTokenizer(); diff --git a/core/src/test/java/org/elasticsearch/index/analysis/MinHashFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/MinHashFilterFactoryTests.java index 60f01cac700..fc78afa7ab9 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/MinHashFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/MinHashFilterFactoryTests.java @@ -23,6 +23,7 @@ import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.core.WhitespaceTokenizer; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import java.io.IOException; @@ -36,8 +37,8 @@ public class MinHashFilterFactoryTests extends ESTokenStreamTestCase { Settings settings = Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("min_hash"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("min_hash"); String source = "the quick brown fox"; Tokenizer tokenizer = new WhitespaceTokenizer(); tokenizer.setReader(new StringReader(source)); @@ -57,8 +58,8 @@ public class MinHashFilterFactoryTests extends ESTokenStreamTestCase { .put("index.analysis.filter.test_min_hash.with_rotation", false) .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("test_min_hash"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("test_min_hash"); String source = "sushi"; Tokenizer tokenizer = new WhitespaceTokenizer(); tokenizer.setReader(new StringReader(source)); diff --git a/core/src/test/java/org/elasticsearch/index/analysis/PatternCaptureTokenFilterTests.java b/core/src/test/java/org/elasticsearch/index/analysis/PatternCaptureTokenFilterTests.java index caefb1039c2..126bbe2ab93 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/PatternCaptureTokenFilterTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/PatternCaptureTokenFilterTests.java @@ -27,7 +27,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.test.ESTokenStreamTestCase; import org.elasticsearch.test.IndexSettingsModule; -import static org.elasticsearch.test.ESTestCase.createAnalysisService; +import static org.elasticsearch.test.ESTestCase.createTestAnalysis; import static org.hamcrest.Matchers.containsString; public class PatternCaptureTokenFilterTests extends ESTokenStreamTestCase { @@ -40,17 +40,16 @@ public class PatternCaptureTokenFilterTests extends ESTokenStreamTestCase { .build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", settings); - AnalysisService analysisService = createAnalysisService(idxSettings, settings); - - NamedAnalyzer analyzer1 = analysisService.analyzer("single"); + IndexAnalyzers indexAnalyzers = createTestAnalysis(idxSettings, settings).indexAnalyzers; + NamedAnalyzer analyzer1 = indexAnalyzers.get("single"); assertTokenStreamContents(analyzer1.tokenStream("test", "foobarbaz"), new String[]{"foobarbaz","foobar","foo"}); - NamedAnalyzer analyzer2 = analysisService.analyzer("multi"); + NamedAnalyzer analyzer2 = indexAnalyzers.get("multi"); assertTokenStreamContents(analyzer2.tokenStream("test", "abc123def"), new String[]{"abc123def","abc","123","def"}); - NamedAnalyzer analyzer3 = analysisService.analyzer("preserve"); + NamedAnalyzer analyzer3 = indexAnalyzers.get("preserve"); assertTokenStreamContents(analyzer3.tokenStream("test", "foobarbaz"), new String[]{"foobar","foo"}); } diff --git a/core/src/test/java/org/elasticsearch/index/analysis/ShingleTokenFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/ShingleTokenFilterFactoryTests.java index f33ddc88cf4..64663c3682c 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/ShingleTokenFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/ShingleTokenFilterFactoryTests.java @@ -26,6 +26,7 @@ import org.apache.lucene.analysis.StopFilter; import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.core.WhitespaceTokenizer; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import java.io.IOException; @@ -38,8 +39,8 @@ public class ShingleTokenFilterFactoryTests extends ESTokenStreamTestCase { private static final String RESOURCE = "/org/elasticsearch/index/analysis/shingle_analysis.json"; public void testDefault() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("shingle"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("shingle"); String source = "the quick brown fox"; String[] expected = new String[]{"the", "the quick", "quick", "quick brown", "brown", "brown fox", "fox"}; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -48,8 +49,8 @@ public class ShingleTokenFilterFactoryTests extends ESTokenStreamTestCase { } public void testInverseMapping() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("shingle_inverse"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("shingle_inverse"); assertThat(tokenFilter, instanceOf(ShingleTokenFilterFactory.class)); String source = "the quick brown fox"; String[] expected = new String[]{"the_quick_brown", "quick_brown_fox"}; @@ -59,8 +60,8 @@ public class ShingleTokenFilterFactoryTests extends ESTokenStreamTestCase { } public void testInverseMappingNoShingles() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("shingle_inverse"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("shingle_inverse"); assertThat(tokenFilter, instanceOf(ShingleTokenFilterFactory.class)); String source = "the quick"; String[] expected = new String[]{"the", "quick"}; @@ -70,8 +71,8 @@ public class ShingleTokenFilterFactoryTests extends ESTokenStreamTestCase { } public void testFillerToken() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromClassPath(createTempDir(), RESOURCE); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("shingle_filler"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromClassPath(createTempDir(), RESOURCE); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("shingle_filler"); String source = "simon the sorcerer"; String[] expected = new String[]{"simon FILLER", "simon FILLER sorcerer", "FILLER sorcerer"}; Tokenizer tokenizer = new WhitespaceTokenizer(); diff --git a/core/src/test/java/org/elasticsearch/index/analysis/StemmerTokenFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/StemmerTokenFilterFactoryTests.java index a414f412767..7c4818c63b3 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/StemmerTokenFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/StemmerTokenFilterFactoryTests.java @@ -26,6 +26,7 @@ import org.apache.lucene.analysis.snowball.SnowballFilter; import org.elasticsearch.Version; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import org.elasticsearch.test.VersionUtils; @@ -53,13 +54,14 @@ public class StemmerTokenFilterFactoryTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_english"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_english"); assertThat(tokenFilter, instanceOf(StemmerTokenFilterFactory.class)); Tokenizer tokenizer = new WhitespaceTokenizer(); tokenizer.setReader(new StringReader("foo bar")); TokenStream create = tokenFilter.create(tokenizer); - NamedAnalyzer analyzer = analysisService.analyzer("my_english"); + IndexAnalyzers indexAnalyzers = analysis.indexAnalyzers; + NamedAnalyzer analyzer = indexAnalyzers.get("my_english"); assertThat(create, instanceOf(PorterStemFilter.class)); assertAnalyzesTo(analyzer, "consolingly", new String[]{"consolingli"}); } @@ -80,13 +82,14 @@ public class StemmerTokenFilterFactoryTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_porter2"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_porter2"); assertThat(tokenFilter, instanceOf(StemmerTokenFilterFactory.class)); Tokenizer tokenizer = new WhitespaceTokenizer(); tokenizer.setReader(new StringReader("foo bar")); TokenStream create = tokenFilter.create(tokenizer); - NamedAnalyzer analyzer = analysisService.analyzer("my_porter2"); + IndexAnalyzers indexAnalyzers = analysis.indexAnalyzers; + NamedAnalyzer analyzer = indexAnalyzers.get("my_porter2"); assertThat(create, instanceOf(SnowballFilter.class)); assertAnalyzesTo(analyzer, "possibly", new String[]{"possibl"}); } diff --git a/core/src/test/java/org/elasticsearch/index/analysis/StopAnalyzerTests.java b/core/src/test/java/org/elasticsearch/index/analysis/StopAnalyzerTests.java index 88c5fe692d6..e166f4b7b9e 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/StopAnalyzerTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/StopAnalyzerTests.java @@ -27,7 +27,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.test.ESTokenStreamTestCase; import org.elasticsearch.test.IndexSettingsModule; -import static org.elasticsearch.test.ESTestCase.createAnalysisService; +import static org.elasticsearch.test.ESTestCase.createTestAnalysis; public class StopAnalyzerTests extends ESTokenStreamTestCase { public void testDefaultsCompoundAnalysis() throws Exception { @@ -38,13 +38,12 @@ public class StopAnalyzerTests extends ESTokenStreamTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", settings); - AnalysisService analysisService = createAnalysisService(idxSettings, settings); - - NamedAnalyzer analyzer1 = analysisService.analyzer("analyzer1"); + IndexAnalyzers indexAnalyzers = createTestAnalysis(idxSettings, settings).indexAnalyzers; + NamedAnalyzer analyzer1 = indexAnalyzers.get("analyzer1"); assertTokenStreamContents(analyzer1.tokenStream("test", "to be or not to be"), new String[0]); - NamedAnalyzer analyzer2 = analysisService.analyzer("analyzer2"); + NamedAnalyzer analyzer2 = indexAnalyzers.get("analyzer2"); assertTokenStreamContents(analyzer2.tokenStream("test", "to be or not to be"), new String[0]); } diff --git a/core/src/test/java/org/elasticsearch/index/analysis/StopTokenFilterTests.java b/core/src/test/java/org/elasticsearch/index/analysis/StopTokenFilterTests.java index 07b00f30e20..7d91619075b 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/StopTokenFilterTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/StopTokenFilterTests.java @@ -28,6 +28,7 @@ import org.apache.lucene.util.Version; import org.elasticsearch.common.settings.Settings.Builder; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import java.io.IOException; @@ -47,7 +48,7 @@ public class StopTokenFilterTests extends ESTokenStreamTestCase { builder.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()); Settings settings = builder.build(); try { - AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); + AnalysisTestsHelper.createTestAnalysisFromSettings(settings); fail("Expected IllegalArgumentException"); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), containsString("enable_position_increments is not supported anymore")); @@ -62,8 +63,8 @@ public class StopTokenFilterTests extends ESTokenStreamTestCase { // don't specify } builder.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(builder.build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_stop"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(builder.build()); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_stop"); assertThat(tokenFilter, instanceOf(StopTokenFilterFactory.class)); Tokenizer tokenizer = new WhitespaceTokenizer(); tokenizer.setReader(new StringReader("foo bar")); @@ -77,8 +78,8 @@ public class StopTokenFilterTests extends ESTokenStreamTestCase { .put("index.analysis.filter.my_stop.remove_trailing", false) .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_stop"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_stop"); assertThat(tokenFilter, instanceOf(StopTokenFilterFactory.class)); Tokenizer tokenizer = new WhitespaceTokenizer(); tokenizer.setReader(new StringReader("foo an")); diff --git a/core/src/test/java/org/elasticsearch/index/analysis/WordDelimiterTokenFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/WordDelimiterTokenFilterFactoryTests.java index f94252caba4..1a7903bcfac 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/WordDelimiterTokenFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/WordDelimiterTokenFilterFactoryTests.java @@ -23,6 +23,7 @@ import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.core.WhitespaceTokenizer; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import java.io.IOException; @@ -30,11 +31,11 @@ import java.io.StringReader; public class WordDelimiterTokenFilterFactoryTests extends ESTokenStreamTestCase { public void testDefault() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(Settings.builder() + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put("index.analysis.filter.my_word_delimiter.type", "word_delimiter") .build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_word_delimiter"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_word_delimiter"); String source = "PowerShot 500-42 wi-fi wi-fi-4000 j2se O'Neil's"; String[] expected = new String[]{"Power", "Shot", "500", "42", "wi", "fi", "wi", "fi", "4000", "j", "2", "se", "O", "Neil"}; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -43,13 +44,13 @@ public class WordDelimiterTokenFilterFactoryTests extends ESTokenStreamTestCase } public void testCatenateWords() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(Settings.builder() + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put("index.analysis.filter.my_word_delimiter.type", "word_delimiter") .put("index.analysis.filter.my_word_delimiter.catenate_words", "true") .put("index.analysis.filter.my_word_delimiter.generate_word_parts", "false") .build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_word_delimiter"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_word_delimiter"); String source = "PowerShot 500-42 wi-fi wi-fi-4000 j2se O'Neil's"; String[] expected = new String[]{"PowerShot", "500", "42", "wifi", "wifi", "4000", "j", "2", "se", "ONeil"}; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -58,13 +59,13 @@ public class WordDelimiterTokenFilterFactoryTests extends ESTokenStreamTestCase } public void testCatenateNumbers() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(Settings.builder() + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put("index.analysis.filter.my_word_delimiter.type", "word_delimiter") .put("index.analysis.filter.my_word_delimiter.generate_number_parts", "false") .put("index.analysis.filter.my_word_delimiter.catenate_numbers", "true") .build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_word_delimiter"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_word_delimiter"); String source = "PowerShot 500-42 wi-fi wi-fi-4000 j2se O'Neil's"; String[] expected = new String[]{"Power", "Shot", "50042", "wi", "fi", "wi", "fi", "4000", "j", "2", "se", "O", "Neil"}; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -73,14 +74,14 @@ public class WordDelimiterTokenFilterFactoryTests extends ESTokenStreamTestCase } public void testCatenateAll() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(Settings.builder() + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put("index.analysis.filter.my_word_delimiter.type", "word_delimiter") .put("index.analysis.filter.my_word_delimiter.generate_word_parts", "false") .put("index.analysis.filter.my_word_delimiter.generate_number_parts", "false") .put("index.analysis.filter.my_word_delimiter.catenate_all", "true") .build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_word_delimiter"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_word_delimiter"); String source = "PowerShot 500-42 wi-fi wi-fi-4000 j2se O'Neil's"; String[] expected = new String[]{"PowerShot", "50042", "wifi", "wifi4000", "j2se", "ONeil"}; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -89,12 +90,12 @@ public class WordDelimiterTokenFilterFactoryTests extends ESTokenStreamTestCase } public void testSplitOnCaseChange() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(Settings.builder() + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put("index.analysis.filter.my_word_delimiter.type", "word_delimiter") .put("index.analysis.filter.my_word_delimiter.split_on_case_change", "false") .build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_word_delimiter"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_word_delimiter"); String source = "PowerShot"; String[] expected = new String[]{"PowerShot"}; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -103,12 +104,12 @@ public class WordDelimiterTokenFilterFactoryTests extends ESTokenStreamTestCase } public void testPreserveOriginal() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(Settings.builder() + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put("index.analysis.filter.my_word_delimiter.type", "word_delimiter") .put("index.analysis.filter.my_word_delimiter.preserve_original", "true") .build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_word_delimiter"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_word_delimiter"); String source = "PowerShot 500-42 wi-fi wi-fi-4000 j2se O'Neil's"; String[] expected = new String[]{"PowerShot", "Power", "Shot", "500-42", "500", "42", "wi-fi", "wi", "fi", "wi-fi-4000", "wi", "fi", "4000", "j2se", "j", "2", "se", "O'Neil's", "O", "Neil"}; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -117,12 +118,12 @@ public class WordDelimiterTokenFilterFactoryTests extends ESTokenStreamTestCase } public void testStemEnglishPossessive() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(Settings.builder() + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put("index.analysis.filter.my_word_delimiter.type", "word_delimiter") .put("index.analysis.filter.my_word_delimiter.stem_english_possessive", "false") .build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_word_delimiter"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_word_delimiter"); String source = "PowerShot 500-42 wi-fi wi-fi-4000 j2se O'Neil's"; String[] expected = new String[]{"Power", "Shot", "500", "42", "wi", "fi", "wi", "fi", "4000", "j", "2", "se", "O", "Neil", "s"}; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -132,13 +133,13 @@ public class WordDelimiterTokenFilterFactoryTests extends ESTokenStreamTestCase /** Correct offset order when doing both parts and concatenation: PowerShot is a synonym of Power */ public void testPartsAndCatenate() throws IOException { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(Settings.builder() + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put("index.analysis.filter.my_word_delimiter.type", "word_delimiter") .put("index.analysis.filter.my_word_delimiter.catenate_words", "true") .put("index.analysis.filter.my_word_delimiter.generate_word_parts", "true") .build()); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("my_word_delimiter"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_word_delimiter"); String source = "PowerShot"; String[] expected = new String[]{"Power", "PowerShot", "Shot" }; Tokenizer tokenizer = new WhitespaceTokenizer(); diff --git a/core/src/test/java/org/elasticsearch/index/analysis/commongrams/CommonGramsTokenFilterFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/commongrams/CommonGramsTokenFilterFactoryTests.java index 3c192052db5..ed6866e6a81 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/commongrams/CommonGramsTokenFilterFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/commongrams/CommonGramsTokenFilterFactoryTests.java @@ -24,9 +24,10 @@ import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.core.WhitespaceTokenizer; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; -import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.analysis.AnalysisTestsHelper; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.analysis.TokenFilterFactory; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTokenStreamTestCase; import org.junit.Assert; @@ -43,7 +44,7 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .build(); try { - AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); + AnalysisTestsHelper.createTestAnalysisFromSettings(settings); Assert.fail("[common_words] or [common_words_path] is set"); } catch (IllegalArgumentException e) { } catch (IOException e) { @@ -58,9 +59,9 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); { - TokenFilterFactory tokenFilter = analysisService.tokenFilter("common_grams_default"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("common_grams_default"); String source = "the quick brown is a fox Or noT"; String[] expected = new String[] { "the", "quick", "brown", "is", "a", "fox", "Or", "noT" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -75,9 +76,9 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .putArray("index.analysis.filter.common_grams_default.common_words", "chromosome", "protein") .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); { - TokenFilterFactory tokenFilter = analysisService.tokenFilter("common_grams_default"); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("common_grams_default"); String source = "the quick brown is a fox Or noT"; String[] expected = new String[] { "the", "quick", "brown", "is", "a", "fox", "Or", "noT" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -94,8 +95,8 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .putArray("index.analysis.filter.common_grams_1.common_words", "the", "Or", "Not", "a", "is", "an", "they", "are") .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("common_grams_1"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("common_grams_1"); String source = "the quick brown is a fox or noT"; String[] expected = new String[] { "the", "the_quick", "quick", "brown", "brown_is", "is", "is_a", "a", "a_fox", "fox", "fox_or", "or", "or_noT", "noT" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -108,8 +109,8 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .putArray("index.analysis.filter.common_grams_2.common_words", "the", "Or", "noT", "a", "is", "an", "they", "are") .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("common_grams_2"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("common_grams_2"); String source = "the quick brown is a fox or why noT"; String[] expected = new String[] { "the", "the_quick", "quick", "brown", "brown_is", "is", "is_a", "a", "a_fox", "fox", "or", "why", "why_noT", "noT" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -121,8 +122,8 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .putArray("index.analysis.filter.common_grams_3.common_words", "the", "or", "not", "a", "is", "an", "they", "are") .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("common_grams_3"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("common_grams_3"); String source = "the quick brown is a fox Or noT"; String[] expected = new String[] { "the", "the_quick", "quick", "brown", "brown_is", "is", "is_a", "a", "a_fox", "fox", "Or", "noT" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -138,15 +139,17 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createHome()) .build(); { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - Analyzer analyzer = analysisService.analyzer("commongramsAnalyzer").analyzer(); + IndexAnalyzers indexAnalyzers = AnalysisTestsHelper.createTestAnalysisFromSettings(settings) + .indexAnalyzers; + Analyzer analyzer = indexAnalyzers.get("commongramsAnalyzer").analyzer(); String source = "the quick brown is a fox or not"; String[] expected = new String[] { "the", "quick", "quick_brown", "brown", "brown_is", "is", "a", "a_fox", "fox", "fox_or", "or", "not" }; assertTokenStreamContents(analyzer.tokenStream("test", source), expected); } { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - Analyzer analyzer = analysisService.analyzer("commongramsAnalyzer_file").analyzer(); + IndexAnalyzers indexAnalyzers = AnalysisTestsHelper.createTestAnalysisFromSettings(settings) + .indexAnalyzers; + Analyzer analyzer = indexAnalyzers.get("commongramsAnalyzer_file").analyzer(); String source = "the quick brown is a fox or not"; String[] expected = new String[] { "the", "quick", "quick_brown", "brown", "brown_is", "is", "a", "a_fox", "fox", "fox_or", "or", "not" }; assertTokenStreamContents(analyzer.tokenStream("test", source), expected); @@ -161,8 +164,8 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .put("index.analysis.filter.common_grams_1.ignore_case", true) .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("common_grams_1"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("common_grams_1"); String source = "the quick brown is a fox or noT"; String[] expected = new String[] { "the_quick", "quick", "brown_is", "is_a", "a_fox", "fox_or", "or_noT" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -176,8 +179,8 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .put("index.analysis.filter.common_grams_2.ignore_case", false) .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("common_grams_2"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("common_grams_2"); String source = "the quick brown is a fox or why noT"; String[] expected = new String[] { "the_quick", "quick", "brown_is", "is_a", "a_fox", "fox", "or", "why_noT" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -190,8 +193,8 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .putArray("index.analysis.filter.common_grams_3.common_words", "the", "Or", "noT", "a", "is", "an", "they", "are") .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("common_grams_3"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("common_grams_3"); String source = "the quick brown is a fox or why noT"; String[] expected = new String[] { "the_quick", "quick", "brown_is", "is_a", "a_fox", "fox", "or", "why_noT" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -204,8 +207,8 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .putArray("index.analysis.filter.common_grams_4.common_words", "the", "or", "not", "a", "is", "an", "they", "are") .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("common_grams_4"); + ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("common_grams_4"); String source = "the quick brown is a fox Or noT"; String[] expected = new String[] { "the_quick", "quick", "brown_is", "is_a", "a_fox", "fox", "Or", "noT" }; Tokenizer tokenizer = new WhitespaceTokenizer(); @@ -221,15 +224,17 @@ public class CommonGramsTokenFilterFactoryTests extends ESTokenStreamTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createHome()) .build(); { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - Analyzer analyzer = analysisService.analyzer("commongramsAnalyzer").analyzer(); + IndexAnalyzers indexAnalyzers = AnalysisTestsHelper.createTestAnalysisFromSettings(settings) + .indexAnalyzers; + Analyzer analyzer = indexAnalyzers.get("commongramsAnalyzer").analyzer(); String source = "the quick brown is a fox or not"; String[] expected = new String[] { "the", "quick_brown", "brown_is", "is", "a_fox", "fox_or", "or", "not" }; assertTokenStreamContents(analyzer.tokenStream("test", source), expected); } { - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - Analyzer analyzer = analysisService.analyzer("commongramsAnalyzer_file").analyzer(); + IndexAnalyzers indexAnalyzers = AnalysisTestsHelper.createTestAnalysisFromSettings(settings) + .indexAnalyzers; + Analyzer analyzer = indexAnalyzers.get("commongramsAnalyzer_file").analyzer(); String source = "the quick brown is a fox or not"; String[] expected = new String[] { "the", "quick_brown", "brown_is", "is", "a_fox", "fox_or", "or", "not" }; assertTokenStreamContents(analyzer.tokenStream("test", source), expected); diff --git a/core/src/test/java/org/elasticsearch/index/analysis/synonyms/SynonymsAnalysisTests.java b/core/src/test/java/org/elasticsearch/index/analysis/synonyms/SynonymsAnalysisTests.java index de271d720c3..c6dfdc1a413 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/synonyms/SynonymsAnalysisTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/synonyms/SynonymsAnalysisTests.java @@ -30,7 +30,7 @@ import org.elasticsearch.common.lucene.all.AllTokenStream; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.hamcrest.MatcherAssert; @@ -46,7 +46,7 @@ import static org.hamcrest.Matchers.equalTo; */ public class SynonymsAnalysisTests extends ESTestCase { protected final Logger logger = Loggers.getLogger(getClass()); - private AnalysisService analysisService; + private IndexAnalyzers indexAnalyzers; public void testSynonymsAnalysis() throws IOException { InputStream synonyms = getClass().getResourceAsStream("synonyms.txt"); @@ -64,7 +64,7 @@ public class SynonymsAnalysisTests extends ESTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", settings); - analysisService = createAnalysisService(idxSettings, settings); + indexAnalyzers = createTestAnalysis(idxSettings, settings).indexAnalyzers; match("synonymAnalyzer", "kimchy is the dude abides", "shay is the elasticsearch man!"); match("synonymAnalyzer_file", "kimchy is the dude abides", "shay is the elasticsearch man!"); @@ -74,8 +74,7 @@ public class SynonymsAnalysisTests extends ESTestCase { } private void match(String analyzerName, String source, String target) throws IOException { - - Analyzer analyzer = analysisService.analyzer(analyzerName).analyzer(); + Analyzer analyzer = indexAnalyzers.get(analyzerName).analyzer(); TokenStream stream = AllTokenStream.allTokenStream("_all", source, 1.0f, analyzer); stream.reset(); diff --git a/core/src/test/java/org/elasticsearch/index/codec/CodecTests.java b/core/src/test/java/org/elasticsearch/index/codec/CodecTests.java index cf706f33cc2..ea89acbd8f8 100644 --- a/core/src/test/java/org/elasticsearch/index/codec/CodecTests.java +++ b/core/src/test/java/org/elasticsearch/index/codec/CodecTests.java @@ -37,7 +37,7 @@ import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.mapper.MapperRegistry; @@ -95,9 +95,9 @@ public class CodecTests extends ESTestCase { .build(); IndexSettings settings = IndexSettingsModule.newIndexSettings("_na", nodeSettings); SimilarityService similarityService = new SimilarityService(settings, Collections.emptyMap()); - AnalysisService analysisService = createAnalysisService(settings, nodeSettings); + IndexAnalyzers indexAnalyzers = createTestAnalysis(settings, nodeSettings).indexAnalyzers; MapperRegistry mapperRegistry = new MapperRegistry(Collections.emptyMap(), Collections.emptyMap()); - MapperService service = new MapperService(settings, analysisService, similarityService, mapperRegistry, () -> null); + MapperService service = new MapperService(settings, indexAnalyzers, similarityService, mapperRegistry, () -> null); return new CodecService(service, ESLoggerFactory.getLogger("test")); } diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index e92b620f9d4..2ea45f7a40c 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -26,6 +26,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.core.LogEvent; import org.apache.logging.log4j.core.appender.AbstractAppender; import org.apache.logging.log4j.core.filter.RegexFilter; +import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.codecs.Codec; import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; @@ -67,10 +68,13 @@ import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.env.Environment; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.AnalysisRegistry; +import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.engine.Engine.Searcher; import org.elasticsearch.index.mapper.ContentPath; @@ -2006,10 +2010,12 @@ public class InternalEngineTests extends ESTestCase { RootObjectMapper.Builder rootBuilder = new RootObjectMapper.Builder("test"); Index index = new Index(indexName, "_na_"); IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings); - AnalysisService analysisService = new AnalysisService(indexSettings, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); + IndexAnalyzers indexAnalyzers = null; + NamedAnalyzer defaultAnalyzer = new NamedAnalyzer("default", new StandardAnalyzer()); + indexAnalyzers = new IndexAnalyzers(indexSettings, defaultAnalyzer, defaultAnalyzer, defaultAnalyzer, Collections.emptyMap()); SimilarityService similarityService = new SimilarityService(indexSettings, Collections.emptyMap()); MapperRegistry mapperRegistry = new IndicesModule(Collections.emptyList()).getMapperRegistry(); - MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry, () -> null); + MapperService mapperService = new MapperService(indexSettings, indexAnalyzers, similarityService, mapperRegistry, () -> null); DocumentMapper.Builder b = new DocumentMapper.Builder(rootBuilder, mapperService); this.docMapper = b.build(mapperService); } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java index 7fa0a3c1161..6be11ced1e6 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java @@ -25,7 +25,6 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; @@ -63,7 +62,7 @@ public class ExternalFieldMapperTests extends ESSingleNodeTestCase { Collections.singletonMap(ExternalMetadataMapper.CONTENT_TYPE, new ExternalMetadataMapper.TypeParser())); DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(), - indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); DocumentMapper documentMapper = parser.parse("type", new CompressedXContent( XContentFactory.jsonBuilder().startObject().startObject("type") .startObject(ExternalMetadataMapper.CONTENT_TYPE) @@ -112,7 +111,7 @@ public class ExternalFieldMapperTests extends ESSingleNodeTestCase { MapperRegistry mapperRegistry = new MapperRegistry(mapperParsers, Collections.emptyMap()); DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(), - indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); DocumentMapper documentMapper = parser.parse("type", new CompressedXContent( XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") @@ -182,7 +181,7 @@ public class ExternalFieldMapperTests extends ESSingleNodeTestCase { MapperRegistry mapperRegistry = new MapperRegistry(mapperParsers, Collections.emptyMap()); DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(), - indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); DocumentMapper documentMapper = parser.parse("type", new CompressedXContent( XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") diff --git a/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java index baa9f728019..544764a9b53 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java @@ -24,24 +24,11 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.mapper.DocumentMapper; -import org.elasticsearch.index.mapper.DocumentMapperParser; -import org.elasticsearch.index.mapper.FieldNamesFieldMapper; -import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.Mapper; -import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.MetadataFieldMapper; -import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.mapper.TermBasedFieldType; import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.mapper.MapperRegistry; -import org.elasticsearch.plugins.MapperPlugin; import org.elasticsearch.test.ESSingleNodeTestCase; import java.io.IOException; @@ -244,9 +231,9 @@ public class FieldNamesFieldMapperTests extends ESSingleNodeTestCase { Collections.singletonMap("_dummy", new DummyMetadataFieldMapper.TypeParser()) ); final MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); - MapperService mapperService = new MapperService(indexService.getIndexSettings(), indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + MapperService mapperService = new MapperService(indexService.getIndexSettings(), indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), mapperService, - indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); String mapping = XContentFactory.jsonBuilder().startObject().startObject("type").endObject().endObject().string(); DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping)); ParsedDocument parsedDocument = mapper.parse("index", "type", "id", new BytesArray("{}")); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/ParentFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/ParentFieldMapperTests.java index 15738ada3bc..078c60ce19b 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/ParentFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/ParentFieldMapperTests.java @@ -18,30 +18,22 @@ */ package org.elasticsearch.index.mapper; +import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.IndexableField; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.AnalysisService; -import org.elasticsearch.index.mapper.ContentPath; -import org.elasticsearch.index.mapper.DocumentMapper; -import org.elasticsearch.index.mapper.Mapper; -import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.ParentFieldMapper; +import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.mapper.MapperService.MergeReason; -import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -111,10 +103,11 @@ public class ParentFieldMapperTests extends ESSingleNodeTestCase { public void testNoParentNullFieldCreatedIfNoParentSpecified() throws Exception { Index index = new Index("_index", "testUUID"); IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, Settings.EMPTY); - AnalysisService analysisService = new AnalysisService(indexSettings, Collections.emptyMap(), Collections.emptyMap(), - Collections.emptyMap(), Collections.emptyMap()); + NamedAnalyzer namedAnalyzer = new NamedAnalyzer("default", new StandardAnalyzer()); + IndexAnalyzers indexAnalyzers = new IndexAnalyzers(indexSettings, namedAnalyzer, namedAnalyzer, namedAnalyzer, + Collections.emptyMap()); SimilarityService similarityService = new SimilarityService(indexSettings, Collections.emptyMap()); - MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService, + MapperService mapperService = new MapperService(indexSettings, indexAnalyzers, similarityService, new IndicesModule(emptyList()).getMapperRegistry(), () -> null); XContentBuilder mappingSource = jsonBuilder().startObject().startObject("some_type") .startObject("properties") diff --git a/core/src/test/java/org/elasticsearch/indices/analysis/AnalysisModuleTests.java b/core/src/test/java/org/elasticsearch/indices/analysis/AnalysisModuleTests.java index 33763571daf..02cc8ff30a7 100644 --- a/core/src/test/java/org/elasticsearch/indices/analysis/AnalysisModuleTests.java +++ b/core/src/test/java/org/elasticsearch/indices/analysis/AnalysisModuleTests.java @@ -41,9 +41,9 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.Analysis; import org.elasticsearch.index.analysis.AnalysisRegistry; -import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.analysis.AnalysisTestsHelper; import org.elasticsearch.index.analysis.CustomAnalyzer; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.analysis.MappingCharFilterFactory; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.PatternReplaceCharFilterFactory; @@ -79,11 +79,11 @@ import static org.hamcrest.Matchers.is; */ public class AnalysisModuleTests extends ModuleTestCase { - public AnalysisService getAnalysisService(Settings settings) throws IOException { - return getAnalysisService(getNewRegistry(settings), settings); + public IndexAnalyzers getIndexAnalyzers(Settings settings) throws IOException { + return getIndexAnalyzers(getNewRegistry(settings), settings); } - public AnalysisService getAnalysisService(AnalysisRegistry registry, Settings settings) throws IOException { + public IndexAnalyzers getIndexAnalyzers(AnalysisRegistry registry, Settings settings) throws IOException { IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("test", settings); return registry.build(idxSettings); } @@ -136,9 +136,9 @@ public class AnalysisModuleTests extends ModuleTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, VersionUtils.randomVersionBetween(random(), Version.V_2_0_0, Version.V_2_3_5)) .build(); AnalysisRegistry newRegistry = getNewRegistry(settings); - AnalysisService as = getAnalysisService(newRegistry, settings); - assertThat(as.analyzer("default").analyzer(), is(instanceOf(KeywordAnalyzer.class))); - assertThat(as.analyzer("default_search").analyzer(), is(instanceOf(EnglishAnalyzer.class))); + IndexAnalyzers indexAnalyzers = getIndexAnalyzers(newRegistry, settings); + assertThat(indexAnalyzers.get("default").analyzer(), is(instanceOf(KeywordAnalyzer.class))); + assertThat(indexAnalyzers.get("default_search").analyzer(), is(instanceOf(EnglishAnalyzer.class))); } public void testAnalyzerAliasReferencesAlias() throws IOException { @@ -152,10 +152,11 @@ public class AnalysisModuleTests extends ModuleTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, VersionUtils.randomVersionBetween(random(), Version.V_2_0_0, Version.V_2_3_5)) .build(); AnalysisRegistry newRegistry = getNewRegistry(settings); - AnalysisService as = getAnalysisService(newRegistry, settings); - assertThat(as.analyzer("default").analyzer(), is(instanceOf(GermanAnalyzer.class))); + IndexAnalyzers indexAnalyzers = getIndexAnalyzers(newRegistry, settings); + + assertThat(indexAnalyzers.get("default").analyzer(), is(instanceOf(GermanAnalyzer.class))); // analyzer types are bound early before we resolve aliases - assertThat(as.analyzer("default_search").analyzer(), is(instanceOf(StandardAnalyzer.class))); + assertThat(indexAnalyzers.get("default_search").analyzer(), is(instanceOf(StandardAnalyzer.class))); } public void testAnalyzerAliasDefault() throws IOException { @@ -167,9 +168,9 @@ public class AnalysisModuleTests extends ModuleTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, VersionUtils.randomVersionBetween(random(), Version.V_2_0_0, Version.V_2_3_5)) .build(); AnalysisRegistry newRegistry = getNewRegistry(settings); - AnalysisService as = getAnalysisService(newRegistry, settings); - assertThat(as.analyzer("default").analyzer(), is(instanceOf(KeywordAnalyzer.class))); - assertThat(as.analyzer("default_search").analyzer(), is(instanceOf(KeywordAnalyzer.class))); + IndexAnalyzers indexAnalyzers = getIndexAnalyzers(newRegistry, settings); + assertThat(indexAnalyzers.get("default").analyzer(), is(instanceOf(KeywordAnalyzer.class))); + assertThat(indexAnalyzers.get("default_search").analyzer(), is(instanceOf(KeywordAnalyzer.class))); } public void testAnalyzerAliasMoreThanOnce() throws IOException { @@ -183,7 +184,7 @@ public class AnalysisModuleTests extends ModuleTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); AnalysisRegistry newRegistry = getNewRegistry(settings); - IllegalStateException ise = expectThrows(IllegalStateException.class, () -> getAnalysisService(newRegistry, settings)); + IllegalStateException ise = expectThrows(IllegalStateException.class, () -> getIndexAnalyzers(newRegistry, settings)); assertEquals("alias [default] is already used by [foobar]", ise.getMessage()); } @@ -196,7 +197,7 @@ public class AnalysisModuleTests extends ModuleTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); AnalysisRegistry registry = getNewRegistry(settings); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> getAnalysisService(registry, settings)); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> getIndexAnalyzers(registry, settings)); assertEquals("setting [index.analysis.analyzer.foobar.alias] is not supported", e.getMessage()); } @@ -208,7 +209,7 @@ public class AnalysisModuleTests extends ModuleTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_2_0_0) .build(); AnalysisRegistry newRegistry = getNewRegistry(settings2); - AnalysisService analysisService2 = getAnalysisService(newRegistry, settings2); + IndexAnalyzers indexAnalyzers = getIndexAnalyzers(newRegistry, settings2); // registry always has the current version assertThat(newRegistry.getAnalyzer("default"), is(instanceOf(NamedAnalyzer.class))); @@ -217,20 +218,20 @@ public class AnalysisModuleTests extends ModuleTestCase { assertEquals(Version.CURRENT.luceneVersion, defaultNamedAnalyzer.analyzer().getVersion()); // analysis service has the expected version - assertThat(analysisService2.analyzer("standard").analyzer(), is(instanceOf(StandardAnalyzer.class))); - assertEquals(Version.V_2_0_0.luceneVersion, analysisService2.analyzer("standard").analyzer().getVersion()); - assertEquals(Version.V_2_0_0.luceneVersion, analysisService2.analyzer("thai").analyzer().getVersion()); + assertThat(indexAnalyzers.get("standard").analyzer(), is(instanceOf(StandardAnalyzer.class))); + assertEquals(Version.V_2_0_0.luceneVersion, indexAnalyzers.get("standard").analyzer().getVersion()); + assertEquals(Version.V_2_0_0.luceneVersion, indexAnalyzers.get("thai").analyzer().getVersion()); - assertThat(analysisService2.analyzer("custom7").analyzer(), is(instanceOf(StandardAnalyzer.class))); - assertEquals(org.apache.lucene.util.Version.fromBits(3,6,0), analysisService2.analyzer("custom7").analyzer().getVersion()); + assertThat(indexAnalyzers.get("custom7").analyzer(), is(instanceOf(StandardAnalyzer.class))); + assertEquals(org.apache.lucene.util.Version.fromBits(3,6,0), indexAnalyzers.get("custom7").analyzer().getVersion()); } private void assertTokenFilter(String name, Class clazz) throws IOException { Settings settings = Settings.builder() .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()).build(); - AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings); - TokenFilterFactory tokenFilter = analysisService.tokenFilter(name); + TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get(name); Tokenizer tokenizer = new WhitespaceTokenizer(); tokenizer.setReader(new StringReader("foo bar")); TokenStream stream = tokenFilter.create(tokenizer); @@ -238,8 +239,8 @@ public class AnalysisModuleTests extends ModuleTestCase { } private void testSimpleConfiguration(Settings settings) throws IOException { - AnalysisService analysisService = getAnalysisService(settings); - Analyzer analyzer = analysisService.analyzer("custom1").analyzer(); + IndexAnalyzers indexAnalyzers = getIndexAnalyzers(settings); + Analyzer analyzer = indexAnalyzers.get("custom1").analyzer(); assertThat(analyzer, instanceOf(CustomAnalyzer.class)); CustomAnalyzer custom1 = (CustomAnalyzer) analyzer; @@ -249,23 +250,23 @@ public class AnalysisModuleTests extends ModuleTestCase { StopTokenFilterFactory stop1 = (StopTokenFilterFactory) custom1.tokenFilters()[0]; assertThat(stop1.stopWords().size(), equalTo(1)); - analyzer = analysisService.analyzer("custom2").analyzer(); + analyzer = indexAnalyzers.get("custom2").analyzer(); assertThat(analyzer, instanceOf(CustomAnalyzer.class)); // verify position increment gap - analyzer = analysisService.analyzer("custom6").analyzer(); + analyzer = indexAnalyzers.get("custom6").analyzer(); assertThat(analyzer, instanceOf(CustomAnalyzer.class)); CustomAnalyzer custom6 = (CustomAnalyzer) analyzer; assertThat(custom6.getPositionIncrementGap("any_string"), equalTo(256)); // verify characters mapping - analyzer = analysisService.analyzer("custom5").analyzer(); + analyzer = indexAnalyzers.get("custom5").analyzer(); assertThat(analyzer, instanceOf(CustomAnalyzer.class)); CustomAnalyzer custom5 = (CustomAnalyzer) analyzer; assertThat(custom5.charFilters()[0], instanceOf(MappingCharFilterFactory.class)); // check custom pattern replace filter - analyzer = analysisService.analyzer("custom3").analyzer(); + analyzer = indexAnalyzers.get("custom3").analyzer(); assertThat(analyzer, instanceOf(CustomAnalyzer.class)); CustomAnalyzer custom3 = (CustomAnalyzer) analyzer; PatternReplaceCharFilterFactory patternReplaceCharFilterFactory = (PatternReplaceCharFilterFactory) custom3.charFilters()[0]; @@ -273,7 +274,7 @@ public class AnalysisModuleTests extends ModuleTestCase { assertThat(patternReplaceCharFilterFactory.getReplacement(), equalTo("replacedSample $1")); // check custom class name (my) - analyzer = analysisService.analyzer("custom4").analyzer(); + analyzer = indexAnalyzers.get("custom4").analyzer(); assertThat(analyzer, instanceOf(CustomAnalyzer.class)); CustomAnalyzer custom4 = (CustomAnalyzer) analyzer; assertThat(custom4.tokenFilters()[0], instanceOf(MyFilterTokenFilterFactory.class)); @@ -333,7 +334,7 @@ public class AnalysisModuleTests extends ModuleTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, "1") .build(); try { - getAnalysisService(settings); + getIndexAnalyzers(settings); fail("This should fail with IllegalArgumentException because the analyzers name starts with _"); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), either(equalTo("analyzer name must not start with '_'. got \"_invalid_name\"")) @@ -350,7 +351,7 @@ public class AnalysisModuleTests extends ModuleTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, VersionUtils.randomVersionBetween(random(), Version.V_2_0_0, Version.V_2_3_5)) .build(); try { - getAnalysisService(settings); + getIndexAnalyzers(settings); fail("This should fail with IllegalArgumentException because the analyzers alias starts with _"); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), equalTo("analyzer name must not start with '_'. got \"_invalid_name\"")); @@ -365,7 +366,7 @@ public class AnalysisModuleTests extends ModuleTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .build(); try { - getAnalysisService(settings); + getIndexAnalyzers(settings); fail("Analyzer should fail if it has position_offset_gap"); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), equalTo("Option [position_offset_gap] in Custom Analyzer [custom] " + diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index 2ac204a218f..dcadd60cc77 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -53,7 +53,7 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService; import org.elasticsearch.cluster.metadata.MetaDataUpdateSettingsService; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; +import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.cluster.routing.allocation.RandomAllocationDeciderTests; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; @@ -70,7 +70,7 @@ import org.elasticsearch.index.NodeServicesProvider; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; @@ -116,7 +116,7 @@ public class ClusterStateChanges extends AbstractComponent { new HashSet<>(Arrays.asList(new SameShardAllocationDecider(settings), new ReplicaAfterPrimaryActiveAllocationDecider(settings), new RandomAllocationDeciderTests.RandomAllocationDecider(getRandom())))), - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(settings), + new TestGatewayAllocator(), new BalancedShardsAllocator(settings), EmptyClusterInfoService.INSTANCE); shardFailedClusterStateTaskExecutor = new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocationService, null, logger); shardStartedClusterStateTaskExecutor = new ShardStateAction.ShardStartedClusterStateTaskExecutor(allocationService, logger); @@ -210,10 +210,10 @@ public class ClusterStateChanges extends AbstractComponent { return allocationService.deassociateDeadNodes(clusterState, reroute, reason); } - public ClusterState applyFailedShards(ClusterState clusterState, List failedShards) { + public ClusterState applyFailedShards(ClusterState clusterState, List failedShards) { List entries = failedShards.stream().map(failedShard -> - new ShardStateAction.ShardEntry(failedShard.routingEntry.shardId(), failedShard.routingEntry.allocationId().getId(), - 0L, failedShard.message, failedShard.failure)) + new ShardStateAction.ShardEntry(failedShard.getRoutingEntry().shardId(), failedShard.getRoutingEntry().allocationId().getId(), + 0L, failedShard.getMessage(), failedShard.getFailure())) .collect(Collectors.toList()); try { return shardFailedClusterStateTaskExecutor.execute(clusterState, entries).resultingState; diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index a102f8dff28..a711ccbe02e 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -38,7 +38,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation.FailedShard; +import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; diff --git a/dev-tools/smoke_test_rc.py b/dev-tools/smoke_test_rc.py index 0a349851bd2..420ef35f36f 100644 --- a/dev-tools/smoke_test_rc.py +++ b/dev-tools/smoke_test_rc.py @@ -69,8 +69,10 @@ DEFAULT_PLUGINS = ["analysis-icu", "discovery-gce", "ingest-attachment", "ingest-geoip", + "ingest-user-agent", "lang-javascript", "lang-python", + "mapper-attachments", "mapper-murmur3", "mapper-size", "repository-azure", @@ -131,6 +133,13 @@ def download_and_verify(version, hash, files, base_url, plugins=DEFAULT_PLUGINS) try: downloaded_files = [] print(' ' + '*' * 80) + # here we create a temp gpg home where we download the release key as the only key into + # when we verify the signature it will fail if the signed key is not in the keystore and that + # way we keep the executing host unmodified since we don't have to import the key into the default keystore + gpg_home_dir = os.path.join(tmp_dir, "gpg_home_dir") + os.makedirs(gpg_home_dir, 0o700) + run('gpg --homedir %s --keyserver pool.sks-keyservers.net --recv-key D88E42B4' % gpg_home_dir) + for file in files: name = os.path.basename(file) print(' Smoketest file: %s' % name) @@ -139,7 +148,6 @@ def download_and_verify(version, hash, files, base_url, plugins=DEFAULT_PLUGINS) artifact_path = os.path.join(tmp_dir, file) downloaded_files.append(artifact_path) current_artifact_dir = os.path.dirname(artifact_path) - os.makedirs(current_artifact_dir) urllib.request.urlretrieve(url, os.path.join(tmp_dir, file)) sha1_url = ''.join([url, '.sha1']) checksum_file = artifact_path + ".sha1" @@ -155,12 +163,6 @@ def download_and_verify(version, hash, files, base_url, plugins=DEFAULT_PLUGINS) print(' Downloading %s' % (gpg_url)) urllib.request.urlretrieve(gpg_url, gpg_file) print(' Verifying gpg signature %s' % (gpg_file)) - # here we create a temp gpg home where we download the release key as the only key into - # when we verify the signature it will fail if the signed key is not in the keystore and that - # way we keep the executing host unmodified since we don't have to import the key into the default keystore - gpg_home_dir = os.path.join(current_artifact_dir, "gpg_home_dir") - os.makedirs(gpg_home_dir, 0o700) - run('gpg --homedir %s --keyserver pool.sks-keyservers.net --recv-key D88E42B4' % gpg_home_dir) run('cd %s && gpg --homedir %s --verify %s' % (current_artifact_dir, gpg_home_dir, os.path.basename(gpg_file))) print(' ' + '*' * 80) print() @@ -172,7 +174,7 @@ def download_and_verify(version, hash, files, base_url, plugins=DEFAULT_PLUGINS) def get_host_from_ports_file(es_dir): return read_fully(os.path.join(es_dir, 'logs/http.ports')).splitlines()[0] -def smoke_test_release(release, files, expected_hash, plugins): +def smoke_test_release(release, files, hash, plugins): for release_file in files: if not os.path.isfile(release_file): raise RuntimeError('Smoketest failed missing file %s' % (release_file)) @@ -191,7 +193,7 @@ def smoke_test_release(release, files, expected_hash, plugins): plugin_names = {} for plugin in plugins: print(' Install plugin [%s]' % (plugin)) - run('%s; export ES_JAVA_OPTS="-Des.plugins.staging=%s"; %s %s %s' % (java_exe(), expected_hash, es_plugin_path, 'install -b', plugin)) + run('%s; export ES_JAVA_OPTS="-Des.plugins.staging=%s"; %s %s %s' % (java_exe(), hash, es_plugin_path, 'install -b', plugin)) plugin_names[plugin] = True if 'x-pack' in plugin_names: headers = { 'Authorization' : 'Basic %s' % base64.b64encode(b"es_admin:foobar").decode("UTF-8") } @@ -222,8 +224,6 @@ def smoke_test_release(release, files, expected_hash, plugins): raise RuntimeError('Expected version [%s] but was [%s]' % (release, version['number'])) if version['build_snapshot']: raise RuntimeError('Expected non snapshot version') - if expected_hash != version['build_hash'].strip(): - raise RuntimeError('HEAD hash does not match expected [%s] but got [%s]' % (expected_hash, version['build_hash'])) print(' Verify if plugins are listed in _nodes') conn.request('GET', '/_nodes?plugin=true&pretty=true', headers=headers) res = conn.getresponse() @@ -262,7 +262,7 @@ if __name__ == "__main__": parser.add_argument('--version', '-v', dest='version', default=None, help='The Elasticsearch Version to smoke-tests', required=True) parser.add_argument('--hash', '-s', dest='hash', default=None, required=True, - help='The sha1 short hash of the git commit to smoketest') + help='The hash of the unified release') parser.add_argument('--plugins', '-p', dest='plugins', default=[], required=False, type=parse_list, help='A list of additional plugins to smoketest') parser.add_argument('--fetch_url', '-u', dest='url', default=None, @@ -277,16 +277,16 @@ if __name__ == "__main__": hash = args.hash url = args.url files = [ x % {'version': version} for x in [ - 'org/elasticsearch/distribution/tar/elasticsearch/%(version)s/elasticsearch-%(version)s.tar.gz', - 'org/elasticsearch/distribution/zip/elasticsearch/%(version)s/elasticsearch-%(version)s.zip', - 'org/elasticsearch/distribution/deb/elasticsearch/%(version)s/elasticsearch-%(version)s.deb', - 'org/elasticsearch/distribution/rpm/elasticsearch/%(version)s/elasticsearch-%(version)s.rpm' + 'elasticsearch-%(version)s.tar.gz', + 'elasticsearch-%(version)s.zip', + 'elasticsearch-%(version)s.deb', + 'elasticsearch-%(version)s.rpm' ]] verify_java_version('1.8') if url: download_url = url else: - download_url = '%s/%s-%s' % ('http://download.elasticsearch.org/elasticsearch/staging', version, hash) + download_url = 'https://staging.elastic.co/%s-%s/downloads/elasticsearch' % (version, hash) download_and_verify(version, hash, files, download_url, plugins=DEFAULT_PLUGINS + plugins) diff --git a/docs/reference/docs/delete-by-query.asciidoc b/docs/reference/docs/delete-by-query.asciidoc index d90d9084aa6..468f4545627 100644 --- a/docs/reference/docs/delete-by-query.asciidoc +++ b/docs/reference/docs/delete-by-query.asciidoc @@ -160,7 +160,7 @@ to keep or remove as you see fit. When you are done with it, delete it so Elasticsearch can reclaim the space it uses. `wait_for_active_shards` controls how many copies of a shard must be active -before proceeding with the request. See <> +before proceeding with the request. See <> for details. `timeout` controls how long each write request waits for unavailable shards to become available. Both work exactly how they work in the <>. @@ -339,3 +339,74 @@ like `1.7` or `12` to throttle to that level. Rethrottling that speeds up the query takes effect immediately but rethrotting that slows down the query will take effect on after completing the current batch. This prevents scroll timeouts. + +[float] +=== Manually slicing + +Delete-by-query supports <> allowing you to manually parallelize +the process relatively easily: + +[source,js] +---------------------------------------------------------------- +POST twitter/_delete_by_query +{ + "slice": { + "id": 0, + "max": 2 + }, + "query": { + "range": { + "likes": { + "lt": 10 + } + } + } +} +POST twitter/_delete_by_query +{ + "slice": { + "id": 1, + "max": 2 + }, + "query": { + "range": { + "likes": { + "lt": 10 + } + } + } +} +---------------------------------------------------------------- +// CONSOLE +// TEST[setup:big_twitter] + +Which you can verify works with: + +[source,js] +---------------------------------------------------------------- +GET _refresh +POST twitter/_search?size=0&filter_path=hits.total +{ + "query": { + "range": { + "likes": { + "lt": 10 + } + } + } +} +---------------------------------------------------------------- +// CONSOLE +// TEST[continued] + +Which results in a sensible `total` like this one: + +[source,js] +---------------------------------------------------------------- +{ + "hits": { + "total": 0 + } +} +---------------------------------------------------------------- +// TESTRESPONSE diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc index 673f0aa0234..46786c0b6df 100644 --- a/docs/reference/docs/reindex.asciidoc +++ b/docs/reference/docs/reindex.asciidoc @@ -694,6 +694,65 @@ and it'll look like: Or you can search by `tag` or whatever you want. +[float] +=== Manually slicing + +Reindex supports <> allowing you to manually parallelize the +process relatively easily: + +[source,js] +---------------------------------------------------------------- +POST _reindex +{ + "source": { + "index": "twitter", + "slice": { + "id": 0, + "max": 2 + } + }, + "dest": { + "index": "new_twitter" + } +} +POST _reindex +{ + "source": { + "index": "twitter", + "slice": { + "id": 1, + "max": 2 + } + }, + "dest": { + "index": "new_twitter" + } +} +---------------------------------------------------------------- +// CONSOLE +// TEST[setup:big_twitter] + +Which you can verify works with: + +[source,js] +---------------------------------------------------------------- +GET _refresh +POST new_twitter/_search?size=0&filter_path=hits.total +---------------------------------------------------------------- +// CONSOLE +// TEST[continued] + +Which results in a sensible `total` like this one: + +[source,js] +---------------------------------------------------------------- +{ + "hits": { + "total": 120 + } +} +---------------------------------------------------------------- +// TESTRESPONSE [float] === Reindex daily indices diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index 0151828fbed..7299d398e60 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -217,7 +217,7 @@ to keep or remove as you see fit. When you are done with it, delete it so Elasticsearch can reclaim the space it uses. `wait_for_active_shards` controls how many copies of a shard must be active -before proceeding with the request. See <> +before proceeding with the request. See <> for details. `timeout` controls how long each write request waits for unavailable shards to become available. Both work exactly how they work in the <>. @@ -405,6 +405,60 @@ query takes effect immediately but rethrotting that slows down the query will take effect on after completing the current batch. This prevents scroll timeouts. +[float] +=== Manually slicing + +Update-by-query supports <> allowing you to manually parallelize +the process relatively easily: + +[source,js] +---------------------------------------------------------------- +POST twitter/_update_by_query +{ + "slice": { + "id": 0, + "max": 2 + }, + "script": { + "inline": "ctx._source['extra'] = 'test'" + } +} +POST twitter/_update_by_query +{ + "slice": { + "id": 1, + "max": 2 + }, + "script": { + "inline": "ctx._source['extra'] = 'test'" + } +} +---------------------------------------------------------------- +// CONSOLE +// TEST[setup:big_twitter] + +Which you can verify works with: + +[source,js] +---------------------------------------------------------------- +GET _refresh +POST twitter/_search?size=0&q=extra:test&filter_path=hits.total +---------------------------------------------------------------- +// CONSOLE +// TEST[continued] + +Which results in a sensible `total` like this one: + +[source,js] +---------------------------------------------------------------- +{ + "hits": { + "total": 120 + } +} +---------------------------------------------------------------- +// TESTRESPONSE + [float] [[picking-up-a-new-property]] === Pick up a new property diff --git a/docs/reference/getting-started.asciidoc b/docs/reference/getting-started.asciidoc index e8ace29be22..b8b48b21bd9 100755 --- a/docs/reference/getting-started.asciidoc +++ b/docs/reference/getting-started.asciidoc @@ -171,7 +171,7 @@ If everything goes well, you should see a bunch of messages that look like below [2016-09-16T14:17:56,748][INFO ][o.e.n.Node ] [6-bjhwl] started -------------------------------------------------- -Without going too much into detail, we can see that our node named "I8hydUG" (which will be a different Marvel character in your case) has started and elected itself as a master in a single cluster. Don't worry yet at the moment what master means. The main thing that is important here is that we have started one node within one cluster. +Without going too much into detail, we can see that our node named "I8hydUG" (which will be a different set of characters in your case) has started and elected itself as a master in a single cluster. Don't worry yet at the moment what master means. The main thing that is important here is that we have started one node within one cluster. As mentioned previously, we can override either the cluster or node name. This can be done from the command line when starting Elasticsearch as follows: diff --git a/docs/reference/mapping.asciidoc b/docs/reference/mapping.asciidoc index 5507ba97ff6..dffb4f94ff5 100644 --- a/docs/reference/mapping.asciidoc +++ b/docs/reference/mapping.asciidoc @@ -64,6 +64,29 @@ the <>, the This is the purpose of _multi-fields_. Most datatypes support multi-fields via the <> parameter. +[[mapping-limit-settings]] +[float] +=== Settings to prevent mappings explosion + +The following settings allow you to limit the number of field mappings that +can be created manually or dynamically, in order to prevent bad documents from +causing a mapping explosion: + +`index.mapping.total_fields.limit`:: + The maximum number of fields in an index. The default value is `1000`. + +`index.mapping.depth.limit`:: + The maximum depth for a field, which is measured as the number of inner + objects. For instance, if all fields are defined at the root object level, + then the depth is `1`. If there is one object mapping, then the depth is + `2`, etc. The default is `20`. + +`index.mapping.nested_fields.limit`:: + The maximum number of `nested` fields in an index, defaults to `50`. + Indexing 1 document with 100 nested fields actually indexes 101 documents + as each nested document is indexed as a separate hidden document. + + [float] == Dynamic mapping diff --git a/docs/reference/mapping/dynamic/field-mapping.asciidoc b/docs/reference/mapping/dynamic/field-mapping.asciidoc index 020c5d24552..7bed12b5b78 100644 --- a/docs/reference/mapping/dynamic/field-mapping.asciidoc +++ b/docs/reference/mapping/dynamic/field-mapping.asciidoc @@ -30,20 +30,6 @@ detected. All other datatypes must be mapped explicitly. Besides the options listed below, dynamic field mapping rules can be further customised with <>. -[[mapping-limit-settings]] -==== Settings to prevent mappings explosion - -Two settings allow to control mapping explosion, in order to prevent adversary -documents to create huge mappings through dynamic mappings for instance: - -`index.mapping.total_fields.limit`:: - The maximum number of fields in an index. The default value is `1000`. -`index.mapping.depth.limit`:: - The maximum depth for a field, which is measured as the number of nested - objects. For instance, if all fields are defined at the root object level, - then the depth is `1`. If there is one object mapping, then the depth is - `2`, etc. The default is `20`. - [[date-detection]] ==== Date detection diff --git a/docs/reference/mapping/types/nested.asciidoc b/docs/reference/mapping/types/nested.asciidoc index 8850763cdc8..91bbf98c921 100644 --- a/docs/reference/mapping/types/nested.asciidoc +++ b/docs/reference/mapping/types/nested.asciidoc @@ -205,5 +205,5 @@ phase. Instead, highlighting needs to be performed via Indexing a document with 100 nested fields actually indexes 101 documents as each nested document is indexed as a separate document. To safeguard against ill-defined mappings -the number of nested fields that can be defined per index has been limited to 50. This -default limit can be changed with the index setting `index.mapping.nested_fields.limit`. +the number of nested fields that can be defined per index has been limited to 50. See +<>. diff --git a/docs/reference/mapping/types/percolator.asciidoc b/docs/reference/mapping/types/percolator.asciidoc index ca8c8386e9b..65f0d72550e 100644 --- a/docs/reference/mapping/types/percolator.asciidoc +++ b/docs/reference/mapping/types/percolator.asciidoc @@ -82,4 +82,7 @@ time (using `now`). There are a number of queries that fetch data via a get call during query parsing. For example the `terms` query when using terms lookup, `template` query when using indexed scripts and `geo_shape` when using pre-indexed shapes. When these queries are indexed by the `percolator` field type then the get call is executed once. So each time the `percolator` -query evaluates these queries, the fetches terms, shapes etc. as the were upon index time will be used. \ No newline at end of file +query evaluates these queries, the fetches terms, shapes etc. as the were upon index time will be used. Important to note +is that fetching of terms that these queries do, happens both each time the percolator query gets indexed on both primary +and replica shards, so the terms that are actually indexed can be different between shard copies, if the source index +changed while indexing. \ No newline at end of file diff --git a/docs/reference/search/request/scroll.asciidoc b/docs/reference/search/request/scroll.asciidoc index a082bf3ba4c..d924a56b652 100644 --- a/docs/reference/search/request/scroll.asciidoc +++ b/docs/reference/search/request/scroll.asciidoc @@ -175,7 +175,7 @@ curl -XDELETE localhost:9200/_search/scroll \ -d 'c2Nhbjs2OzM0NDg1ODpzRlBLc0FXNlNyNm5JWUc1,aGVuRmV0Y2g7NTsxOnkxaDZ' --------------------------------------- - +[[sliced-scroll]] ==== Sliced Scroll For scroll queries that return a lot of documents it is possible to split the scroll in multiple slices which @@ -183,7 +183,7 @@ can be consumed independently: [source,js] -------------------------------------------------- -curl -XGET 'localhost:9200/twitter/tweet/_search?scroll=1m' -d ' +GET /twitter/tweet/_search?scroll=1m { "slice": { "id": 0, <1> @@ -195,9 +195,7 @@ curl -XGET 'localhost:9200/twitter/tweet/_search?scroll=1m' -d ' } } } -' - -curl -XGET 'localhost:9200/twitter/tweet/_search?scroll=1m' -d ' +GET /twitter/tweet/_search?scroll=1m { "slice": { "id": 1, @@ -209,8 +207,9 @@ curl -XGET 'localhost:9200/twitter/tweet/_search?scroll=1m' -d ' } } } -' -------------------------------------------------- +// CONSOLE +// TEST[setup:big_twitter] <1> The id of the slice <2> The maximum number of slices @@ -247,10 +246,10 @@ slice gets deterministic results. [source,js] -------------------------------------------------- -curl -XGET 'localhost:9200/twitter/tweet/_search?scroll=1m' -d ' +GET /twitter/tweet/_search?scroll=1m { "slice": { - "field": "my_random_integer_field", + "field": "date", "id": 0, "max": 10 }, @@ -260,10 +259,11 @@ curl -XGET 'localhost:9200/twitter/tweet/_search?scroll=1m' -d ' } } } -' -------------------------------------------------- +// CONSOLE +// TEST[setup:big_twitter] For append only time-based indices, the `timestamp` field can be used safely. NOTE: By default the maximum number of slices allowed per scroll is limited to 1024. -You can update the `index.max_slices_per_scroll` index setting to bypass this limit. \ No newline at end of file +You can update the `index.max_slices_per_scroll` index setting to bypass this limit. diff --git a/docs/reference/setup/sysconfig/swap.asciidoc b/docs/reference/setup/sysconfig/swap.asciidoc index ce95e52fbeb..08752e6fd93 100644 --- a/docs/reference/setup/sysconfig/swap.asciidoc +++ b/docs/reference/setup/sysconfig/swap.asciidoc @@ -64,14 +64,7 @@ Systems using `systemd`:: Another possible reason why `mlockall` can fail is that the temporary directory (usually `/tmp`) is mounted with the `noexec` option. This can be solved by -specifying a new temp directory, by starting Elasticsearch with: - -[source,sh] --------------- -./bin/elasticsearch -Djava.io.tmpdir=/path/to/temp/dir --------------- - -or using the `ES_JAVA_OPTS` environment variable: +specifying a new temp directory using the `ES_JAVA_OPTS` environment variable: [source,sh] -------------- @@ -79,6 +72,8 @@ export ES_JAVA_OPTS="$ES_JAVA_OPTS -Djava.io.tmpdir=/path/to/temp/dir" ./bin/elasticsearch -------------- +or setting this JVM flag in the jvm.options configuration file. + [[disable-swap-files]] ==== Disable all swap files diff --git a/docs/resiliency/index.asciidoc b/docs/resiliency/index.asciidoc index bb2f384bc17..47ca68e00f5 100644 --- a/docs/resiliency/index.asciidoc +++ b/docs/resiliency/index.asciidoc @@ -112,7 +112,7 @@ exceptions, but it is still possible to cause a node to run out of heap space. The following issues have been identified: * Set a hard limit on `from`/`size` parameters {GIT}9311[#9311]. (STATUS: DONE, v2.1.0) -* Prevent combinatorial explosion in aggregations from causing OOM {GIT}8081[#8081]. (STATUS: ONGOING) +* Prevent combinatorial explosion in aggregations from causing OOM {GIT}8081[#8081]. (STATUS: DONE, v5.0.0) * Add the byte size of each hit to the request circuit breaker {GIT}9310[#9310]. (STATUS: ONGOING) * Limit the size of individual requests and also add a circuit breaker for the total memory used by in-flight request objects {GIT}16011[#16011]. (STATUS: DONE, v5.0.0) diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java index 09e8afbca0b..74ce3a5be1e 100644 --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java @@ -390,7 +390,7 @@ public class PercolateQueryBuilder extends AbstractQueryBuilder HIT_PARSER = new ConstructingObjectParser<>("hit", - a -> { + public static final ConstructingObjectParser HIT_PARSER = + new ConstructingObjectParser<>("hit", true, a -> { int i = 0; String index = (String) a[i++]; String type = (String) a[i++]; @@ -90,26 +87,23 @@ final class RemoteResponseParsers { HIT_PARSER.declareString(BasicHit::setParent, new ParseField("_parent")); HIT_PARSER.declareLong(BasicHit::setTTL, new ParseField("_ttl")); HIT_PARSER.declareLong(BasicHit::setTimestamp, new ParseField("_timestamp")); - HIT_PARSER.declareField((b, v) -> {}, p -> null, new ParseField("_score"), ValueType.FLOAT_OR_NULL); - HIT_PARSER.declareStringArray((b, v) -> {}, new ParseField("sort")); } /** * Parser for the {@code hits} element. Parsed to an array of {@code [total (Long), hits (List)]}. */ - public static final ConstructingObjectParser HITS_PARSER = new ConstructingObjectParser<>("hits", - a -> a); + public static final ConstructingObjectParser HITS_PARSER = + new ConstructingObjectParser<>("hits", true, a -> a); static { HITS_PARSER.declareLong(constructorArg(), new ParseField("total")); HITS_PARSER.declareObjectArray(constructorArg(), HIT_PARSER, new ParseField("hits")); - HITS_PARSER.declareField((b, v) -> {}, p -> null, new ParseField("max_score"), ValueType.FLOAT_OR_NULL); } /** * Parser for {@code failed} shards in the {@code _shards} elements. */ public static final ConstructingObjectParser SEARCH_FAILURE_PARSER = - new ConstructingObjectParser<>("failure", a -> { + new ConstructingObjectParser<>("failure", true, a -> { int i = 0; String index = (String) a[i++]; Integer shardId = (Integer) a[i++]; @@ -135,7 +129,6 @@ final class RemoteResponseParsers { return p.text(); } }, new ParseField("reason"), ValueType.OBJECT_OR_STRING); - SEARCH_FAILURE_PARSER.declareInt((b, v) -> {}, new ParseField("status")); } /** @@ -143,7 +136,7 @@ final class RemoteResponseParsers { * parses to an empty list. */ public static final ConstructingObjectParser, ParseFieldMatcherSupplier> SHARDS_PARSER = - new ConstructingObjectParser<>("_shards", a -> { + new ConstructingObjectParser<>("_shards", true, a -> { @SuppressWarnings("unchecked") List failures = (List) a[0]; failures = failures == null ? emptyList() : failures; @@ -151,13 +144,10 @@ final class RemoteResponseParsers { }); static { SHARDS_PARSER.declareObjectArray(optionalConstructorArg(), SEARCH_FAILURE_PARSER, new ParseField("failures")); - SHARDS_PARSER.declareInt((b, v) -> {}, new ParseField("total")); - SHARDS_PARSER.declareInt((b, v) -> {}, new ParseField("successful")); - SHARDS_PARSER.declareInt((b, v) -> {}, new ParseField("failed")); } public static final ConstructingObjectParser RESPONSE_PARSER = - new ConstructingObjectParser<>("search_response", a -> { + new ConstructingObjectParser<>("search_response", true, a -> { int i = 0; Throwable catastrophicFailure = (Throwable) a[i++]; if (catastrophicFailure != null) { @@ -189,9 +179,6 @@ final class RemoteResponseParsers { RESPONSE_PARSER.declareString(optionalConstructorArg(), new ParseField("_scroll_id")); RESPONSE_PARSER.declareObject(optionalConstructorArg(), HITS_PARSER, new ParseField("hits")); RESPONSE_PARSER.declareObject(optionalConstructorArg(), SHARDS_PARSER, new ParseField("_shards")); - RESPONSE_PARSER.declareInt((b, v) -> {}, new ParseField("took")); - RESPONSE_PARSER.declareBoolean((b, v) -> {}, new ParseField("terminated_early")); - RESPONSE_PARSER.declareInt((b, v) -> {}, new ParseField("status")); } /** @@ -200,7 +187,7 @@ final class RemoteResponseParsers { public static class ThrowableBuilder { public static final BiFunction PARSER; static { - ObjectParser parser = new ObjectParser<>("reason", ThrowableBuilder::new); + ObjectParser parser = new ObjectParser<>("reason", true, ThrowableBuilder::new); PARSER = parser.andThen(ThrowableBuilder::build); parser.declareString(ThrowableBuilder::setType, new ParseField("type")); parser.declareString(ThrowableBuilder::setReason, new ParseField("reason")); @@ -209,14 +196,6 @@ final class RemoteResponseParsers { // So we can give a nice error for parsing exceptions parser.declareInt(ThrowableBuilder::setLine, new ParseField("line")); parser.declareInt(ThrowableBuilder::setColumn, new ParseField("col")); - - // So we don't blow up on search exceptions - parser.declareString((b, v) -> {}, new ParseField("phase")); - parser.declareBoolean((b, v) -> {}, new ParseField("grouped")); - parser.declareField((p, v, c) -> p.skipChildren(), new ParseField("failed_shards"), ValueType.OBJECT_ARRAY); - - // Just throw away the root_cause - parser.declareField((p, v, c) -> p.skipChildren(), new ParseField("root_cause"), ValueType.OBJECT_ARRAY); } private String type; @@ -269,34 +248,15 @@ final class RemoteResponseParsers { } } - /** - * Parses the {@code version} field of the main action. There are a surprising number of fields in this that we don't need! - */ - public static final ConstructingObjectParser VERSION_PARSER = new ConstructingObjectParser<>( - "version", a -> Version.fromString((String) a[0])); - static { - VERSION_PARSER.declareString(constructorArg(), new ParseField("number")); - VERSION_PARSER.declareBoolean((p, v) -> {}, new ParseField("snapshot_build")); - VERSION_PARSER.declareBoolean((p, v) -> {}, new ParseField("build_snapshot")); - VERSION_PARSER.declareString((p, v) -> {}, new ParseField("build_hash")); - VERSION_PARSER.declareString((p, v) -> {}, new ParseField("build_date")); - VERSION_PARSER.declareString((p, v) -> {}, new ParseField("build_timestamp")); - VERSION_PARSER.declareString((p, v) -> {}, new ParseField("lucene_version")); - } - /** * Parses the main action to return just the {@linkplain Version} that it returns. We throw everything else out. */ public static final ConstructingObjectParser MAIN_ACTION_PARSER = new ConstructingObjectParser<>( - "/", a -> (Version) a[0]); + "/", true, a -> (Version) a[0]); static { - MAIN_ACTION_PARSER.declareBoolean((p, v) -> {}, new ParseField("ok")); - MAIN_ACTION_PARSER.declareInt((p, v) -> {}, new ParseField("status")); - MAIN_ACTION_PARSER.declareString((p, v) -> {}, new ParseField("name")); - MAIN_ACTION_PARSER.declareString((p, v) -> {}, new ParseField("cluster_name")); - MAIN_ACTION_PARSER.declareString((p, v) -> {}, new ParseField("cluster_uuid")); - MAIN_ACTION_PARSER.declareString((p, v) -> {}, new ParseField("name")); - MAIN_ACTION_PARSER.declareString((p, v) -> {}, new ParseField("tagline")); - MAIN_ACTION_PARSER.declareObject(constructorArg(), VERSION_PARSER, new ParseField("version")); + ConstructingObjectParser versionParser = new ConstructingObjectParser<>( + "version", true, a -> Version.fromString((String) a[0])); + versionParser.declareString(constructorArg(), new ParseField("number")); + MAIN_ACTION_PARSER.declareObject(constructorArg(), versionParser, new ParseField("version")); } } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java index 6407bc0195b..351eb49f906 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java @@ -113,11 +113,42 @@ public class RemoteScrollableHitSourceTests extends ESTestCase { } public void testLookupRemoteVersion() throws Exception { - sourceWithMockedRemoteCall(false, "main/0_20_5.json").lookupRemoteVersion(v -> assertEquals(Version.fromString("0.20.5"), v)); - sourceWithMockedRemoteCall(false, "main/0_90_13.json").lookupRemoteVersion(v -> assertEquals(Version.fromString("0.90.13"), v)); - sourceWithMockedRemoteCall(false, "main/1_7_5.json").lookupRemoteVersion(v -> assertEquals(Version.fromString("1.7.5"), v)); - sourceWithMockedRemoteCall(false, "main/2_3_3.json").lookupRemoteVersion(v -> assertEquals(Version.V_2_3_3, v)); - sourceWithMockedRemoteCall(false, "main/5_0_0_alpha_3.json").lookupRemoteVersion(v -> assertEquals(Version.V_5_0_0_alpha3, v)); + AtomicBoolean called = new AtomicBoolean(); + sourceWithMockedRemoteCall(false, "main/0_20_5.json").lookupRemoteVersion(v -> { + assertEquals(Version.fromString("0.20.5"), v); + called.set(true); + }); + assertTrue(called.get()); + called.set(false); + sourceWithMockedRemoteCall(false, "main/0_90_13.json").lookupRemoteVersion(v -> { + assertEquals(Version.fromString("0.90.13"), v); + called.set(true); + }); + assertTrue(called.get()); + called.set(false); + sourceWithMockedRemoteCall(false, "main/1_7_5.json").lookupRemoteVersion(v -> { + assertEquals(Version.fromString("1.7.5"), v); + called.set(true); + }); + assertTrue(called.get()); + called.set(false); + sourceWithMockedRemoteCall(false, "main/2_3_3.json").lookupRemoteVersion(v -> { + assertEquals(Version.V_2_3_3, v); + called.set(true); + }); + assertTrue(called.get()); + called.set(false); + sourceWithMockedRemoteCall(false, "main/5_0_0_alpha_3.json").lookupRemoteVersion(v -> { + assertEquals(Version.V_5_0_0_alpha3, v); + called.set(true); + }); + assertTrue(called.get()); + called.set(false); + sourceWithMockedRemoteCall(false, "main/with_unknown_fields.json").lookupRemoteVersion(v -> { + assertEquals(Version.V_5_0_0_alpha3, v); + called.set(true); + }); + assertTrue(called.get()); } public void testParseStartOk() throws Exception { diff --git a/modules/reindex/src/test/resources/responses/main/with_unknown_fields.json b/modules/reindex/src/test/resources/responses/main/with_unknown_fields.json new file mode 100644 index 00000000000..6aec2496314 --- /dev/null +++ b/modules/reindex/src/test/resources/responses/main/with_unknown_fields.json @@ -0,0 +1,22 @@ +{ + "name" : "Crazy Node With Weird Stuff In The Response", + "cluster_name" : "distribution_run", + "cats": "knock things over", + "cake": "is tasty", + "version" : { + "number" : "5.0.0-alpha3", + "build_hash" : "42e092f", + "build_date" : "2016-05-26T16:55:45.405Z", + "build_snapshot" : true, + "lucene_version" : "6.0.0", + "blort_version" : "not even a valid version number, what are you going to do about it?" + }, + "tagline" : "You Know, for Search", + "extra_object" : { + "stuff": "stuff" + }, + "extra_array" : [ + "stuff", + "more stuff" + ] +} diff --git a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/IcuTokenizerFactoryTests.java b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/IcuTokenizerFactoryTests.java index 180c4268612..716d07385b8 100644 --- a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/IcuTokenizerFactoryTests.java +++ b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/IcuTokenizerFactoryTests.java @@ -42,9 +42,9 @@ import static org.apache.lucene.analysis.BaseTokenStreamTestCase.assertTokenStre public class IcuTokenizerFactoryTests extends ESTestCase { public void testSimpleIcuTokenizer() throws IOException { - AnalysisService analysisService = createAnalysisService(); + TestAnalysis analysis = createTestAnalysis(); - TokenizerFactory tokenizerFactory = analysisService.tokenizer("icu_tokenizer"); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("icu_tokenizer"); ICUTokenizer tokenizer = (ICUTokenizer) tokenizerFactory.create(); Reader reader = new StringReader("向日葵, one-two"); @@ -53,10 +53,10 @@ public class IcuTokenizerFactoryTests extends ESTestCase { } public void testIcuCustomizeRuleFile() throws IOException { - AnalysisService analysisService = createAnalysisService(); + TestAnalysis analysis = createTestAnalysis(); // test the tokenizer with single rule file - TokenizerFactory tokenizerFactory = analysisService.tokenizer("user_rule_tokenizer"); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("user_rule_tokenizer"); ICUTokenizer tokenizer = (ICUTokenizer) tokenizerFactory.create(); Reader reader = new StringReader ("One-two punch. Brang-, not brung-it. This one--not that one--is the right one, -ish."); @@ -68,10 +68,10 @@ public class IcuTokenizerFactoryTests extends ESTestCase { } public void testMultipleIcuCustomizeRuleFiles() throws IOException { - AnalysisService analysisService = createAnalysisService(); + TestAnalysis analysis = createTestAnalysis(); // test the tokenizer with two rule files - TokenizerFactory tokenizerFactory = analysisService.tokenizer("multi_rule_tokenizer"); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("multi_rule_tokenizer"); ICUTokenizer tokenizer = (ICUTokenizer) tokenizerFactory.create(); StringReader reader = new StringReader ("Some English. Немного русский. ข้อความภาษาไทยเล็ก ๆ น้อย ๆ More English."); @@ -84,7 +84,7 @@ public class IcuTokenizerFactoryTests extends ESTestCase { } - private static AnalysisService createAnalysisService() throws IOException { + private static TestAnalysis createTestAnalysis() throws IOException { InputStream keywords = IcuTokenizerFactoryTests.class.getResourceAsStream("KeywordTokenizer.rbbi"); InputStream latin = IcuTokenizerFactoryTests.class.getResourceAsStream("Latin-dont-break-on-hyphens.rbbi"); @@ -102,6 +102,6 @@ public class IcuTokenizerFactoryTests extends ESTestCase { .build(); Settings nodeSettings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), home).build(); - return createAnalysisService(new Index("test", "_na_"), nodeSettings, settings, new AnalysisICUPlugin()); + return createTestAnalysis(new Index("test", "_na_"), nodeSettings, settings, new AnalysisICUPlugin()); } } diff --git a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuAnalysisTests.java b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuAnalysisTests.java index 9255a250f16..86338e0670d 100644 --- a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuAnalysisTests.java +++ b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuAnalysisTests.java @@ -31,24 +31,24 @@ import static org.hamcrest.Matchers.instanceOf; */ public class SimpleIcuAnalysisTests extends ESTestCase { public void testDefaultsIcuAnalysis() throws IOException { - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), Settings.EMPTY, new AnalysisICUPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), Settings.EMPTY, new AnalysisICUPlugin()); - TokenizerFactory tokenizerFactory = analysisService.tokenizer("icu_tokenizer"); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("icu_tokenizer"); assertThat(tokenizerFactory, instanceOf(IcuTokenizerFactory.class)); - TokenFilterFactory filterFactory = analysisService.tokenFilter("icu_normalizer"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("icu_normalizer"); assertThat(filterFactory, instanceOf(IcuNormalizerTokenFilterFactory.class)); - filterFactory = analysisService.tokenFilter("icu_folding"); + filterFactory = analysis.tokenFilter.get("icu_folding"); assertThat(filterFactory, instanceOf(IcuFoldingTokenFilterFactory.class)); - filterFactory = analysisService.tokenFilter("icu_collation"); + filterFactory = analysis.tokenFilter.get("icu_collation"); assertThat(filterFactory, instanceOf(IcuCollationTokenFilterFactory.class)); - filterFactory = analysisService.tokenFilter("icu_transform"); + filterFactory = analysis.tokenFilter.get("icu_transform"); assertThat(filterFactory, instanceOf(IcuTransformTokenFilterFactory.class)); - CharFilterFactory charFilterFactory = analysisService.charFilter("icu_normalizer"); + CharFilterFactory charFilterFactory = analysis.charFilter.get("icu_normalizer"); assertThat(charFilterFactory, instanceOf(IcuNormalizerCharFilterFactory.class)); } } diff --git a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuCollationTokenFilterTests.java b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuCollationTokenFilterTests.java index 62e9c9db145..8f9a38dc8f1 100644 --- a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuCollationTokenFilterTests.java +++ b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuCollationTokenFilterTests.java @@ -50,9 +50,9 @@ public class SimpleIcuCollationTokenFilterTests extends ESTestCase { .put("index.analysis.filter.myCollator.language", "tr") .put("index.analysis.filter.myCollator.strength", "primary") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("myCollator"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("myCollator"); assertCollatesToSame(filterFactory, "I WİLL USE TURKİSH CASING", "ı will use turkish casıng"); } @@ -66,9 +66,9 @@ public class SimpleIcuCollationTokenFilterTests extends ESTestCase { .put("index.analysis.filter.myCollator.strength", "primary") .put("index.analysis.filter.myCollator.decomposition", "canonical") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("myCollator"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("myCollator"); assertCollatesToSame(filterFactory, "I W\u0049\u0307LL USE TURKİSH CASING", "ı will use turkish casıng"); } @@ -82,9 +82,9 @@ public class SimpleIcuCollationTokenFilterTests extends ESTestCase { .put("index.analysis.filter.myCollator.strength", "secondary") .put("index.analysis.filter.myCollator.decomposition", "no") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("myCollator"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("myCollator"); assertCollatesToSame(filterFactory, "TESTING", "testing"); } @@ -99,9 +99,9 @@ public class SimpleIcuCollationTokenFilterTests extends ESTestCase { .put("index.analysis.filter.myCollator.strength", "primary") .put("index.analysis.filter.myCollator.alternate", "shifted") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("myCollator"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("myCollator"); assertCollatesToSame(filterFactory, "foo-bar", "foo bar"); } @@ -117,9 +117,9 @@ public class SimpleIcuCollationTokenFilterTests extends ESTestCase { .put("index.analysis.filter.myCollator.alternate", "shifted") .put("index.analysis.filter.myCollator.variableTop", " ") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("myCollator"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("myCollator"); assertCollatesToSame(filterFactory, "foo bar", "foobar"); // now assert that punctuation still matters: foo-bar < foo bar assertCollation(filterFactory, "foo-bar", "foo bar", -1); @@ -135,9 +135,9 @@ public class SimpleIcuCollationTokenFilterTests extends ESTestCase { .put("index.analysis.filter.myCollator.language", "en") .put("index.analysis.filter.myCollator.numeric", "true") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("myCollator"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("myCollator"); assertCollation(filterFactory, "foobar-9", "foobar-10", -1); } @@ -152,9 +152,9 @@ public class SimpleIcuCollationTokenFilterTests extends ESTestCase { .put("index.analysis.filter.myCollator.strength", "primary") .put("index.analysis.filter.myCollator.caseLevel", "true") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("myCollator"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("myCollator"); assertCollatesToSame(filterFactory, "résumé", "resume"); assertCollatesToSame(filterFactory, "Résumé", "Resume"); // now assert that case still matters: resume < Resume @@ -172,9 +172,9 @@ public class SimpleIcuCollationTokenFilterTests extends ESTestCase { .put("index.analysis.filter.myCollator.strength", "tertiary") .put("index.analysis.filter.myCollator.caseFirst", "upper") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("myCollator"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("myCollator"); assertCollation(filterFactory, "Resume", "resume", -1); } @@ -200,9 +200,9 @@ public class SimpleIcuCollationTokenFilterTests extends ESTestCase { .put("index.analysis.filter.myCollator.rules", tailoredRules) .put("index.analysis.filter.myCollator.strength", "primary") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("myCollator"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("myCollator"); assertCollatesToSame(filterFactory, "Töne", "Toene"); } diff --git a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuNormalizerCharFilterTests.java b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuNormalizerCharFilterTests.java index b82accf0cf8..32cbabfc9af 100644 --- a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuNormalizerCharFilterTests.java +++ b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/SimpleIcuNormalizerCharFilterTests.java @@ -37,8 +37,8 @@ public class SimpleIcuNormalizerCharFilterTests extends ESTestCase { Settings settings = Settings.builder() .put("index.analysis.char_filter.myNormalizerChar.type", "icu_normalizer") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - CharFilterFactory charFilterFactory = analysisService.charFilter("myNormalizerChar"); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + CharFilterFactory charFilterFactory = analysis.charFilter.get("myNormalizerChar"); String input = "ʰ㌰゙5℃№㈱㌘,バッファーの正規化のテスト.㋐㋑㋒㋓㋔カキクケコザジズゼゾg̈각/각நிเกषिchkʷक्षि"; Normalizer2 normalizer = Normalizer2.getInstance(null, "nfkc_cf", Normalizer2.Mode.COMPOSE); @@ -61,8 +61,8 @@ public class SimpleIcuNormalizerCharFilterTests extends ESTestCase { .put("index.analysis.char_filter.myNormalizerChar.name", "nfkc") .put("index.analysis.char_filter.myNormalizerChar.mode", "decompose") .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); - CharFilterFactory charFilterFactory = analysisService.charFilter("myNormalizerChar"); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisICUPlugin()); + CharFilterFactory charFilterFactory = analysis.charFilter.get("myNormalizerChar"); String input = "ʰ㌰゙5℃№㈱㌘,バッファーの正規化のテスト.㋐㋑㋒㋓㋔カキクケコザジズゼゾg̈각/각நிเกषिchkʷक्षि"; Normalizer2 normalizer = Normalizer2.getInstance(null, "nfkc", Normalizer2.Mode.DECOMPOSE); diff --git a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiAnalysisTests.java b/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiAnalysisTests.java index 53196ac7462..2da9416fbdc 100644 --- a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiAnalysisTests.java +++ b/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiAnalysisTests.java @@ -48,44 +48,45 @@ import static org.hamcrest.Matchers.notNullValue; */ public class KuromojiAnalysisTests extends ESTestCase { public void testDefaultsKuromojiAnalysis() throws IOException { - AnalysisService analysisService = createAnalysisService(); + TestAnalysis analysis = createTestAnalysis(); - TokenizerFactory tokenizerFactory = analysisService.tokenizer("kuromoji_tokenizer"); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("kuromoji_tokenizer"); assertThat(tokenizerFactory, instanceOf(KuromojiTokenizerFactory.class)); - TokenFilterFactory filterFactory = analysisService.tokenFilter("kuromoji_part_of_speech"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("kuromoji_part_of_speech"); assertThat(filterFactory, instanceOf(KuromojiPartOfSpeechFilterFactory.class)); - filterFactory = analysisService.tokenFilter("kuromoji_readingform"); + filterFactory = analysis.tokenFilter.get("kuromoji_readingform"); assertThat(filterFactory, instanceOf(KuromojiReadingFormFilterFactory.class)); - filterFactory = analysisService.tokenFilter("kuromoji_baseform"); + filterFactory = analysis.tokenFilter.get("kuromoji_baseform"); assertThat(filterFactory, instanceOf(KuromojiBaseFormFilterFactory.class)); - filterFactory = analysisService.tokenFilter("kuromoji_stemmer"); + filterFactory = analysis.tokenFilter.get("kuromoji_stemmer"); assertThat(filterFactory, instanceOf(KuromojiKatakanaStemmerFactory.class)); - filterFactory = analysisService.tokenFilter("ja_stop"); + filterFactory = analysis.tokenFilter.get("ja_stop"); assertThat(filterFactory, instanceOf(JapaneseStopTokenFilterFactory.class)); - filterFactory = analysisService.tokenFilter("kuromoji_number"); + filterFactory = analysis.tokenFilter.get("kuromoji_number"); assertThat(filterFactory, instanceOf(KuromojiNumberFilterFactory.class)); - NamedAnalyzer analyzer = analysisService.analyzer("kuromoji"); + IndexAnalyzers indexAnalyzers = analysis.indexAnalyzers; + NamedAnalyzer analyzer = indexAnalyzers.get("kuromoji"); assertThat(analyzer.analyzer(), instanceOf(JapaneseAnalyzer.class)); - analyzer = analysisService.analyzer("my_analyzer"); + analyzer = indexAnalyzers.get("my_analyzer"); assertThat(analyzer.analyzer(), instanceOf(CustomAnalyzer.class)); assertThat(analyzer.analyzer().tokenStream(null, new StringReader("")), instanceOf(JapaneseTokenizer.class)); - CharFilterFactory charFilterFactory = analysisService.charFilter("kuromoji_iteration_mark"); + CharFilterFactory charFilterFactory = analysis.charFilter.get("kuromoji_iteration_mark"); assertThat(charFilterFactory, instanceOf(KuromojiIterationMarkCharFilterFactory.class)); } public void testBaseFormFilterFactory() throws IOException { - AnalysisService analysisService = createAnalysisService(); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("kuromoji_pos"); + TestAnalysis analysis = createTestAnalysis(); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("kuromoji_pos"); assertThat(tokenFilter, instanceOf(KuromojiPartOfSpeechFilterFactory.class)); String source = "私は制限スピードを超える。"; String[] expected = new String[]{"私", "は", "制限", "スピード", "を"}; @@ -95,8 +96,8 @@ public class KuromojiAnalysisTests extends ESTestCase { } public void testReadingFormFilterFactory() throws IOException { - AnalysisService analysisService = createAnalysisService(); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("kuromoji_rf"); + TestAnalysis analysis = createTestAnalysis(); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("kuromoji_rf"); assertThat(tokenFilter, instanceOf(KuromojiReadingFormFilterFactory.class)); String source = "今夜はロバート先生と話した"; String[] expected_tokens_romaji = new String[]{"kon'ya", "ha", "robato", "sensei", "to", "hanashi", "ta"}; @@ -109,14 +110,14 @@ public class KuromojiAnalysisTests extends ESTestCase { tokenizer = new JapaneseTokenizer(null, true, JapaneseTokenizer.Mode.SEARCH); tokenizer.setReader(new StringReader(source)); String[] expected_tokens_katakana = new String[]{"コンヤ", "ハ", "ロバート", "センセイ", "ト", "ハナシ", "タ"}; - tokenFilter = analysisService.tokenFilter("kuromoji_readingform"); + tokenFilter = analysis.tokenFilter.get("kuromoji_readingform"); assertThat(tokenFilter, instanceOf(KuromojiReadingFormFilterFactory.class)); assertSimpleTSOutput(tokenFilter.create(tokenizer), expected_tokens_katakana); } public void testKatakanaStemFilter() throws IOException { - AnalysisService analysisService = createAnalysisService(); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("kuromoji_stemmer"); + TestAnalysis analysis = createTestAnalysis(); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("kuromoji_stemmer"); assertThat(tokenFilter, instanceOf(KuromojiKatakanaStemmerFactory.class)); String source = "明後日パーティーに行く予定がある。図書館で資料をコピーしました。"; @@ -128,7 +129,7 @@ public class KuromojiAnalysisTests extends ESTestCase { String[] expected_tokens_katakana = new String[]{"明後日", "パーティ", "に", "行く", "予定", "が", "ある", "図書館", "で", "資料", "を", "コピー", "し", "まし", "た"}; assertSimpleTSOutput(tokenFilter.create(tokenizer), expected_tokens_katakana); - tokenFilter = analysisService.tokenFilter("kuromoji_ks"); + tokenFilter = analysis.tokenFilter.get("kuromoji_ks"); assertThat(tokenFilter, instanceOf(KuromojiKatakanaStemmerFactory.class)); tokenizer = new JapaneseTokenizer(null, true, JapaneseTokenizer.Mode.SEARCH); tokenizer.setReader(new StringReader(source)); @@ -140,9 +141,9 @@ public class KuromojiAnalysisTests extends ESTestCase { } public void testIterationMarkCharFilter() throws IOException { - AnalysisService analysisService = createAnalysisService(); + TestAnalysis analysis = createTestAnalysis(); // test only kanji - CharFilterFactory charFilterFactory = analysisService.charFilter("kuromoji_im_only_kanji"); + CharFilterFactory charFilterFactory = analysis.charFilter.get("kuromoji_im_only_kanji"); assertNotNull(charFilterFactory); assertThat(charFilterFactory, instanceOf(KuromojiIterationMarkCharFilterFactory.class)); @@ -153,7 +154,7 @@ public class KuromojiAnalysisTests extends ESTestCase { // test only kana - charFilterFactory = analysisService.charFilter("kuromoji_im_only_kana"); + charFilterFactory = analysis.charFilter.get("kuromoji_im_only_kana"); assertNotNull(charFilterFactory); assertThat(charFilterFactory, instanceOf(KuromojiIterationMarkCharFilterFactory.class)); @@ -163,7 +164,7 @@ public class KuromojiAnalysisTests extends ESTestCase { // test default - charFilterFactory = analysisService.charFilter("kuromoji_im_default"); + charFilterFactory = analysis.charFilter.get("kuromoji_im_default"); assertNotNull(charFilterFactory); assertThat(charFilterFactory, instanceOf(KuromojiIterationMarkCharFilterFactory.class)); @@ -173,8 +174,8 @@ public class KuromojiAnalysisTests extends ESTestCase { } public void testJapaneseStopFilterFactory() throws IOException { - AnalysisService analysisService = createAnalysisService(); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("ja_stop"); + TestAnalysis analysis = createTestAnalysis(); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("ja_stop"); assertThat(tokenFilter, instanceOf(JapaneseStopTokenFilterFactory.class)); String source = "私は制限スピードを超える。"; String[] expected = new String[]{"私", "制限", "超える"}; @@ -183,7 +184,7 @@ public class KuromojiAnalysisTests extends ESTestCase { assertSimpleTSOutput(tokenFilter.create(tokenizer), expected); } - private static AnalysisService createAnalysisService() throws IOException { + private static TestAnalysis createTestAnalysis() throws IOException { InputStream empty_dict = KuromojiAnalysisTests.class.getResourceAsStream("empty_user_dict.txt"); InputStream dict = KuromojiAnalysisTests.class.getResourceAsStream("user_dict.txt"); Path home = createTempDir(); @@ -198,7 +199,7 @@ public class KuromojiAnalysisTests extends ESTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .build(); Settings nodeSettings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), home).build(); - return createAnalysisService(new Index("test", "_na_"), nodeSettings, settings, new AnalysisKuromojiPlugin()); + return createTestAnalysis(new Index("test", "_na_"), nodeSettings, settings, new AnalysisKuromojiPlugin()); } public static void assertSimpleTSOutput(TokenStream stream, @@ -230,8 +231,8 @@ public class KuromojiAnalysisTests extends ESTestCase { } public void testKuromojiUserDict() throws IOException { - AnalysisService analysisService = createAnalysisService(); - TokenizerFactory tokenizerFactory = analysisService.tokenizer("kuromoji_user_dict"); + TestAnalysis analysis = createTestAnalysis(); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("kuromoji_user_dict"); String source = "私は制限スピードを超える。"; String[] expected = new String[]{"私", "は", "制限スピード", "を", "超える"}; @@ -242,14 +243,14 @@ public class KuromojiAnalysisTests extends ESTestCase { // fix #59 public void testKuromojiEmptyUserDict() throws IOException { - AnalysisService analysisService = createAnalysisService(); - TokenizerFactory tokenizerFactory = analysisService.tokenizer("kuromoji_empty_user_dict"); + TestAnalysis analysis = createTestAnalysis(); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("kuromoji_empty_user_dict"); assertThat(tokenizerFactory, instanceOf(KuromojiTokenizerFactory.class)); } public void testNbestCost() throws IOException { - AnalysisService analysisService = createAnalysisService(); - TokenizerFactory tokenizerFactory = analysisService.tokenizer("kuromoji_nbest_cost"); + TestAnalysis analysis = createTestAnalysis(); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("kuromoji_nbest_cost"); String source = "鳩山積み"; String[] expected = new String[] {"鳩", "鳩山", "山積み", "積み"}; @@ -259,8 +260,8 @@ public class KuromojiAnalysisTests extends ESTestCase { } public void testNbestExample() throws IOException { - AnalysisService analysisService = createAnalysisService(); - TokenizerFactory tokenizerFactory = analysisService.tokenizer("kuromoji_nbest_examples"); + TestAnalysis analysis = createTestAnalysis(); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("kuromoji_nbest_examples"); String source = "鳩山積み"; String[] expected = new String[] {"鳩", "鳩山", "山積み", "積み"}; @@ -270,8 +271,8 @@ public class KuromojiAnalysisTests extends ESTestCase { } public void testNbestBothOptions() throws IOException { - AnalysisService analysisService = createAnalysisService(); - TokenizerFactory tokenizerFactory = analysisService.tokenizer("kuromoji_nbest_both"); + TestAnalysis analysis = createTestAnalysis(); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("kuromoji_nbest_both"); String source = "鳩山積み"; String[] expected = new String[] {"鳩", "鳩山", "山積み", "積み"}; @@ -282,8 +283,8 @@ public class KuromojiAnalysisTests extends ESTestCase { } public void testNumberFilterFactory() throws Exception { - AnalysisService analysisService = createAnalysisService(); - TokenFilterFactory tokenFilter = analysisService.tokenFilter("kuromoji_number"); + TestAnalysis analysis = createTestAnalysis(); + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("kuromoji_number"); assertThat(tokenFilter, instanceOf(KuromojiNumberFilterFactory.class)); String source = "本日十万二千五百円のワインを買った"; String[] expected = new String[]{"本日", "102500", "円", "の", "ワイン", "を", "買っ", "た"}; diff --git a/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticAnalysisTests.java b/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticAnalysisTests.java index 3dcfadce781..b0c23e29abd 100644 --- a/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticAnalysisTests.java +++ b/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticAnalysisTests.java @@ -39,8 +39,8 @@ public class SimplePhoneticAnalysisTests extends ESTestCase { Settings settings = Settings.builder().loadFromStream(yaml, getClass().getResourceAsStream(yaml)) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .build(); - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), settings, new AnalysisPhoneticPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("phonetic"); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), settings, new AnalysisPhoneticPlugin()); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("phonetic"); MatcherAssert.assertThat(filterFactory, instanceOf(PhoneticTokenFilterFactory.class)); } } diff --git a/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/SimpleSmartChineseAnalysisTests.java b/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/SimpleSmartChineseAnalysisTests.java index 08aebdee2bb..e2d6f6db513 100644 --- a/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/SimpleSmartChineseAnalysisTests.java +++ b/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/SimpleSmartChineseAnalysisTests.java @@ -31,9 +31,9 @@ import static org.hamcrest.Matchers.instanceOf; public class SimpleSmartChineseAnalysisTests extends ESTestCase { public void testDefaultsIcuAnalysis() throws IOException { - final AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), Settings.EMPTY, + final TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), Settings.EMPTY, new AnalysisSmartChinesePlugin()); - TokenizerFactory tokenizerFactory = analysisService.tokenizer("smartcn_tokenizer"); + TokenizerFactory tokenizerFactory = analysis.tokenizer.get("smartcn_tokenizer"); MatcherAssert.assertThat(tokenizerFactory, instanceOf(SmartChineseTokenizerTokenizerFactory.class)); } } diff --git a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/PolishAnalysisTests.java b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/PolishAnalysisTests.java index 4f7ee642ebd..d0b81f01d01 100644 --- a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/PolishAnalysisTests.java +++ b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/PolishAnalysisTests.java @@ -36,12 +36,12 @@ import static org.hamcrest.Matchers.instanceOf; */ public class PolishAnalysisTests extends ESTestCase { public void testDefaultsPolishAnalysis() throws IOException { - final AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), Settings.EMPTY, + final TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), Settings.EMPTY, new AnalysisStempelPlugin()); - TokenFilterFactory tokenizerFactory = analysisService.tokenFilter("polish_stem"); + TokenFilterFactory tokenizerFactory = analysis.tokenFilter.get("polish_stem"); MatcherAssert.assertThat(tokenizerFactory, instanceOf(PolishStemTokenFilterFactory.class)); - Analyzer analyzer = analysisService.analyzer("polish").analyzer(); + Analyzer analyzer = analysis.indexAnalyzers.get("polish").analyzer(); MatcherAssert.assertThat(analyzer, instanceOf(PolishAnalyzer.class)); } } diff --git a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/SimplePolishTokenFilterTests.java b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/SimplePolishTokenFilterTests.java index 3fc12ccdfed..26f02c9df40 100644 --- a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/SimplePolishTokenFilterTests.java +++ b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/SimplePolishTokenFilterTests.java @@ -49,9 +49,9 @@ public class SimplePolishTokenFilterTests extends ESTestCase { Settings settings = Settings.builder() .put("index.analysis.filter.myStemmer.type", "polish_stem") .build(); - AnalysisService analysisService = createAnalysisService(index, settings, new AnalysisStempelPlugin()); + TestAnalysis analysis = createTestAnalysis(index, settings, new AnalysisStempelPlugin()); - TokenFilterFactory filterFactory = analysisService.tokenFilter("myStemmer"); + TokenFilterFactory filterFactory = analysis.tokenFilter.get("myStemmer"); Tokenizer tokenizer = new KeywordTokenizer(); tokenizer.setReader(new StringReader(source)); @@ -65,9 +65,9 @@ public class SimplePolishTokenFilterTests extends ESTestCase { } private void testAnalyzer(String source, String... expected_terms) throws IOException { - AnalysisService analysisService = createAnalysisService(new Index("test", "_na_"), Settings.EMPTY, new AnalysisStempelPlugin()); + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), Settings.EMPTY, new AnalysisStempelPlugin()); - Analyzer analyzer = analysisService.analyzer("polish").analyzer(); + Analyzer analyzer = analysis.indexAnalyzers.get("polish").analyzer(); TokenStream ts = analyzer.tokenStream("test", source); diff --git a/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/Murmur3FieldMapperTests.java b/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/Murmur3FieldMapperTests.java index d1a6c0463fb..e10fdb72ff7 100644 --- a/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/Murmur3FieldMapperTests.java +++ b/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/Murmur3FieldMapperTests.java @@ -59,7 +59,7 @@ public class Murmur3FieldMapperTests extends ESSingleNodeTestCase { Collections.singletonMap(Murmur3FieldMapper.CONTENT_TYPE, new Murmur3FieldMapper.TypeParser()), Collections.emptyMap()); parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(), - indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); } @Override @@ -152,7 +152,7 @@ public class Murmur3FieldMapperTests extends ESSingleNodeTestCase { Settings oldIndexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, oldVersion).build(); IndexService indexService2x = createIndex("test_old", oldIndexSettings); - DocumentMapperParser parser = new DocumentMapperParser(indexService2x.getIndexSettings(), indexService2x.mapperService(), indexService2x.analysisService(), + DocumentMapperParser parser = new DocumentMapperParser(indexService2x.getIndexSettings(), indexService2x.mapperService(), indexService2x.getIndexAnalyzers(), indexService2x.similarityService(), mapperRegistry, indexService2x::newQueryShardContext); DocumentMapper defaultMapper = parser.parse("type", new CompressedXContent(mapping)); diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/plugins/InstallPluginCommandTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/plugins/InstallPluginCommandTests.java index af1f311dd23..6b930c17111 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/plugins/InstallPluginCommandTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/plugins/InstallPluginCommandTests.java @@ -291,6 +291,12 @@ public class InstallPluginCommandTests extends ESTestCase { } } + public void testMissingPluginId() throws IOException { + final Tuple env = createEnv(fs, temp); + final UserException e = expectThrows(UserException.class, () -> installPlugin(null, env.v1())); + assertTrue(e.getMessage(), e.getMessage().contains("plugin id is required")); + } + public void testSomethingWorks() throws Exception { Tuple env = createEnv(fs, temp); Path pluginDir = createPluginDir(temp); diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java index 404d4495894..0171dfb99d1 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java @@ -39,6 +39,7 @@ import org.junit.BeforeClass; import java.io.IOException; import java.nio.file.Path; +import java.util.Collections; import static org.hamcrest.CoreMatchers.either; import static org.hamcrest.CoreMatchers.equalTo; @@ -71,14 +72,14 @@ public class TribeUnitTests extends ESTestCase { .put("cluster.name", "tribe1") .put("node.name", "tribe1_node") .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) - .build()).start(); + .build(), Collections.emptyList()).start(); tribe2 = new TribeClientNode( Settings.builder() .put(baseSettings) .put("cluster.name", "tribe2") .put("node.name", "tribe2_node") .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) - .build()).start(); + .build(), Collections.emptyList()).start(); } @AfterClass diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json index ae5ef206aa4..03b67fd14c2 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json @@ -34,8 +34,10 @@ "description" : "Comma-separated list of column names to display" }, "health": { - "type" : "string", - "description" : "A health status (\"green\", \"yellow\", or \"red\" to filter only indices matching the specified health status" + "type" : "enum", + "options" : ["green","yellow","red"], + "default" : null, + "description" : "A health status (\"green\", \"yellow\", or \"red\" to filter only indices matching the specified health status" }, "help": { "type": "boolean", diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java index bddda142f0c..89fafc74c86 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java @@ -26,9 +26,8 @@ import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; +import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; -import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; @@ -40,7 +39,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.gateway.NoopGatewayAllocator; +import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.util.ArrayList; import java.util.Arrays; @@ -76,13 +75,13 @@ public abstract class ESAllocationTestCase extends ESTestCase { public static MockAllocationService createAllocationService(Settings settings, ClusterSettings clusterSettings, Random random) { return new MockAllocationService(settings, randomAllocationDeciders(settings, clusterSettings, random), - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(settings), EmptyClusterInfoService.INSTANCE); + new TestGatewayAllocator(), new BalancedShardsAllocator(settings), EmptyClusterInfoService.INSTANCE); } public static MockAllocationService createAllocationService(Settings settings, ClusterInfoService clusterInfoService) { return new MockAllocationService(settings, randomAllocationDeciders(settings, EMPTY_CLUSTER_SETTINGS, random()), - NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(settings), clusterInfoService); + new TestGatewayAllocator(), new BalancedShardsAllocator(settings), clusterInfoService); } public static MockAllocationService createAllocationService(Settings settings, GatewayAllocator gatewayAllocator) { @@ -211,10 +210,14 @@ public abstract class ESAllocationTestCase extends ESTestCase { } @Override - public void applyStartedShards(StartedRerouteAllocation allocation) {} + public void applyStartedShards(RoutingAllocation allocation, List startedShards) { + // no-op + } @Override - public void applyFailedShards(FailedRerouteAllocation allocation) {} + public void applyFailedShards(RoutingAllocation allocation, List failedShards) { + // no-op + } @Override public void allocateUnassigned(RoutingAllocation allocation) { diff --git a/test/framework/src/main/java/org/elasticsearch/index/MapperTestUtils.java b/test/framework/src/main/java/org/elasticsearch/index/MapperTestUtils.java index a897de7073a..854260f02a4 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/MapperTestUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/index/MapperTestUtils.java @@ -21,11 +21,9 @@ package org.elasticsearch.index; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; -import org.elasticsearch.index.analysis.AnalysisRegistry; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.IndicesModule; @@ -36,7 +34,7 @@ import java.io.IOException; import java.nio.file.Path; import java.util.Collections; -import static org.elasticsearch.test.ESTestCase.createAnalysisService; +import static org.elasticsearch.test.ESTestCase.createTestAnalysis; public class MapperTestUtils { @@ -56,10 +54,10 @@ public class MapperTestUtils { Settings finalSettings = settingsBuilder.build(); MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", finalSettings); - AnalysisService analysisService = createAnalysisService(indexSettings, finalSettings); + IndexAnalyzers indexAnalyzers = createTestAnalysis(indexSettings, finalSettings).indexAnalyzers; SimilarityService similarityService = new SimilarityService(indexSettings, Collections.emptyMap()); return new MapperService(indexSettings, - analysisService, + indexAnalyzers, similarityService, mapperRegistry, () -> null); diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java index c76d4b03cff..6225f5fa5d0 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java @@ -65,7 +65,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.env.Environment; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; @@ -1044,11 +1044,11 @@ public abstract class AbstractQueryTestCase> IndexScopedSettings indexScopedSettings = settingsModule.getIndexScopedSettings(); idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings, indexScopedSettings); AnalysisModule analysisModule = new AnalysisModule(new Environment(nodeSettings), emptyList()); - AnalysisService analysisService = analysisModule.getAnalysisRegistry().build(idxSettings); + IndexAnalyzers indexAnalyzers = analysisModule.getAnalysisRegistry().build(idxSettings); scriptService = scriptModule.getScriptService(); similarityService = new SimilarityService(idxSettings, Collections.emptyMap()); MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); - mapperService = new MapperService(idxSettings, analysisService, similarityService, mapperRegistry, this::createShardContext); + mapperService = new MapperService(idxSettings, indexAnalyzers, similarityService, mapperRegistry, this::createShardContext); IndicesFieldDataCache indicesFieldDataCache = new IndicesFieldDataCache(nodeSettings, new IndexFieldDataCache.Listener() { }); indexFieldDataService = new IndexFieldDataService(idxSettings, indicesFieldDataCache, diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index adc3e1ec2d2..799fdbf8943 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -55,7 +55,11 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.index.analysis.AnalysisRegistry; +import org.elasticsearch.index.analysis.CharFilterFactory; +import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.analysis.TokenFilterFactory; +import org.elasticsearch.index.analysis.TokenizerFactory; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.indices.IndicesModule; @@ -810,35 +814,37 @@ public abstract class ESTestCase extends LuceneTestCase { } /** - * Creates an AnalysisService with all the default analyzers configured. + * Creates an TestAnalysis with all the default analyzers configured. */ - public static AnalysisService createAnalysisService(Index index, Settings settings, AnalysisPlugin... analysisPlugins) + public static TestAnalysis createTestAnalysis(Index index, Settings settings, AnalysisPlugin... analysisPlugins) throws IOException { Settings nodeSettings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build(); - return createAnalysisService(index, nodeSettings, settings, analysisPlugins); + return createTestAnalysis(index, nodeSettings, settings, analysisPlugins); } /** - * Creates an AnalysisService with all the default analyzers configured. + * Creates an TestAnalysis with all the default analyzers configured. */ - public static AnalysisService createAnalysisService(Index index, Settings nodeSettings, Settings settings, - AnalysisPlugin... analysisPlugins) throws IOException { + public static TestAnalysis createTestAnalysis(Index index, Settings nodeSettings, Settings settings, + AnalysisPlugin... analysisPlugins) throws IOException { Settings indexSettings = Settings.builder().put(settings) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .build(); - return createAnalysisService(IndexSettingsModule.newIndexSettings(index, indexSettings), nodeSettings, analysisPlugins); + return createTestAnalysis(IndexSettingsModule.newIndexSettings(index, indexSettings), nodeSettings, analysisPlugins); } /** - * Creates an AnalysisService with all the default analyzers configured. + * Creates an TestAnalysis with all the default analyzers configured. */ - public static AnalysisService createAnalysisService(IndexSettings indexSettings, Settings nodeSettings, - AnalysisPlugin... analysisPlugins) throws IOException { + public static TestAnalysis createTestAnalysis(IndexSettings indexSettings, Settings nodeSettings, + AnalysisPlugin... analysisPlugins) throws IOException { Environment env = new Environment(nodeSettings); AnalysisModule analysisModule = new AnalysisModule(env, Arrays.asList(analysisPlugins)); - final AnalysisService analysisService = analysisModule.getAnalysisRegistry() - .build(indexSettings); - return analysisService; + AnalysisRegistry analysisRegistry = analysisModule.getAnalysisRegistry(); + return new TestAnalysis(analysisRegistry.build(indexSettings), + analysisRegistry.buildTokenFilterFactories(indexSettings), + analysisRegistry.buildTokenizerFactories(indexSettings), + analysisRegistry.buildCharFilterFactories(indexSettings)); } public static ScriptModule newTestScriptModule() { @@ -868,4 +874,27 @@ public abstract class ESTestCase extends LuceneTestCase { } )); } + + /** + * This cute helper class just holds all analysis building blocks that are used + * to build IndexAnalyzers. This is only for testing since in production we only need the + * result and we don't even expose it there. + */ + public static final class TestAnalysis { + + public final IndexAnalyzers indexAnalyzers; + public final Map tokenFilter; + public final Map tokenizer; + public final Map charFilter; + + public TestAnalysis(IndexAnalyzers indexAnalyzers, + Map tokenFilter, + Map tokenizer, + Map charFilter) { + this.indexAnalyzers = indexAnalyzers; + this.tokenFilter = tokenFilter; + this.tokenizer = tokenizer; + this.charFilter = charFilter; + } + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java index 813686f4a3a..c6a1f64820b 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java @@ -27,7 +27,6 @@ import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.fielddata.IndexFieldDataService; @@ -295,9 +294,6 @@ public class TestSearchContext extends SearchContext { return null; } - @Override - public AnalysisService analysisService() { return indexService.analysisService();} - @Override public SimilarityService similarityService() { return null; diff --git a/test/framework/src/main/java/org/elasticsearch/test/gateway/NoopGatewayAllocator.java b/test/framework/src/main/java/org/elasticsearch/test/gateway/NoopGatewayAllocator.java index e321a98f371..b2b41b31461 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/gateway/NoopGatewayAllocator.java +++ b/test/framework/src/main/java/org/elasticsearch/test/gateway/NoopGatewayAllocator.java @@ -19,12 +19,14 @@ package org.elasticsearch.test.gateway; -import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; -import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.gateway.GatewayAllocator; +import java.util.List; + /** * An allocator used for tests that doesn't do anything */ @@ -37,12 +39,12 @@ public class NoopGatewayAllocator extends GatewayAllocator { } @Override - public void applyStartedShards(StartedRerouteAllocation allocation) { + public void applyStartedShards(RoutingAllocation allocation, List startedShards) { // noop } @Override - public void applyFailedShards(FailedRerouteAllocation allocation) { + public void applyFailedShards(RoutingAllocation allocation, List failedShards) { // noop } diff --git a/test/framework/src/main/java/org/elasticsearch/test/gateway/TestGatewayAllocator.java b/test/framework/src/main/java/org/elasticsearch/test/gateway/TestGatewayAllocator.java new file mode 100644 index 00000000000..5caf4571272 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/test/gateway/TestGatewayAllocator.java @@ -0,0 +1,132 @@ +/* + * 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.gateway; + +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.allocation.FailedShard; +import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.gateway.AsyncShardFetch; +import org.elasticsearch.gateway.GatewayAllocator; +import org.elasticsearch.gateway.PrimaryShardAllocator; +import org.elasticsearch.gateway.ReplicaShardAllocator; +import org.elasticsearch.gateway.TransportNodesListGatewayStartedShards.NodeGatewayStartedShards; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * A gateway allocator implementation that keeps an in memory list of started shard allocation + * that are used as replies to the, normally async, fetch data requests. The in memory list + * is adapted when shards are started and failed. + * + * Nodes leaving and joining the cluster do not change the list of shards the class tracks but + * rather serves as a filter to what is returned by fetch data. Concretely - fetch data will + * only return shards that were started on nodes that are currently part of the cluster. + * + * For now only primary shard related data is fetched. Replica request always get an empty response. + * + * + * This class is useful to use in unit tests that require the functionality of {@link GatewayAllocator} but do + * not have all the infrastructure required to use it. + */ +public class TestGatewayAllocator extends GatewayAllocator { + + Map> knownAllocations = new HashMap<>(); + DiscoveryNodes currentNodes = DiscoveryNodes.EMPTY_NODES; + + PrimaryShardAllocator primaryShardAllocator = new PrimaryShardAllocator(Settings.EMPTY) { + @Override + protected AsyncShardFetch.FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation) { + // for now always return immediately what we know + final ShardId shardId = shard.shardId(); + final Set ignoreNodes = allocation.getIgnoreNodes(shardId); + Map foundShards = knownAllocations.values().stream() + .flatMap(shardMap -> shardMap.values().stream()) + .filter(ks -> ks.shardId().equals(shardId)) + .filter(ks -> ignoreNodes.contains(ks.currentNodeId()) == false) + .filter(ks -> currentNodes.nodeExists(ks.currentNodeId())) + .collect(Collectors.toMap( + routing -> currentNodes.get(routing.currentNodeId()), + routing -> + new NodeGatewayStartedShards( + currentNodes.get(routing.currentNodeId()), -1, routing.allocationId().getId(), routing.primary()))); + + return new AsyncShardFetch.FetchResult<>(shardId, foundShards, Collections.emptySet(), ignoreNodes); + } + }; + + ReplicaShardAllocator replicaShardAllocator = new ReplicaShardAllocator(Settings.EMPTY) { + @Override + protected AsyncShardFetch.FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation) { + // for now, just pretend no node has data + final ShardId shardId = shard.shardId(); + return new AsyncShardFetch.FetchResult<>(shardId, Collections.emptyMap(), Collections.emptySet(), + allocation.getIgnoreNodes(shardId)); + } + }; + + public TestGatewayAllocator() { + super(Settings.EMPTY, null, null); + } + + @Override + public void applyStartedShards(RoutingAllocation allocation, List startedShards) { + currentNodes = allocation.nodes(); + allocation.routingNodes().shards(ShardRouting::active).forEach(this::addKnownAllocation); + } + + @Override + public void applyFailedShards(RoutingAllocation allocation, List failedShards) { + currentNodes = allocation.nodes(); + for (FailedShard failedShard : failedShards) { + final ShardRouting failedRouting = failedShard.getRoutingEntry(); + Map nodeAllocations = knownAllocations.get(failedRouting.currentNodeId()); + if (nodeAllocations != null) { + nodeAllocations.remove(failedRouting.shardId()); + if (nodeAllocations.isEmpty()) { + knownAllocations.remove(failedRouting.currentNodeId()); + } + } + } + } + + @Override + public void allocateUnassigned(RoutingAllocation allocation) { + currentNodes = allocation.nodes(); + innerAllocatedUnassigned(allocation, primaryShardAllocator, replicaShardAllocator); + } + + /** + * manually add a specific shard to the allocations the gateway keeps track of + */ + public void addKnownAllocation(ShardRouting shard) { + knownAllocations.computeIfAbsent(shard.currentNodeId(), id -> new HashMap<>()) + .put(shard.shardId(), shard); + } +}