Merge branch 'master' into feature/rank-eval
This commit is contained in:
commit
92baf14c0a
|
@ -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<ShardRouting> startedShards) {
|
||||
// noop
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyFailedShards(FailedRerouteAllocation allocation) {
|
||||
public void applyFailedShards(RoutingAllocation allocation, List<FailedShard> failedShards) {
|
||||
// noop
|
||||
}
|
||||
|
||||
|
|
|
@ -342,7 +342,6 @@
|
|||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]MergePolicyConfig.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]SearchSlowLog.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]analysis[/\\]AnalysisRegistry.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]analysis[/\\]AnalysisService.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]analysis[/\\]CommonGramsTokenFilterFactory.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]analysis[/\\]CustomAnalyzerProvider.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]analysis[/\\]NumericDoubleAnalyzer.java" checks="LineLength" />
|
||||
|
|
|
@ -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
|
|
@ -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
|
|
@ -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<AnalyzeRe
|
|||
}
|
||||
}
|
||||
final AnalysisRegistry analysisRegistry = indicesService.getAnalysis();
|
||||
return analyze(request, field, analyzer, indexService != null ? indexService.analysisService() : null, analysisRegistry, environment);
|
||||
return analyze(request, field, analyzer, indexService != null ? indexService.getIndexAnalyzers() : null, analysisRegistry, environment);
|
||||
} catch (IOException e) {
|
||||
throw new ElasticsearchException("analysis failed", e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static AnalyzeResponse analyze(AnalyzeRequest request, String field, Analyzer analyzer, AnalysisService analysisService, AnalysisRegistry analysisRegistry, Environment environment) throws IOException {
|
||||
public static AnalyzeResponse analyze(AnalyzeRequest request, String field, Analyzer analyzer, IndexAnalyzers indexAnalyzers, AnalysisRegistry analysisRegistry, Environment environment) throws IOException {
|
||||
|
||||
boolean closeAnalyzer = false;
|
||||
if (analyzer == null && request.analyzer() != null) {
|
||||
if (analysisService == null) {
|
||||
if (indexAnalyzers == null) {
|
||||
analyzer = analysisRegistry.getAnalyzer(request.analyzer());
|
||||
if (analyzer == null) {
|
||||
throw new IllegalArgumentException("failed to find global analyzer [" + request.analyzer() + "]");
|
||||
}
|
||||
} else {
|
||||
analyzer = analysisService.analyzer(request.analyzer());
|
||||
analyzer = indexAnalyzers.get(request.analyzer());
|
||||
if (analyzer == null) {
|
||||
throw new IllegalArgumentException("failed to find analyzer [" + request.analyzer() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
} else if (request.tokenizer() != null) {
|
||||
TokenizerFactory tokenizerFactory = parseTokenizerFactory(request, analysisService, analysisRegistry, environment);
|
||||
final IndexSettings indexSettings = indexAnalyzers == null ? null : indexAnalyzers.getIndexSettings();
|
||||
TokenizerFactory tokenizerFactory = parseTokenizerFactory(request, indexAnalyzers, analysisRegistry, environment);
|
||||
|
||||
TokenFilterFactory[] tokenFilterFactories = new TokenFilterFactory[0];
|
||||
tokenFilterFactories = getTokenFilterFactories(request, analysisService, analysisRegistry, environment, tokenFilterFactories);
|
||||
tokenFilterFactories = getTokenFilterFactories(request, indexSettings, analysisRegistry, environment, tokenFilterFactories);
|
||||
|
||||
CharFilterFactory[] charFilterFactories = new CharFilterFactory[0];
|
||||
charFilterFactories = getCharFilterFactories(request, analysisService, analysisRegistry, environment, charFilterFactories);
|
||||
charFilterFactories = getCharFilterFactories(request, indexSettings, analysisRegistry, environment, charFilterFactories);
|
||||
|
||||
analyzer = new CustomAnalyzer(tokenizerFactory, charFilterFactories, tokenFilterFactories);
|
||||
closeAnalyzer = true;
|
||||
} else if (analyzer == null) {
|
||||
if (analysisService == null) {
|
||||
if (indexAnalyzers == null) {
|
||||
analyzer = analysisRegistry.getAnalyzer("standard");
|
||||
} else {
|
||||
analyzer = analysisService.defaultIndexAnalyzer();
|
||||
analyzer = indexAnalyzers.getDefaultIndexAnalyzer();
|
||||
}
|
||||
}
|
||||
if (analyzer == null) {
|
||||
|
@ -446,7 +447,7 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
|
|||
return extendedAttributes;
|
||||
}
|
||||
|
||||
private static CharFilterFactory[] getCharFilterFactories(AnalyzeRequest request, AnalysisService analysisService, AnalysisRegistry analysisRegistry,
|
||||
private static CharFilterFactory[] getCharFilterFactories(AnalyzeRequest request, IndexSettings indexSettings, AnalysisRegistry analysisRegistry,
|
||||
Environment environment, CharFilterFactory[] charFilterFactories) throws IOException {
|
||||
if (request.charFilters() != null && request.charFilters().size() > 0) {
|
||||
charFilterFactories = new CharFilterFactory[request.charFilters().size()];
|
||||
|
@ -468,19 +469,19 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
|
|||
charFilterFactories[i] = charFilterFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_charfilter_[" + i + "]", settings);
|
||||
} else {
|
||||
AnalysisModule.AnalysisProvider<CharFilterFactory> 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<AnalyzeRe
|
|||
return charFilterFactories;
|
||||
}
|
||||
|
||||
private static TokenFilterFactory[] getTokenFilterFactories(AnalyzeRequest request, AnalysisService analysisService, AnalysisRegistry analysisRegistry,
|
||||
private static TokenFilterFactory[] getTokenFilterFactories(AnalyzeRequest request, IndexSettings indexSettings, AnalysisRegistry analysisRegistry,
|
||||
Environment environment, TokenFilterFactory[] tokenFilterFactories) throws IOException {
|
||||
if (request.tokenFilters() != null && request.tokenFilters().size() > 0) {
|
||||
tokenFilterFactories = new TokenFilterFactory[request.tokenFilters().size()];
|
||||
|
@ -514,19 +515,19 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
|
|||
tokenFilterFactories[i] = tokenFilterFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_tokenfilter_[" + i + "]", settings);
|
||||
} else {
|
||||
AnalysisModule.AnalysisProvider<TokenFilterFactory> 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<AnalyzeRe
|
|||
return tokenFilterFactories;
|
||||
}
|
||||
|
||||
private static TokenizerFactory parseTokenizerFactory(AnalyzeRequest request, AnalysisService analysisService,
|
||||
private static TokenizerFactory parseTokenizerFactory(AnalyzeRequest request, IndexAnalyzers indexAnalzyers,
|
||||
AnalysisRegistry analysisRegistry, Environment environment) throws IOException {
|
||||
TokenizerFactory tokenizerFactory;
|
||||
final AnalyzeRequest.NameOrDefinition tokenizer = request.tokenizer();
|
||||
|
@ -558,19 +559,19 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
|
|||
tokenizerFactory = tokenizerFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_tokenizer", settings);
|
||||
} else {
|
||||
AnalysisModule.AnalysisProvider<TokenizerFactory> 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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -290,7 +290,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
|
|||
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());
|
||||
|
|
|
@ -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<ShardEntry> execute(ClusterState currentState, List<ShardEntry> tasks) throws Exception {
|
||||
BatchResult.Builder<ShardEntry> batchResultBuilder = BatchResult.builder();
|
||||
List<ShardEntry> tasksToBeApplied = new ArrayList<>();
|
||||
List<FailedRerouteAllocation.FailedShard> shardRoutingsToBeApplied = new ArrayList<>();
|
||||
List<FailedRerouteAllocation.StaleShard> staleShardsToBeApplied = new ArrayList<>();
|
||||
List<FailedShard> failedShardsToBeApplied = new ArrayList<>();
|
||||
List<StaleShard> 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<FailedRerouteAllocation.FailedShard> failedShards,
|
||||
List<FailedRerouteAllocation.StaleShard> staleShards) {
|
||||
ClusterState applyFailedShards(ClusterState currentState, List<FailedShard> failedShards, List<StaleShard> 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);
|
||||
|
|
|
@ -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<String, NamedAnalyzer> analyzerMap = new AbstractMap<String, NamedAnalyzer>() {
|
||||
@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<Entry<String, NamedAnalyzer>> 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<MappingMetaData> 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);
|
||||
|
|
|
@ -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<IndexRoutingTable> imple
|
|||
throw new IllegalStateException("shard routing has an index [" + shardRouting.index() + "] that is different " +
|
||||
"from the routing table");
|
||||
}
|
||||
final Set<String> 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<IndexRoutingTable> 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<IndexRoutingTable> 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());
|
||||
}
|
||||
|
|
|
@ -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.</p>
|
||||
*/
|
||||
public ClusterState applyStartedShards(ClusterState clusterState, List<ShardRouting> startedShards) {
|
||||
return applyStartedShards(clusterState, startedShards, true);
|
||||
}
|
||||
|
||||
public ClusterState applyStartedShards(ClusterState clusterState, List<ShardRouting> 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<FailedRerouteAllocation.FailedShard> failedShards) {
|
||||
public ClusterState applyFailedShards(ClusterState clusterState, List<FailedShard> failedShards) {
|
||||
return applyFailedShards(clusterState, failedShards, Collections.emptyList());
|
||||
}
|
||||
|
||||
|
@ -139,8 +130,8 @@ public class AllocationService extends AbstractComponent {
|
|||
* <p>
|
||||
* If the same instance of ClusterState is returned, then no change has been made.</p>
|
||||
*/
|
||||
public ClusterState applyFailedShards(final ClusterState clusterState, List<FailedRerouteAllocation.FailedShard> failedShards,
|
||||
List<FailedRerouteAllocation.StaleShard> staleShards) {
|
||||
public ClusterState applyFailedShards(final ClusterState clusterState, final List<FailedShard> failedShards,
|
||||
final List<StaleShard> 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 + "] ...");
|
||||
}
|
||||
|
||||
|
|
|
@ -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<FailedShard> failedShards;
|
||||
|
||||
public FailedRerouteAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, ClusterState clusterState,
|
||||
List<FailedShard> failedShards, ClusterInfo clusterInfo, long currentNanoTime) {
|
||||
super(deciders, routingNodes, clusterState, clusterInfo, currentNanoTime, false);
|
||||
this.failedShards = failedShards;
|
||||
}
|
||||
|
||||
public List<FailedShard> failedShards() {
|
||||
return failedShards;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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<Index, List<StaleShard>> 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<ShardId, List<StaleShard>> shardEntry : indexEntry.getValue().stream().collect(
|
||||
Collectors.groupingBy(staleShard -> staleShard.shardId)).entrySet()) {
|
||||
Collectors.groupingBy(staleShard -> staleShard.getShardId())).entrySet()) {
|
||||
int shardNumber = shardEntry.getKey().getId();
|
||||
Set<String> oldInSyncAllocations = oldIndexMetaData.inSyncAllocationIds(shardNumber);
|
||||
Set<String> idsToRemove = shardEntry.getValue().stream().map(e -> e.allocationId).collect(Collectors.toSet());
|
||||
Set<String> 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<String> remainingInSyncAllocations = Sets.difference(oldInSyncAllocations, idsToRemove);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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<ShardRouting> startedShards;
|
||||
|
||||
public StartedRerouteAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, ClusterState clusterState,
|
||||
List<ShardRouting> 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<ShardRouting> startedShards() {
|
||||
return startedShards;
|
||||
}
|
||||
}
|
|
@ -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 <tt>SizeUnit</tt> represents size at a given unit of
|
||||
* granularity and provides utility methods to convert across units.
|
||||
* A <tt>SizeUnit</tt> 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());
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -60,6 +60,9 @@ public abstract class AbstractObjectParser<Value, Context extends ParseFieldMatc
|
|||
ValueType type);
|
||||
|
||||
public <T> void declareField(BiConsumer<Value, T> consumer, NoContextParser<T> parser, ParseField parseField, ValueType type) {
|
||||
if (parser == null) {
|
||||
throw new IllegalArgumentException("[parser] is required");
|
||||
}
|
||||
declareField(consumer, (p, c) -> parser.parse(p), parseField, type);
|
||||
}
|
||||
|
||||
|
|
|
@ -103,7 +103,7 @@ public final class ConstructingObjectParser<Value, Context extends ParseFieldMat
|
|||
|
||||
/**
|
||||
* 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 builder A function that builds the object from an array of Objects. Declare this inline with the parser, casting the elements
|
||||
|
@ -113,7 +113,24 @@ public final class ConstructingObjectParser<Value, Context extends ParseFieldMat
|
|||
* allocations.
|
||||
*/
|
||||
public ConstructingObjectParser(String name, Function<Object[], Value> 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<String, Object>} to save on
|
||||
* allocations.
|
||||
*/
|
||||
public ConstructingObjectParser(String name, boolean ignoreUnknownFields, Function<Object[], Value> builder) {
|
||||
objectParser = new ObjectParser<>(name, ignoreUnknownFields, null);
|
||||
this.builder = builder;
|
||||
}
|
||||
|
||||
|
@ -153,6 +170,19 @@ public final class ConstructingObjectParser<Value, Context extends ParseFieldMat
|
|||
|
||||
@Override
|
||||
public <T> void declareField(BiConsumer<Value, T> consumer, ContextParser<Context, T> 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
|
||||
|
|
|
@ -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<Value, Context extends ParseFieldMatcherSupplier
|
|||
private final Map<String, FieldParser> fieldParserMap = new HashMap<>();
|
||||
private final String name;
|
||||
private final Supplier<Value> 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<Value, Context extends ParseFieldMatcherSupplier
|
|||
* @param name the parsers name, used to reference the parser in exceptions and messages.
|
||||
* @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, Supplier<Value> valueSupplier) {
|
||||
public ObjectParser(String name, @Nullable Supplier<Value> 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<Value> valueSupplier) {
|
||||
this.name = name;
|
||||
this.valueSupplier = valueSupplier;
|
||||
this.ignoreUnknownFields = ignoreUnknownFields;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -144,9 +162,13 @@ public final class ObjectParser<Value, Context extends ParseFieldMatcherSupplier
|
|||
if (currentFieldName == null) {
|
||||
throw new IllegalStateException("[" + name + "] no field found");
|
||||
}
|
||||
assert fieldParser != null;
|
||||
fieldParser.assertSupports(name, token, currentFieldName, context.getParseFieldMatcher());
|
||||
parseSub(parser, fieldParser, currentFieldName, value, context);
|
||||
if (fieldParser == null) {
|
||||
assert ignoreUnknownFields : "this should only be possible if configured to ignore known fields";
|
||||
parser.skipChildren(); // noop if parser points to a value, skips children if parser is start object or start array
|
||||
} else {
|
||||
fieldParser.assertSupports(name, token, currentFieldName, context.getParseFieldMatcher());
|
||||
parseSub(parser, fieldParser, currentFieldName, value, context);
|
||||
}
|
||||
fieldParser = null;
|
||||
}
|
||||
}
|
||||
|
@ -169,6 +191,12 @@ public final class ObjectParser<Value, Context extends ParseFieldMatcherSupplier
|
|||
void parse(XContentParser parser, Value value, Context context) throws IOException;
|
||||
}
|
||||
public void declareField(Parser<Value, Context> 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<Value, Context extends ParseFieldMatcherSupplier
|
|||
@Override
|
||||
public <T> void declareField(BiConsumer<Value, T> consumer, ContextParser<Context, T> 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<Value, Context extends ParseFieldMatcherSupplier
|
|||
|
||||
private FieldParser getParser(String fieldName) {
|
||||
FieldParser<Value> parser = fieldParserMap.get(fieldName);
|
||||
if (parser == null) {
|
||||
if (parser == null && false == ignoreUnknownFields) {
|
||||
throw new IllegalArgumentException("[" + name + "] unknown field [" + fieldName + "], parser not found");
|
||||
}
|
||||
return parser;
|
||||
|
|
|
@ -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<ShardRouting> 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<FailedShard> 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
|
||||
|
||||
|
|
|
@ -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<String> ignoreNodes, Set<String> lastActiveAllocationIds,
|
||||
Set<String> ignoreNodes, Set<String> inSyncAllocationIds,
|
||||
FetchResult<NodeGatewayStartedShards> shardState,
|
||||
Logger logger) {
|
||||
LinkedList<NodeGatewayStartedShards> 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 {
|
||||
|
|
|
@ -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<IndexShard> {
|
||||
|
||||
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<IndexingOperationListener> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String, Settings> charFiltersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_CHAR_FILTER);
|
||||
public IndexAnalyzers build(IndexSettings indexSettings) throws IOException {
|
||||
|
||||
final Map<String, CharFilterFactory> charFilterFactories = buildCharFilterFactories(indexSettings);
|
||||
final Map<String, TokenizerFactory> tokenizerFactories = buildTokenizerFactories(indexSettings);
|
||||
final Map<String, TokenFilterFactory> tokenFilterFactories = buildTokenFilterFactories(indexSettings);
|
||||
final Map<String, AnalyzerProvider<?>> analyzierFactories = buildAnalyzerFactories(indexSettings);
|
||||
return build(indexSettings, analyzierFactories, tokenizerFactories, charFilterFactories, tokenFilterFactories);
|
||||
}
|
||||
|
||||
public Map<String, TokenFilterFactory> buildTokenFilterFactories(IndexSettings indexSettings) throws IOException {
|
||||
final Map<String, Settings> tokenFiltersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_FILTER);
|
||||
final Map<String, Settings> tokenizersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_TOKENIZER);
|
||||
final Map<String, Settings> analyzersSettings = indexSettings.getSettings().getGroups("index.analysis.analyzer");
|
||||
|
||||
final Map<String, CharFilterFactory> charFilterFactories = buildMapping(false, "charfilter", indexSettings, charFiltersSettings, charFilters, prebuiltAnalysis.charFilterFactories);
|
||||
final Map<String, TokenizerFactory> tokenizerFactories = buildMapping(false, "tokenizer", indexSettings, tokenizersSettings, tokenizers, prebuiltAnalysis.tokenizerFactories);
|
||||
|
||||
Map<String, AnalysisModule.AnalysisProvider<TokenFilterFactory>> 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<String, TokenFilterFactory> tokenFilterFactories = buildMapping(false, "tokenfilter", indexSettings, tokenFiltersSettings, Collections.unmodifiableMap(tokenFilters), prebuiltAnalysis.tokenFilterFactories);
|
||||
final Map<String, AnalyzerProvider<?>> 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<String, TokenizerFactory> buildTokenizerFactories(IndexSettings indexSettings) throws IOException {
|
||||
final Map<String, Settings> tokenizersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_TOKENIZER);
|
||||
return buildMapping(false, "tokenizer", indexSettings, tokenizersSettings, tokenizers, prebuiltAnalysis.tokenizerFactories);
|
||||
}
|
||||
|
||||
public Map<String, CharFilterFactory> buildCharFilterFactories(IndexSettings indexSettings) throws IOException {
|
||||
final Map<String, Settings> charFiltersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_CHAR_FILTER);
|
||||
return buildMapping(false, "charfilter", indexSettings, charFiltersSettings, charFilters, prebuiltAnalysis.charFilterFactories);
|
||||
}
|
||||
|
||||
public Map<String, AnalyzerProvider<?>> buildAnalyzerFactories(IndexSettings indexSettings) throws IOException {
|
||||
final Map<String, Settings> 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<String, AnalyzerProvider<?>> analyzerProviders,
|
||||
Map<String, TokenizerFactory> tokenizerFactoryFactories,
|
||||
Map<String, CharFilterFactory> charFilterFactoryFactories,
|
||||
Map<String, TokenFilterFactory> tokenFilterFactoryFactories) {
|
||||
|
||||
Index index = indexSettings.getIndex();
|
||||
analyzerProviders = new HashMap<>(analyzerProviders);
|
||||
Logger logger = Loggers.getLogger(getClass(), indexSettings.getSettings());
|
||||
DeprecationLogger deprecationLogger = new DeprecationLogger(logger);
|
||||
Map<String, NamedAnalyzer> analyzerAliases = new HashMap<>();
|
||||
Map<String, NamedAnalyzer> analyzers = new HashMap<>();
|
||||
for (Map.Entry<String, AnalyzerProvider<?>> entry : analyzerProviders.entrySet()) {
|
||||
processAnalyzerFactory(deprecationLogger, indexSettings, entry.getKey(), entry.getValue(), analyzerAliases, analyzers,
|
||||
tokenFilterFactoryFactories, charFilterFactoryFactories, tokenizerFactoryFactories);
|
||||
}
|
||||
for (Map.Entry<String, NamedAnalyzer> 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<String, NamedAnalyzer> 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<String, NamedAnalyzer> analyzerAliases,
|
||||
Map<String, NamedAnalyzer> analyzers, Map<String, TokenFilterFactory> tokenFilters,
|
||||
Map<String, CharFilterFactory> charFilters, Map<String, TokenizerFactory> 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<String> 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() + "]");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String, NamedAnalyzer> analyzers;
|
||||
private final Map<String, TokenizerFactory> tokenizers;
|
||||
private final Map<String, CharFilterFactory> charFilters;
|
||||
private final Map<String, TokenFilterFactory> tokenFilters;
|
||||
|
||||
private final NamedAnalyzer defaultIndexAnalyzer;
|
||||
private final NamedAnalyzer defaultSearchAnalyzer;
|
||||
private final NamedAnalyzer defaultSearchQuoteAnalyzer;
|
||||
|
||||
public AnalysisService(IndexSettings indexSettings,
|
||||
Map<String, AnalyzerProvider<?>> analyzerProviders,
|
||||
Map<String, TokenizerFactory> tokenizerFactoryFactories,
|
||||
Map<String, CharFilterFactory> charFilterFactoryFactories,
|
||||
Map<String, TokenFilterFactory> tokenFilterFactoryFactories) {
|
||||
super(indexSettings);
|
||||
this.tokenizers = unmodifiableMap(tokenizerFactoryFactories);
|
||||
this.charFilters = unmodifiableMap(charFilterFactoryFactories);
|
||||
this.tokenFilters = unmodifiableMap(tokenFilterFactoryFactories);
|
||||
analyzerProviders = new HashMap<>(analyzerProviders);
|
||||
|
||||
Map<String, NamedAnalyzer> analyzerAliases = new HashMap<>();
|
||||
Map<String, NamedAnalyzer> analyzers = new HashMap<>();
|
||||
for (Map.Entry<String, AnalyzerProvider<?>> entry : analyzerProviders.entrySet()) {
|
||||
processAnalyzerFactory(entry.getKey(), entry.getValue(), analyzerAliases, analyzers);
|
||||
}
|
||||
for (Map.Entry<String, NamedAnalyzer> 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<String, NamedAnalyzer> 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<String, NamedAnalyzer> analyzerAliases, Map<String, NamedAnalyzer> 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<String> 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);
|
||||
}
|
||||
}
|
|
@ -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<Custom
|
|||
this.analyzerSettings = settings;
|
||||
}
|
||||
|
||||
public void build(AnalysisService analysisService) {
|
||||
public void build(final Map<String, TokenizerFactory> tokenizers, final Map<String, CharFilterFactory> charFilters,
|
||||
final Map<String, TokenFilterFactory> 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<CharFilterFactory> charFilters = new ArrayList<>();
|
||||
List<CharFilterFactory> 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<TokenFilterFactory> tokenFilters = new ArrayList<>();
|
||||
List<TokenFilterFactory> 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<Custom
|
|||
|
||||
int offsetGap = analyzerSettings.getAsInt("offset_gap", -1);;
|
||||
this.customAnalyzer = new CustomAnalyzer(tokenizer,
|
||||
charFilters.toArray(new CharFilterFactory[charFilters.size()]),
|
||||
tokenFilters.toArray(new TokenFilterFactory[tokenFilters.size()]),
|
||||
charFiltersList.toArray(new CharFilterFactory[charFiltersList.size()]),
|
||||
tokenFilterList.toArray(new TokenFilterFactory[tokenFilterList.size()]),
|
||||
positionIncrementGap,
|
||||
offsetGap
|
||||
);
|
||||
|
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* 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.util.IOUtils;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* IndexAnalyzers contains a name to analyzer mapping for a specific index.
|
||||
* This class only holds analyzers that are explicitly configured for an index and doesn't allow
|
||||
* access to individual tokenizers, char or token filter.
|
||||
*
|
||||
* @see AnalysisRegistry
|
||||
*/
|
||||
public final class IndexAnalyzers extends AbstractIndexComponent implements Closeable {
|
||||
private final NamedAnalyzer defaultIndexAnalyzer;
|
||||
private final NamedAnalyzer defaultSearchAnalyzer;
|
||||
private final NamedAnalyzer defaultSearchQuoteAnalyzer;
|
||||
private final Map<String, NamedAnalyzer> analyzers;
|
||||
private final IndexSettings indexSettings;
|
||||
|
||||
public IndexAnalyzers(IndexSettings indexSettings, NamedAnalyzer defaultIndexAnalyzer, NamedAnalyzer defaultSearchAnalyzer,
|
||||
NamedAnalyzer defaultSearchQuoteAnalyzer, Map<String, NamedAnalyzer> 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 <code>null</code> 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;
|
||||
}
|
||||
|
||||
}
|
|
@ -106,9 +106,9 @@ public class AllFieldMapper extends MetadataFieldMapper {
|
|||
public MetadataFieldMapper.Builder parse(String name, Map<String, Object> 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
|
||||
|
|
|
@ -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 + "]");
|
||||
}
|
||||
|
|
|
@ -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 + "]");
|
||||
}
|
||||
|
|
|
@ -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<String, ObjectMapper> builder = new HashMap<>();
|
||||
for (ObjectMapper objectMapper : newObjectMappers) {
|
||||
|
|
|
@ -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<QueryShardContext> queryShardContextSupplier;
|
||||
|
||||
|
@ -56,12 +55,12 @@ public class DocumentMapperParser {
|
|||
private final Map<String, Mapper.TypeParser> typeParsers;
|
||||
private final Map<String, MetadataFieldMapper.TypeParser> rootTypeParsers;
|
||||
|
||||
public DocumentMapperParser(IndexSettings indexSettings, MapperService mapperService, AnalysisService analysisService,
|
||||
public DocumentMapperParser(IndexSettings indexSettings, MapperService mapperService, IndexAnalyzers indexAnalyzers,
|
||||
SimilarityService similarityService, MapperRegistry mapperRegistry,
|
||||
Supplier<QueryShardContext> 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 {
|
||||
|
|
|
@ -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 + "]");
|
||||
}
|
||||
|
|
|
@ -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<Mapper> {
|
|||
|
||||
private final String type;
|
||||
|
||||
private final AnalysisService analysisService;
|
||||
private final IndexAnalyzers indexAnalyzers;
|
||||
|
||||
private final Function<String, SimilarityProvider> similarityLookupService;
|
||||
|
||||
|
@ -99,11 +99,11 @@ public abstract class Mapper implements ToXContent, Iterable<Mapper> {
|
|||
|
||||
private final QueryShardContext queryShardContext;
|
||||
|
||||
public ParserContext(String type, AnalysisService analysisService, Function<String, SimilarityProvider> similarityLookupService,
|
||||
public ParserContext(String type, IndexAnalyzers indexAnalyzers, Function<String, SimilarityProvider> similarityLookupService,
|
||||
MapperService mapperService, Function<String, TypeParser> 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<Mapper> {
|
|||
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<Mapper> {
|
|||
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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<QueryShardContext> 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() {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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")) {
|
||||
|
|
|
@ -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<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
|
||||
Map.Entry<String, Object> entry = iterator.next();
|
||||
|
|
|
@ -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 + "]");
|
||||
}
|
||||
|
|
|
@ -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 + "]");
|
||||
}
|
||||
|
|
|
@ -383,7 +383,7 @@ public class CommonTermsQueryBuilder extends AbstractQueryBuilder<CommonTermsQue
|
|||
analyzerObj = context.getMapperService().searchAnalyzer();
|
||||
}
|
||||
} else {
|
||||
analyzerObj = context.getMapperService().analysisService().analyzer(analyzer);
|
||||
analyzerObj = context.getMapperService().getIndexAnalyzers().get(analyzer);
|
||||
if (analyzerObj == null) {
|
||||
throw new QueryShardException(context, "[common] analyzer [" + analyzer + "] not found");
|
||||
}
|
||||
|
|
|
@ -164,7 +164,7 @@ public class MatchPhrasePrefixQueryBuilder extends AbstractQueryBuilder<MatchPhr
|
|||
@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");
|
||||
}
|
||||
|
||||
|
|
|
@ -140,7 +140,7 @@ public class MatchPhraseQueryBuilder extends AbstractQueryBuilder<MatchPhraseQue
|
|||
@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");
|
||||
}
|
||||
|
||||
|
|
|
@ -444,7 +444,7 @@ public class MatchQueryBuilder extends AbstractQueryBuilder<MatchQueryBuilder> {
|
|||
@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");
|
||||
}
|
||||
|
||||
|
|
|
@ -147,7 +147,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
|
|||
*/
|
||||
public static final class Item implements ToXContent, Writeable {
|
||||
public static final Item[] EMPTY_ARRAY = new Item[0];
|
||||
|
||||
|
||||
public interface Field {
|
||||
ParseField INDEX = new ParseField("_index");
|
||||
ParseField TYPE = new ParseField("_type");
|
||||
|
@ -1021,7 +1021,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
|
|||
}
|
||||
|
||||
// set analyzer
|
||||
Analyzer analyzerObj = context.getAnalysisService().analyzer(analyzer);
|
||||
Analyzer analyzerObj = context.getIndexAnalyzers().get(analyzer);
|
||||
if (analyzerObj == null) {
|
||||
analyzerObj = context.getMapperService().searchAnalyzer();
|
||||
}
|
||||
|
|
|
@ -708,7 +708,7 @@ public class MultiMatchQueryBuilder extends AbstractQueryBuilder<MultiMatchQuery
|
|||
protected Query doToQuery(QueryShardContext context) throws IOException {
|
||||
MultiMatchQuery multiMatchQuery = new MultiMatchQuery(context);
|
||||
if (analyzer != null) {
|
||||
if (context.getAnalysisService().analyzer(analyzer) == null) {
|
||||
if (context.getIndexAnalyzers().get(analyzer) == null) {
|
||||
throw new QueryShardException(context, "[" + NAME + "] analyzer [" + analyzer + "] not found");
|
||||
}
|
||||
multiMatchQuery.setAnalyzer(analyzer);
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.elasticsearch.common.Strings;
|
|||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
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.IndexFieldData;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataService;
|
||||
|
@ -116,8 +116,8 @@ public class QueryShardContext extends QueryRewriteContext {
|
|||
this.isFilter = false;
|
||||
}
|
||||
|
||||
public AnalysisService getAnalysisService() {
|
||||
return mapperService.analysisService();
|
||||
public IndexAnalyzers getIndexAnalyzers() {
|
||||
return mapperService.getIndexAnalyzers();
|
||||
}
|
||||
|
||||
public Similarity getSearchSimilarity() {
|
||||
|
|
|
@ -868,14 +868,14 @@ public class QueryStringQueryBuilder extends AbstractQueryBuilder<QueryStringQue
|
|||
if (analyzer == null) {
|
||||
qpSettings.defaultAnalyzer(context.getMapperService().searchAnalyzer());
|
||||
} else {
|
||||
NamedAnalyzer namedAnalyzer = context.getAnalysisService().analyzer(analyzer);
|
||||
NamedAnalyzer namedAnalyzer = context.getIndexAnalyzers().get(analyzer);
|
||||
if (namedAnalyzer == null) {
|
||||
throw new QueryShardException(context, "[query_string] analyzer [" + analyzer + "] not found");
|
||||
}
|
||||
qpSettings.forceAnalyzer(namedAnalyzer);
|
||||
}
|
||||
if (quoteAnalyzer != null) {
|
||||
NamedAnalyzer namedAnalyzer = context.getAnalysisService().analyzer(quoteAnalyzer);
|
||||
NamedAnalyzer namedAnalyzer = context.getIndexAnalyzers().get(quoteAnalyzer);
|
||||
if (namedAnalyzer == null) {
|
||||
throw new QueryShardException(context, "[query_string] quote_analyzer [" + quoteAnalyzer + "] not found");
|
||||
}
|
||||
|
|
|
@ -355,7 +355,7 @@ public class SimpleQueryStringBuilder extends AbstractQueryBuilder<SimpleQuerySt
|
|||
if (analyzer == null) {
|
||||
luceneAnalyzer = context.getMapperService().searchAnalyzer();
|
||||
} else {
|
||||
luceneAnalyzer = context.getAnalysisService().analyzer(analyzer);
|
||||
luceneAnalyzer = context.getIndexAnalyzers().get(analyzer);
|
||||
if (luceneAnalyzer == null) {
|
||||
throw new QueryShardException(context, "[" + SimpleQueryStringBuilder.NAME + "] analyzer [" + analyzer
|
||||
+ "] not found");
|
||||
|
|
|
@ -204,7 +204,7 @@ public class MatchQuery {
|
|||
}
|
||||
return context.getMapperService().searchAnalyzer();
|
||||
} else {
|
||||
Analyzer analyzer = context.getMapperService().analysisService().analyzer(this.analyzer);
|
||||
Analyzer analyzer = context.getMapperService().getIndexAnalyzers().get(this.analyzer);
|
||||
if (analyzer == null) {
|
||||
throw new IllegalArgumentException("No analyzer found for [" + this.analyzer + "]");
|
||||
}
|
||||
|
|
|
@ -214,12 +214,12 @@ public class TermVectorsService {
|
|||
MapperService mapperService = indexShard.mapperService();
|
||||
Analyzer analyzer;
|
||||
if (perFieldAnalyzer != null && perFieldAnalyzer.containsKey(field)) {
|
||||
analyzer = mapperService.analysisService().analyzer(perFieldAnalyzer.get(field).toString());
|
||||
analyzer = mapperService.getIndexAnalyzers().get(perFieldAnalyzer.get(field).toString());
|
||||
} else {
|
||||
analyzer = mapperService.fullName(field).indexAnalyzer();
|
||||
}
|
||||
if (analyzer == null) {
|
||||
analyzer = mapperService.analysisService().defaultIndexAnalyzer();
|
||||
analyzer = mapperService.getIndexAnalyzers().getDefaultIndexAnalyzer();
|
||||
}
|
||||
return analyzer;
|
||||
}
|
||||
|
|
|
@ -315,7 +315,7 @@ public class Node implements Closeable {
|
|||
final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool);
|
||||
clusterService.add(scriptModule.getScriptService());
|
||||
resourcesToClose.add(clusterService);
|
||||
final TribeService tribeService = new TribeService(settings, clusterService, nodeEnvironment.nodeId());
|
||||
final TribeService tribeService = new TribeService(settings, clusterService, nodeEnvironment.nodeId(), classpathPlugins);
|
||||
resourcesToClose.add(tribeService);
|
||||
final IngestService ingestService = new IngestService(settings, threadPool, this.environment,
|
||||
scriptModule.getScriptService(), analysisModule.getAnalysisRegistry(), pluginsService.filterPlugins(IngestPlugin.class));
|
||||
|
|
|
@ -192,6 +192,9 @@ class InstallPluginCommand extends SettingCommand {
|
|||
|
||||
// pkg private for testing
|
||||
void execute(Terminal terminal, String pluginId, boolean isBatch, Map<String, String> 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) {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -319,7 +319,7 @@ public abstract class SuggestionBuilder<T extends SuggestionBuilder<T>> 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");
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
|
|
@ -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.<Class<? extends Plugin>>emptyList());
|
||||
TribeClientNode(Settings settings, Collection<Class<? extends Plugin>> classpathPlugins) {
|
||||
super(new Environment(settings), classpathPlugins);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<Node> nodes = new CopyOnWriteArrayList<>();
|
||||
|
||||
public TribeService(Settings settings, ClusterService clusterService, final String tribeNodeId) {
|
||||
public TribeService(Settings settings, ClusterService clusterService, final String tribeNodeId,
|
||||
Collection<Class<? extends Plugin>> classpathPlugins) {
|
||||
super(settings);
|
||||
this.clusterService = clusterService;
|
||||
Map<String, Settings> 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<String, Settings> 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);
|
||||
|
|
|
@ -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<String> nodes = internalCluster().startNodesAsync(3).get();
|
||||
|
|
|
@ -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<FailedRerouteAllocation.FailedShard> failedShards = Collections.singletonList(
|
||||
new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom" + i,
|
||||
List<FailedShard> 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)));
|
||||
|
|
|
@ -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("<p>the qu1ck brown fox</p>");
|
||||
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("<ALPHANUM>", 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<AnalyzeResponse.AnalyzeToken> 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<AnalyzeResponse.AnalyzeToken> tokens = analyze.getTokens();
|
||||
int default_hash_count = 1;
|
||||
int default_bucket_size = 512;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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<ShardStateAction.ShardEntry> nonExistentTasks = createNonExistentShards(currentState, reason);
|
||||
ShardStateAction.ShardFailedClusterStateTaskExecutor failingExecutor = new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocationService, null, logger) {
|
||||
@Override
|
||||
ClusterState applyFailedShards(ClusterState currentState, List<FailedRerouteAllocation.FailedShard> failedShards,
|
||||
List<FailedRerouteAllocation.StaleShard> staleShards) {
|
||||
ClusterState applyFailedShards(ClusterState currentState, List<FailedShard> failedShards, List<StaleShard> staleShards) {
|
||||
throw new RuntimeException("simulated applyFailedShards failure");
|
||||
}
|
||||
};
|
||||
|
|
|
@ -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<ClusterHealthResponse> 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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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<FailedRerouteAllocation.FailedShard> failedShards = new ArrayList<>();
|
||||
List<FailedShard> 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()));
|
||||
|
|
|
@ -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<String> activeAllocations = shardTable.activeShards().stream().map(
|
||||
Set<String> 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();
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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<DiscoveryNode, Float> weighShard(RoutingAllocation allocation, ShardRouting shard) {
|
||||
return new HashMap<DiscoveryNode, Float>();
|
||||
|
@ -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<ShardRouting> startedShards) {
|
||||
// noop
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyFailedShards(RoutingAllocation allocation, List<FailedShard> failedShards) {
|
||||
// noop
|
||||
}
|
||||
@Override
|
||||
public void allocateUnassigned(RoutingAllocation allocation) {
|
||||
// noop
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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<AllocationDecider> deciders) {
|
||||
return new AllocationService(settings,
|
||||
new AllocationDeciders(settings, deciders),
|
||||
NoopGatewayAllocator.INSTANCE,
|
||||
new TestGatewayAllocator(),
|
||||
new BalancedShardsAllocator(settings),
|
||||
EmptyClusterInfoService.INSTANCE);
|
||||
}
|
||||
|
|
|
@ -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<FailedRerouteAllocation.FailedShard> failedShards = new ArrayList<>();
|
||||
ArrayList<FailedShard> failedShards = new ArrayList<>();
|
||||
RoutingNodes routingNodes = clusterState.getRoutingNodes();
|
||||
Set<String> failedNodes = new HashSet<>();
|
||||
Set<ShardRouting> 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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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<FailedRerouteAllocation.FailedShard> failedShards = Collections.singletonList(
|
||||
new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom" + i,
|
||||
List<FailedShard> 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<FailedRerouteAllocation.FailedShard> failedShards = Collections.singletonList(
|
||||
new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom",
|
||||
List<FailedShard> 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<FailedRerouteAllocation.FailedShard> failedShards = Collections.singletonList(
|
||||
new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom" + i,
|
||||
List<FailedShard> 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<FailedRerouteAllocation.FailedShard> failedShards = Collections.singletonList(
|
||||
new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "boom",
|
||||
List<FailedShard> 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<FailedRerouteAllocation.FailedShard> failedShards = Collections.singletonList(
|
||||
new FailedRerouteAllocation.FailedShard(routingTable.index("idx").shard(0).shards().get(0), "ZOOOMG",
|
||||
List<FailedShard> 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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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));
|
||||
|
||||
}
|
||||
|
|
|
@ -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.<ShardRouting>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));
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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<IndexMetaData> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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) + "]"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<Void, ParseFieldMatcherSupplier> 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<ParseFieldMatcherSupplier, Object>) null,
|
||||
new ParseField("test"), ObjectParser.ValueType.STRING));
|
||||
assertEquals("[parser] is required", e.getMessage());
|
||||
e = expectThrows(IllegalArgumentException.class, () -> objectParser.declareField(
|
||||
(o, v) -> {}, (NoContextParser<Object>) 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<TestStruct, ParseFieldMatcherSupplier> 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;
|
||||
|
|
|
@ -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<Void, ParseFieldMatcherSupplier> 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<ParseFieldMatcherSupplier, Object>) null,
|
||||
new ParseField("test"), ObjectParser.ValueType.STRING));
|
||||
assertEquals("[parser] is required", e.getMessage());
|
||||
e = expectThrows(IllegalArgumentException.class, () -> objectParser.declareField(
|
||||
(o, v) -> {}, (NoContextParser<Object>) 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<StaticTestStruct, ParseFieldMatcherSupplier> 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<TestStruct, ParseFieldMatcherSupplier> 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<TestStruct, ParseFieldMatcherSupplier> 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<TestStruct, ParseFieldMatcherSupplier> 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<NamedObjectHolder, ParseFieldMatcherSupplier> PARSER = new ObjectParser<>("named_object_holder",
|
||||
NamedObjectHolder::new);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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<ShardRouting> 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<ShardRouting> 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)
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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<String, AnalyzerProvider<?>> 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();
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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", "<b>hello</b>!"), 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"});
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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"};
|
||||
|
|
|
@ -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[] {"<NUM>", "<SOMETHINGELSE>"})
|
||||
.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"};
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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));
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue